mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge branch 'master' into better-range-reader
This commit is contained in:
commit
b07fb1ddda
3
.gitmodules
vendored
3
.gitmodules
vendored
@ -1,6 +1,3 @@
|
||||
[submodule "contrib/zookeeper"]
|
||||
path = contrib/zookeeper
|
||||
url = https://github.com/ClickHouse-Extras/zookeeper.git
|
||||
[submodule "contrib/poco"]
|
||||
path = contrib/poco
|
||||
url = https://github.com/ClickHouse-Extras/poco
|
||||
|
@ -258,7 +258,6 @@ include (cmake/find_sparsehash.cmake)
|
||||
include (cmake/find_rt.cmake)
|
||||
include (cmake/find_execinfo.cmake)
|
||||
include (cmake/find_readline_edit.cmake)
|
||||
include (cmake/find_zookeeper.cmake)
|
||||
include (cmake/find_re2.cmake)
|
||||
include (cmake/find_rdkafka.cmake)
|
||||
include (cmake/find_capnp.cmake)
|
||||
|
@ -1,5 +1,6 @@
|
||||
# ClickHouse
|
||||
ClickHouse is an open-source column-oriented database management system that allows generating analytical data reports in real time.
|
||||
|
||||
[Read more...](https://clickhouse.yandex/)
|
||||
Learn more about ClickHouse at [https://clickhouse.yandex/](https://clickhouse.yandex/)
|
||||
|
||||
[![Build Status](https://travis-ci.org/yandex/ClickHouse.svg?branch=master)](https://travis-ci.org/yandex/ClickHouse)
|
||||
|
@ -49,7 +49,8 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY)
|
||||
set (Poco_DataODBC_INCLUDE_DIRS "${ClickHouse_SOURCE_DIR}/contrib/poco/Data/ODBC/include/")
|
||||
endif ()
|
||||
|
||||
if (OPENSSL_FOUND)
|
||||
# TODO! fix internal ssl
|
||||
if (OPENSSL_FOUND AND NOT USE_INTERNAL_SSL_LIBRARY)
|
||||
set (Poco_NetSSL_FOUND 1)
|
||||
set (Poco_NetSSL_LIBRARY PocoNetSSL)
|
||||
set (Poco_Crypto_LIBRARY PocoCrypto)
|
||||
|
@ -1,35 +0,0 @@
|
||||
if (ARCH_ARM)
|
||||
# bundled have some asm broken for arm, use package libzookeeper-mt-dev
|
||||
set(USE_INTERNAL_ZOOKEEPER_LIBRARY 0 CACHE BOOL "")
|
||||
endif ()
|
||||
|
||||
option (USE_INTERNAL_ZOOKEEPER_LIBRARY "Set to FALSE to use system zookeeper library instead of bundled" ${NOT_UNBUNDLED})
|
||||
|
||||
if (USE_INTERNAL_ZOOKEEPER_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/zookeeper/src/c/CMakeLists.txt")
|
||||
message (WARNING "submodule contrib/zookeeper is missing. to fix try run: \n git submodule update --init --recursive")
|
||||
set (USE_INTERNAL_ZOOKEEPER_LIBRARY 0)
|
||||
endif ()
|
||||
|
||||
if (NOT USE_INTERNAL_ZOOKEEPER_LIBRARY)
|
||||
find_library (ZOOKEEPER_LIBRARY zookeeper_mt)
|
||||
find_path (ZOOKEEPER_INCLUDE_DIR NAMES zookeeper/zookeeper.h PATHS ${ZOOKEEPER_INCLUDE_PATHS})
|
||||
set(ZOOKEEPER_INCLUDE_DIR "${ZOOKEEPER_INCLUDE_DIR}/zookeeper")
|
||||
endif ()
|
||||
|
||||
if (ZOOKEEPER_LIBRARY AND ZOOKEEPER_INCLUDE_DIR)
|
||||
else ()
|
||||
set (USE_INTERNAL_ZOOKEEPER_LIBRARY 1)
|
||||
set(WANT_CPPUNIT 0 CACHE BOOL "")
|
||||
set (ZOOKEEPER_LIBRARY zookeeper)
|
||||
endif ()
|
||||
|
||||
message (STATUS "Using zookeeper: ${ZOOKEEPER_INCLUDE_DIR} : ${ZOOKEEPER_LIBRARY}")
|
||||
|
||||
|
||||
# how to make clickhouse branch of https://github.com/ClickHouse-Extras/zookeeper.git :
|
||||
# git remote add upstream https://github.com/apache/zookeeper.git
|
||||
# git checkhout upstream/master
|
||||
# git branch -D clickhouse
|
||||
# git checkout -b clickhouse
|
||||
# git merge clickhouse_misc
|
||||
# git merge clickhouse_706
|
24
contrib/CMakeLists.txt
vendored
24
contrib/CMakeLists.txt
vendored
@ -25,10 +25,6 @@ if (USE_INTERNAL_DOUBLE_CONVERSION_LIBRARY)
|
||||
add_subdirectory (double-conversion)
|
||||
endif ()
|
||||
|
||||
if (USE_INTERNAL_ZOOKEEPER_LIBRARY)
|
||||
add_subdirectory (zookeeper/src/c)
|
||||
endif ()
|
||||
|
||||
if (USE_INTERNAL_CITYHASH_LIBRARY)
|
||||
add_subdirectory (libcityhash)
|
||||
endif ()
|
||||
@ -100,13 +96,17 @@ if (USE_INTERNAL_RDKAFKA_LIBRARY)
|
||||
mark_as_advanced (ZLIB_INCLUDE_DIR)
|
||||
|
||||
if (USE_INTERNAL_SSL_LIBRARY)
|
||||
add_library(bundled-ssl ALIAS ${OPENSSL_SSL_LIBRARY})
|
||||
set (WITH_BUNDLED_SSL 1)
|
||||
if (MAKE_STATIC_LIBRARIES)
|
||||
add_library(bundled-ssl ALIAS ${OPENSSL_SSL_LIBRARY})
|
||||
set (WITH_BUNDLED_SSL 1 CACHE INTERNAL "")
|
||||
else ()
|
||||
set (WITH_SSL 0 CACHE INTERNAL "")
|
||||
endif ()
|
||||
endif ()
|
||||
|
||||
add_subdirectory (librdkafka)
|
||||
|
||||
if (USE_INTERNAL_SSL_LIBRARY)
|
||||
if (USE_INTERNAL_SSL_LIBRARY AND MAKE_STATIC_LIBRARIES)
|
||||
target_include_directories(rdkafka PRIVATE BEFORE ${OPENSSL_INCLUDE_DIR})
|
||||
endif ()
|
||||
target_include_directories(rdkafka PRIVATE BEFORE ${ZLIB_INCLUDE_DIR})
|
||||
@ -127,16 +127,18 @@ endif ()
|
||||
|
||||
|
||||
if (USE_INTERNAL_POCO_LIBRARY)
|
||||
set (ALLOW_DUPLICATE_CUSTOM_TARGETS 1)
|
||||
set (save_CMAKE_CXX_FLAGS ${CMAKE_CXX_FLAGS})
|
||||
set (save_CMAKE_C_FLAGS ${CMAKE_C_FLAGS})
|
||||
set (_save ${ENABLE_TESTS})
|
||||
set (ENABLE_TESTS 0)
|
||||
set (CMAKE_DISABLE_FIND_PACKAGE_ZLIB 1)
|
||||
if (USE_INTERNAL_SSL_LIBRARY)
|
||||
set (DISABLE_INTERNAL_OPENSSL 1)
|
||||
set (ENABLE_NETSSL 0) # TODO!
|
||||
set (ENABLE_CRYPTO 0) # TODO!
|
||||
set (DISABLE_INTERNAL_OPENSSL 1 CACHE INTERNAL "")
|
||||
set (ENABLE_NETSSL 0 CACHE INTERNAL "") # TODO!
|
||||
set (ENABLE_CRYPTO 0 CACHE INTERNAL "") # TODO!
|
||||
endif ()
|
||||
if (MSVC)
|
||||
set (ENABLE_DATA_ODBC 0 CACHE INTERNAL "") # TODO (build fail)
|
||||
endif ()
|
||||
add_subdirectory (poco)
|
||||
unset (CMAKE_DISABLE_FIND_PACKAGE_ZLIB)
|
||||
|
2
contrib/boost
vendored
2
contrib/boost
vendored
@ -1 +1 @@
|
||||
Subproject commit eb5943711e88d1008583e6ae3720a5489313d02e
|
||||
Subproject commit 5121cc9d0375c7b81b24b6087a51684e6cd62ded
|
@ -5,7 +5,11 @@
|
||||
/* #undef AC_APPLE_UNIVERSAL_BUILD */
|
||||
|
||||
/* Define to 1 if the compiler supports __builtin_expect. */
|
||||
#if _MSC_VER
|
||||
#define HAVE_BUILTIN_EXPECT 0
|
||||
#else
|
||||
#define HAVE_BUILTIN_EXPECT 1
|
||||
#endif
|
||||
|
||||
/* Define to 1 if you have the <dlfcn.h> header file. */
|
||||
#define HAVE_DLFCN_H 1
|
||||
|
@ -2,4 +2,8 @@ add_library(farmhash
|
||||
farmhash.cc
|
||||
farmhash.h)
|
||||
|
||||
if (MSVC)
|
||||
target_compile_definitions (farmhash PRIVATE FARMHASH_NO_BUILTIN_EXPECT=1)
|
||||
endif ()
|
||||
|
||||
target_include_directories (farmhash PUBLIC ${CMAKE_CURRENT_BINARY_DIR})
|
||||
|
@ -13,7 +13,11 @@
|
||||
/* #undef ENABLE_SIZED_DELETE */
|
||||
|
||||
/* Define to 1 if compiler supports __builtin_expect */
|
||||
#if _MSC_VER
|
||||
#define HAVE_BUILTIN_EXPECT 0
|
||||
#else
|
||||
#define HAVE_BUILTIN_EXPECT 1
|
||||
#endif
|
||||
|
||||
/* Define to 1 if compiler supports __builtin_stack_pointer */
|
||||
/* #undef HAVE_BUILTIN_STACK_POINTER */
|
||||
|
2
contrib/poco
vendored
2
contrib/poco
vendored
@ -1 +1 @@
|
||||
Subproject commit 8238852d7ab2a4abdf87adff233b3b83686f4fe4
|
||||
Subproject commit a107b0c9cee109fe0abfbf509df3c78a1e0c05fa
|
1
contrib/zookeeper
vendored
1
contrib/zookeeper
vendored
@ -1 +0,0 @@
|
||||
Subproject commit 438afae5af36c5be9c82d074f43a9bb19e0797c0
|
202
contrib/zookeeper-cmake/CMakeLists.txt
Normal file
202
contrib/zookeeper-cmake/CMakeLists.txt
Normal file
@ -0,0 +1,202 @@
|
||||
# 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()
|
@ -8,7 +8,7 @@
|
||||
# Требуется clang, желательно наиболее свежий (trunk).
|
||||
#
|
||||
# Используется при сборке пакетов.
|
||||
# Заголовочные файлы записываются в пакет clickhouse-server-base, в директорию /usr/share/clickhouse/headers.
|
||||
# Заголовочные файлы записываются в пакет clickhouse-common, в директорию /usr/share/clickhouse/headers.
|
||||
#
|
||||
# Если вы хотите установить их самостоятельно, без сборки пакета,
|
||||
# чтобы clickhouse-server видел их там, где ожидается, выполните:
|
||||
|
@ -131,7 +131,6 @@ target_link_libraries (clickhouse_common_io
|
||||
${LINK_LIBRARIES_ONLY_ON_X86_64}
|
||||
${LZ4_LIBRARY}
|
||||
${ZSTD_LIBRARY}
|
||||
${ZOOKEEPER_LIBRARY}
|
||||
${DOUBLE_CONVERSION_LIBRARIES}
|
||||
${Poco_Net_LIBRARY}
|
||||
${Poco_Data_LIBRARY}
|
||||
@ -158,10 +157,6 @@ if (NOT USE_INTERNAL_RE2_LIBRARY)
|
||||
target_include_directories (dbms BEFORE PRIVATE ${RE2_INCLUDE_DIR})
|
||||
endif ()
|
||||
|
||||
if (NOT USE_INTERNAL_ZOOKEEPER_LIBRARY)
|
||||
target_include_directories (clickhouse_common_io BEFORE PUBLIC ${ZOOKEEPER_INCLUDE_DIR})
|
||||
endif ()
|
||||
|
||||
if (NOT USE_INTERNAL_BOOST_LIBRARY)
|
||||
target_include_directories (clickhouse_common_io BEFORE PUBLIC ${Boost_INCLUDE_DIRS})
|
||||
endif ()
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <DataStreams/NativeBlockInputStream.h>
|
||||
#include <DataStreams/NativeBlockOutputStream.h>
|
||||
#include <Client/Connection.h>
|
||||
#include <Client/TimeoutSetter.h>
|
||||
#include <Common/ClickHouseRevision.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/NetException.h>
|
||||
@ -50,14 +51,15 @@ void Connection::connect()
|
||||
if (connected)
|
||||
disconnect();
|
||||
|
||||
LOG_TRACE(log_wrapper.get(), "Connecting. Database: " << (default_database.empty() ? "(not specified)" : default_database) << ". User: " << user);
|
||||
LOG_TRACE(log_wrapper.get(), "Connecting. Database: " << (default_database.empty() ? "(not specified)" : default_database) << ". User: " << user
|
||||
<< (static_cast<bool>(secure) ? ". Secure" : "") << (static_cast<bool>(compression) ? "" : ". Uncompressed") );
|
||||
|
||||
if (static_cast<bool>(encryption))
|
||||
if (static_cast<bool>(secure))
|
||||
{
|
||||
#if Poco_NetSSL_FOUND
|
||||
socket = std::make_unique<Poco::Net::SecureStreamSocket>();
|
||||
#else
|
||||
throw Exception{"tcp_ssl protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
#endif
|
||||
}
|
||||
else
|
||||
@ -148,6 +150,10 @@ void Connection::receiveHello()
|
||||
{
|
||||
readStringBinary(server_timezone, *in);
|
||||
}
|
||||
if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME)
|
||||
{
|
||||
readStringBinary(server_display_name, *in);
|
||||
}
|
||||
}
|
||||
else if (packet_type == Protocol::Server::Exception)
|
||||
receiveException()->rethrow();
|
||||
@ -203,6 +209,14 @@ const String & Connection::getServerTimezone()
|
||||
return server_timezone;
|
||||
}
|
||||
|
||||
const String & Connection::getServerDisplayName()
|
||||
{
|
||||
if (!connected)
|
||||
connect();
|
||||
|
||||
return server_display_name;
|
||||
}
|
||||
|
||||
void Connection::forceConnected()
|
||||
{
|
||||
if (!connected)
|
||||
@ -216,32 +230,6 @@ void Connection::forceConnected()
|
||||
}
|
||||
}
|
||||
|
||||
struct TimeoutSetter
|
||||
{
|
||||
TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & timeout_)
|
||||
: socket(socket_), timeout(timeout_)
|
||||
{
|
||||
old_send_timeout = socket.getSendTimeout();
|
||||
old_receive_timeout = socket.getReceiveTimeout();
|
||||
|
||||
if (old_send_timeout > timeout)
|
||||
socket.setSendTimeout(timeout);
|
||||
if (old_receive_timeout > timeout)
|
||||
socket.setReceiveTimeout(timeout);
|
||||
}
|
||||
|
||||
~TimeoutSetter()
|
||||
{
|
||||
socket.setSendTimeout(old_send_timeout);
|
||||
socket.setReceiveTimeout(old_receive_timeout);
|
||||
}
|
||||
|
||||
Poco::Net::StreamSocket & socket;
|
||||
Poco::Timespan timeout;
|
||||
Poco::Timespan old_send_timeout;
|
||||
Poco::Timespan old_receive_timeout;
|
||||
};
|
||||
|
||||
bool Connection::ping()
|
||||
{
|
||||
// LOG_TRACE(log_wrapper.get(), "Ping");
|
||||
|
@ -58,14 +58,14 @@ public:
|
||||
const ConnectionTimeouts & timeouts_,
|
||||
const String & client_name_ = "client",
|
||||
Protocol::Compression compression_ = Protocol::Compression::Enable,
|
||||
Protocol::Encryption encryption_ = Protocol::Encryption::Disable,
|
||||
Protocol::Secure secure_ = Protocol::Secure::Disable,
|
||||
Poco::Timespan sync_request_timeout_ = Poco::Timespan(DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC, 0))
|
||||
:
|
||||
host(host_), port(port_), default_database(default_database_),
|
||||
user(user_), password(password_), resolved_address(host, port),
|
||||
client_name(client_name_),
|
||||
compression(compression_),
|
||||
encryption(encryption_),
|
||||
secure(secure_),
|
||||
timeouts(timeouts_),
|
||||
sync_request_timeout(sync_request_timeout_),
|
||||
log_wrapper(*this)
|
||||
@ -84,7 +84,7 @@ public:
|
||||
const ConnectionTimeouts & timeouts_,
|
||||
const String & client_name_ = "client",
|
||||
Protocol::Compression compression_ = Protocol::Compression::Enable,
|
||||
Protocol::Encryption encryption_ = Protocol::Encryption::Disable,
|
||||
Protocol::Secure secure_ = Protocol::Secure::Disable,
|
||||
Poco::Timespan sync_request_timeout_ = Poco::Timespan(DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC, 0))
|
||||
:
|
||||
host(host_), port(port_),
|
||||
@ -93,7 +93,7 @@ public:
|
||||
resolved_address(resolved_address_),
|
||||
client_name(client_name_),
|
||||
compression(compression_),
|
||||
encryption(encryption_),
|
||||
secure(secure_),
|
||||
timeouts(timeouts_),
|
||||
sync_request_timeout(sync_request_timeout_),
|
||||
log_wrapper(*this)
|
||||
@ -134,6 +134,7 @@ public:
|
||||
void getServerVersion(String & name, UInt64 & version_major, UInt64 & version_minor, UInt64 & revision);
|
||||
|
||||
const String & getServerTimezone();
|
||||
const String & getServerDisplayName();
|
||||
|
||||
/// For log and exception messages.
|
||||
const String & getDescription() const;
|
||||
@ -213,6 +214,7 @@ private:
|
||||
UInt64 server_version_minor = 0;
|
||||
UInt64 server_revision = 0;
|
||||
String server_timezone;
|
||||
String server_display_name;
|
||||
|
||||
std::unique_ptr<Poco::Net::StreamSocket> socket;
|
||||
std::shared_ptr<ReadBuffer> in;
|
||||
@ -220,7 +222,7 @@ private:
|
||||
|
||||
String query_id;
|
||||
Protocol::Compression compression; /// Enable data compression for communication.
|
||||
Protocol::Encryption encryption; /// Enable data encryption for communication.
|
||||
Protocol::Secure secure; /// Enable data encryption for communication.
|
||||
|
||||
/// What compression settings to use while sending data for INSERT queries and external tables.
|
||||
CompressionSettings compression_settings;
|
||||
|
@ -51,11 +51,12 @@ public:
|
||||
const ConnectionTimeouts & timeouts,
|
||||
const String & client_name_ = "client",
|
||||
Protocol::Compression compression_ = Protocol::Compression::Enable,
|
||||
Protocol::Encryption encryption_ = Protocol::Encryption::Disable)
|
||||
Protocol::Secure secure_ = Protocol::Secure::Disable)
|
||||
: Base(max_connections_, &Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),
|
||||
host(host_), port(port_), default_database(default_database_),
|
||||
user(user_), password(password_), resolved_address(host_, port_),
|
||||
client_name(client_name_), compression(compression_), encryption(encryption_),
|
||||
client_name(client_name_), compression(compression_),
|
||||
secure{secure_},
|
||||
timeouts(timeouts)
|
||||
{
|
||||
}
|
||||
@ -67,11 +68,12 @@ public:
|
||||
const ConnectionTimeouts & timeouts,
|
||||
const String & client_name_ = "client",
|
||||
Protocol::Compression compression_ = Protocol::Compression::Enable,
|
||||
Protocol::Encryption encryption_ = Protocol::Encryption::Disable)
|
||||
Protocol::Secure secure_ = Protocol::Secure::Disable)
|
||||
: Base(max_connections_, &Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),
|
||||
host(host_), port(port_), default_database(default_database_),
|
||||
user(user_), password(password_), resolved_address(resolved_address_),
|
||||
client_name(client_name_), compression(compression_), encryption(encryption_),
|
||||
client_name(client_name_), compression(compression_),
|
||||
secure{secure_},
|
||||
timeouts(timeouts)
|
||||
{
|
||||
}
|
||||
@ -102,7 +104,7 @@ protected:
|
||||
return std::make_shared<Connection>(
|
||||
host, port, resolved_address,
|
||||
default_database, user, password, timeouts,
|
||||
client_name, compression, encryption);
|
||||
client_name, compression, secure);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -119,7 +121,7 @@ private:
|
||||
|
||||
String client_name;
|
||||
Protocol::Compression compression; /// Whether to compress data when interacting with the server.
|
||||
Protocol::Encryption encryption; /// Whether to encrypt data when interacting with the server.
|
||||
Protocol::Secure secure; /// Whether to encrypt data when interacting with the server.
|
||||
|
||||
ConnectionTimeouts timeouts;
|
||||
};
|
||||
|
46
dbms/src/Client/TimeoutSetter.h
Normal file
46
dbms/src/Client/TimeoutSetter.h
Normal file
@ -0,0 +1,46 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Timespan.h>
|
||||
#include <Poco/Net/StreamSocket.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Temporarily overrides socket send/recieve timeouts and reset them back into destructor
|
||||
/// Timeouts could be only decreased
|
||||
struct TimeoutSetter
|
||||
{
|
||||
TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & send_timeout_, const Poco::Timespan & recieve_timeout_)
|
||||
: socket(socket_), send_timeout(send_timeout_), recieve_timeout(recieve_timeout_)
|
||||
{
|
||||
old_send_timeout = socket.getSendTimeout();
|
||||
old_receive_timeout = socket.getReceiveTimeout();
|
||||
|
||||
if (old_send_timeout > send_timeout)
|
||||
socket.setSendTimeout(send_timeout);
|
||||
|
||||
if (old_receive_timeout > recieve_timeout)
|
||||
socket.setReceiveTimeout(recieve_timeout);
|
||||
}
|
||||
|
||||
TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & timeout_)
|
||||
: TimeoutSetter(socket_, timeout_, timeout_) {}
|
||||
|
||||
~TimeoutSetter()
|
||||
{
|
||||
socket.setSendTimeout(old_send_timeout);
|
||||
socket.setReceiveTimeout(old_receive_timeout);
|
||||
}
|
||||
|
||||
Poco::Net::StreamSocket & socket;
|
||||
|
||||
Poco::Timespan send_timeout;
|
||||
Poco::Timespan recieve_timeout;
|
||||
|
||||
Poco::Timespan old_send_timeout;
|
||||
Poco::Timespan old_receive_timeout;
|
||||
};
|
||||
|
||||
|
||||
}
|
@ -113,7 +113,7 @@ void ColumnAggregateFunction::insertRangeFrom(const IColumn & from, size_t start
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnAggregateFunction::filter(const Filter & filter, ssize_t result_size_hint) const
|
||||
ColumnPtr ColumnAggregateFunction::filter(const Filter & filter, ssize_t result_size_hint) const
|
||||
{
|
||||
size_t size = getData().size();
|
||||
if (size != filter.size())
|
||||
@ -140,7 +140,7 @@ MutableColumnPtr ColumnAggregateFunction::filter(const Filter & filter, ssize_t
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limit) const
|
||||
ColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limit) const
|
||||
{
|
||||
size_t size = getData().size();
|
||||
|
||||
@ -325,7 +325,7 @@ void ColumnAggregateFunction::popBack(size_t n)
|
||||
data.resize_assume_reserved(new_size);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnAggregateFunction::replicate(const IColumn::Offsets & offsets) const
|
||||
ColumnPtr ColumnAggregateFunction::replicate(const IColumn::Offsets & offsets) const
|
||||
{
|
||||
size_t size = data.size();
|
||||
if (size != offsets.size())
|
||||
|
@ -152,11 +152,11 @@ public:
|
||||
|
||||
void popBack(size_t n) override;
|
||||
|
||||
MutableColumnPtr filter(const Filter & filter, ssize_t result_size_hint) const override;
|
||||
ColumnPtr filter(const Filter & filter, ssize_t result_size_hint) const override;
|
||||
|
||||
MutableColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
|
||||
MutableColumnPtr replicate(const Offsets & offsets) const override;
|
||||
ColumnPtr replicate(const Offsets & offsets) const override;
|
||||
|
||||
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override;
|
||||
|
||||
|
@ -31,10 +31,10 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
ColumnArray::ColumnArray(const ColumnPtr & nested_column, const ColumnPtr & offsets_column)
|
||||
: data(nested_column), offsets(offsets_column)
|
||||
ColumnArray::ColumnArray(MutableColumnPtr && nested_column, MutableColumnPtr && offsets_column)
|
||||
: data(std::move(nested_column)), offsets(std::move(offsets_column))
|
||||
{
|
||||
if (!typeid_cast<const ColumnOffsets *>(offsets_column.get()))
|
||||
if (!typeid_cast<const ColumnOffsets *>(offsets.get()))
|
||||
throw Exception("offsets_column must be a ColumnUInt64", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
/** NOTE
|
||||
@ -43,8 +43,8 @@ ColumnArray::ColumnArray(const ColumnPtr & nested_column, const ColumnPtr & offs
|
||||
*/
|
||||
}
|
||||
|
||||
ColumnArray::ColumnArray(const ColumnPtr & nested_column)
|
||||
: data(nested_column)
|
||||
ColumnArray::ColumnArray(MutableColumnPtr && nested_column)
|
||||
: data(std::move(nested_column))
|
||||
{
|
||||
if (!data->empty())
|
||||
throw Exception("Not empty data passed to ColumnArray, but no offsets passed", ErrorCodes::ILLEGAL_COLUMN);
|
||||
@ -317,7 +317,7 @@ bool ColumnArray::hasEqualOffsets(const ColumnArray & other) const
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnArray::convertToFullColumnIfConst() const
|
||||
ColumnPtr ColumnArray::convertToFullColumnIfConst() const
|
||||
{
|
||||
ColumnPtr new_data;
|
||||
|
||||
@ -391,7 +391,7 @@ void ColumnArray::insertRangeFrom(const IColumn & src, size_t start, size_t leng
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnArray::filter(const Filter & filt, ssize_t result_size_hint) const
|
||||
ColumnPtr ColumnArray::filter(const Filter & filt, ssize_t result_size_hint) const
|
||||
{
|
||||
if (typeid_cast<const ColumnUInt8 *>(data.get())) return filterNumber<UInt8>(filt, result_size_hint);
|
||||
if (typeid_cast<const ColumnUInt16 *>(data.get())) return filterNumber<UInt16>(filt, result_size_hint);
|
||||
@ -410,7 +410,7 @@ MutableColumnPtr ColumnArray::filter(const Filter & filt, ssize_t result_size_hi
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
MutableColumnPtr ColumnArray::filterNumber(const Filter & filt, ssize_t result_size_hint) const
|
||||
ColumnPtr ColumnArray::filterNumber(const Filter & filt, ssize_t result_size_hint) const
|
||||
{
|
||||
if (getOffsets().size() == 0)
|
||||
return ColumnArray::create(data);
|
||||
@ -424,7 +424,7 @@ MutableColumnPtr ColumnArray::filterNumber(const Filter & filt, ssize_t result_s
|
||||
return std::move(res);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnArray::filterString(const Filter & filt, ssize_t result_size_hint) const
|
||||
ColumnPtr ColumnArray::filterString(const Filter & filt, ssize_t result_size_hint) const
|
||||
{
|
||||
size_t col_size = getOffsets().size();
|
||||
if (col_size != filt.size())
|
||||
@ -492,7 +492,7 @@ MutableColumnPtr ColumnArray::filterString(const Filter & filt, ssize_t result_s
|
||||
return std::move(res);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnArray::filterGeneric(const Filter & filt, ssize_t result_size_hint) const
|
||||
ColumnPtr ColumnArray::filterGeneric(const Filter & filt, ssize_t result_size_hint) const
|
||||
{
|
||||
size_t size = getOffsets().size();
|
||||
if (size != filt.size())
|
||||
@ -537,7 +537,7 @@ MutableColumnPtr ColumnArray::filterGeneric(const Filter & filt, ssize_t result_
|
||||
return std::move(res);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnArray::filterNullable(const Filter & filt, ssize_t result_size_hint) const
|
||||
ColumnPtr ColumnArray::filterNullable(const Filter & filt, ssize_t result_size_hint) const
|
||||
{
|
||||
if (getOffsets().size() == 0)
|
||||
return ColumnArray::create(data);
|
||||
@ -560,7 +560,7 @@ MutableColumnPtr ColumnArray::filterNullable(const Filter & filt, ssize_t result
|
||||
filtered_offsets);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnArray::filterTuple(const Filter & filt, ssize_t result_size_hint) const
|
||||
ColumnPtr ColumnArray::filterTuple(const Filter & filt, ssize_t result_size_hint) const
|
||||
{
|
||||
if (getOffsets().size() == 0)
|
||||
return ColumnArray::create(data);
|
||||
@ -576,7 +576,8 @@ MutableColumnPtr ColumnArray::filterTuple(const Filter & filt, ssize_t result_si
|
||||
|
||||
Columns temporary_arrays(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
temporary_arrays[i] = ColumnArray(tuple.getColumns()[i], getOffsetsPtr()).filter(filt, result_size_hint);
|
||||
temporary_arrays[i] = ColumnArray(tuple.getColumns()[i]->assumeMutable(), getOffsetsPtr()->assumeMutable())
|
||||
.filter(filt, result_size_hint);
|
||||
|
||||
Columns tuple_columns(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
@ -588,7 +589,7 @@ MutableColumnPtr ColumnArray::filterTuple(const Filter & filt, ssize_t result_si
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnArray::permute(const Permutation & perm, size_t limit) const
|
||||
ColumnPtr ColumnArray::permute(const Permutation & perm, size_t limit) const
|
||||
{
|
||||
size_t size = getOffsets().size();
|
||||
|
||||
@ -652,7 +653,7 @@ void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_h
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnArray::replicate(const Offsets & replicate_offsets) const
|
||||
ColumnPtr ColumnArray::replicate(const Offsets & replicate_offsets) const
|
||||
{
|
||||
if (typeid_cast<const ColumnUInt8 *>(data.get())) return replicateNumber<UInt8>(replicate_offsets);
|
||||
if (typeid_cast<const ColumnUInt16 *>(data.get())) return replicateNumber<UInt16>(replicate_offsets);
|
||||
@ -673,7 +674,7 @@ MutableColumnPtr ColumnArray::replicate(const Offsets & replicate_offsets) const
|
||||
|
||||
|
||||
template <typename T>
|
||||
MutableColumnPtr ColumnArray::replicateNumber(const Offsets & replicate_offsets) const
|
||||
ColumnPtr ColumnArray::replicateNumber(const Offsets & replicate_offsets) const
|
||||
{
|
||||
size_t col_size = size();
|
||||
if (col_size != replicate_offsets.size())
|
||||
@ -721,7 +722,7 @@ MutableColumnPtr ColumnArray::replicateNumber(const Offsets & replicate_offsets)
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnArray::replicateString(const Offsets & replicate_offsets) const
|
||||
ColumnPtr ColumnArray::replicateString(const Offsets & replicate_offsets) const
|
||||
{
|
||||
size_t col_size = size();
|
||||
if (col_size != replicate_offsets.size())
|
||||
@ -796,7 +797,7 @@ MutableColumnPtr ColumnArray::replicateString(const Offsets & replicate_offsets)
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnArray::replicateConst(const Offsets & replicate_offsets) const
|
||||
ColumnPtr ColumnArray::replicateConst(const Offsets & replicate_offsets) const
|
||||
{
|
||||
size_t col_size = size();
|
||||
if (col_size != replicate_offsets.size())
|
||||
@ -834,7 +835,7 @@ MutableColumnPtr ColumnArray::replicateConst(const Offsets & replicate_offsets)
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnArray::replicateGeneric(const Offsets & replicate_offsets) const
|
||||
ColumnPtr ColumnArray::replicateGeneric(const Offsets & replicate_offsets) const
|
||||
{
|
||||
size_t col_size = size();
|
||||
if (col_size != replicate_offsets.size())
|
||||
@ -860,25 +861,27 @@ MutableColumnPtr ColumnArray::replicateGeneric(const Offsets & replicate_offsets
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnArray::replicateNullable(const Offsets & replicate_offsets) const
|
||||
ColumnPtr ColumnArray::replicateNullable(const Offsets & replicate_offsets) const
|
||||
{
|
||||
const ColumnNullable & nullable = static_cast<const ColumnNullable &>(*data);
|
||||
|
||||
/// Make temporary arrays for each components of Nullable. Then replicate them independently and collect back to result.
|
||||
/// NOTE Offsets are calculated twice and it is redundant.
|
||||
|
||||
auto array_of_nested = ColumnArray(nullable.getNestedColumnPtr(), getOffsetsPtr()).replicate(replicate_offsets);
|
||||
auto array_of_null_map = ColumnArray(nullable.getNullMapColumnPtr(), getOffsetsPtr()).replicate(replicate_offsets);
|
||||
auto array_of_nested = ColumnArray(nullable.getNestedColumnPtr()->assumeMutable(), getOffsetsPtr()->assumeMutable())
|
||||
.replicate(replicate_offsets);
|
||||
auto array_of_null_map = ColumnArray(nullable.getNullMapColumnPtr()->assumeMutable(), getOffsetsPtr()->assumeMutable())
|
||||
.replicate(replicate_offsets);
|
||||
|
||||
return ColumnArray::create(
|
||||
ColumnNullable::create(
|
||||
static_cast<ColumnArray &>(*array_of_nested).getDataPtr(),
|
||||
static_cast<ColumnArray &>(*array_of_null_map).getDataPtr()),
|
||||
static_cast<ColumnArray &>(*array_of_nested).getOffsetsPtr());
|
||||
static_cast<const ColumnArray &>(*array_of_nested).getDataPtr(),
|
||||
static_cast<const ColumnArray &>(*array_of_null_map).getDataPtr()),
|
||||
static_cast<const ColumnArray &>(*array_of_nested).getOffsetsPtr());
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnArray::replicateTuple(const Offsets & replicate_offsets) const
|
||||
ColumnPtr ColumnArray::replicateTuple(const Offsets & replicate_offsets) const
|
||||
{
|
||||
const ColumnTuple & tuple = static_cast<const ColumnTuple &>(*data);
|
||||
|
||||
@ -891,7 +894,8 @@ MutableColumnPtr ColumnArray::replicateTuple(const Offsets & replicate_offsets)
|
||||
|
||||
Columns temporary_arrays(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
temporary_arrays[i] = ColumnArray(tuple.getColumns()[i], getOffsetsPtr()).replicate(replicate_offsets);
|
||||
temporary_arrays[i] = ColumnArray(tuple.getColumns()[i]->assumeMutable(), getOffsetsPtr()->assumeMutable())
|
||||
.replicate(replicate_offsets);
|
||||
|
||||
Columns tuple_columns(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
|
@ -24,14 +24,32 @@ private:
|
||||
friend class COWPtrHelper<IColumn, ColumnArray>;
|
||||
|
||||
/** Create an array column with specified values and offsets. */
|
||||
ColumnArray(const ColumnPtr & nested_column, const ColumnPtr & offsets_column);
|
||||
ColumnArray(MutableColumnPtr && nested_column, MutableColumnPtr && offsets_column);
|
||||
|
||||
/** Create an empty column of arrays with the type of values as in the column `nested_column` */
|
||||
ColumnArray(const ColumnPtr & nested_column);
|
||||
explicit ColumnArray(MutableColumnPtr && nested_column);
|
||||
|
||||
ColumnArray(const ColumnArray &) = default;
|
||||
|
||||
public:
|
||||
/** Create immutable column using immutable arguments. This arguments may be shared with other columns.
|
||||
* Use IColumn::mutate in order to make mutable column and mutate shared nested columns.
|
||||
*/
|
||||
using Base = COWPtrHelper<IColumn, ColumnArray>;
|
||||
|
||||
static Ptr create(const ColumnPtr & nested_column, const ColumnPtr & offsets_column)
|
||||
{
|
||||
return ColumnArray::create(nested_column->assumeMutable(), offsets_column->assumeMutable());
|
||||
}
|
||||
|
||||
static Ptr create(const ColumnPtr & nested_column)
|
||||
{
|
||||
return ColumnArray::create(nested_column->assumeMutable());
|
||||
}
|
||||
|
||||
template <typename ... Args, typename = typename std::enable_if<IsMutableColumns<Args ...>::value>::type>
|
||||
static MutablePtr create(Args &&... args) { return Base::create(std::forward<Args>(args)...); }
|
||||
|
||||
/** On the index i there is an offset to the beginning of the i + 1 -th element. */
|
||||
using ColumnOffsets = ColumnVector<Offset>;
|
||||
|
||||
@ -51,15 +69,15 @@ public:
|
||||
void insertFrom(const IColumn & src_, size_t n) override;
|
||||
void insertDefault() override;
|
||||
void popBack(size_t n) override;
|
||||
MutableColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||
MutableColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override;
|
||||
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override;
|
||||
void reserve(size_t n) override;
|
||||
size_t byteSize() const override;
|
||||
size_t allocatedBytes() const override;
|
||||
MutableColumnPtr replicate(const Offsets & replicate_offsets) const override;
|
||||
MutableColumnPtr convertToFullColumnIfConst() const override;
|
||||
ColumnPtr replicate(const Offsets & replicate_offsets) const override;
|
||||
ColumnPtr convertToFullColumnIfConst() const override;
|
||||
void getExtremes(Field & min, Field & max) const override;
|
||||
|
||||
bool hasEqualOffsets(const ColumnArray & other) const;
|
||||
@ -110,33 +128,33 @@ private:
|
||||
|
||||
/// Multiply values if the nested column is ColumnVector<T>.
|
||||
template <typename T>
|
||||
MutableColumnPtr replicateNumber(const Offsets & replicate_offsets) const;
|
||||
ColumnPtr replicateNumber(const Offsets & replicate_offsets) const;
|
||||
|
||||
/// Multiply the values if the nested column is ColumnString. The code is too complicated.
|
||||
MutableColumnPtr replicateString(const Offsets & replicate_offsets) const;
|
||||
ColumnPtr replicateString(const Offsets & replicate_offsets) const;
|
||||
|
||||
/** Non-constant arrays of constant values are quite rare.
|
||||
* Most functions can not work with them, and does not create such columns as a result.
|
||||
* An exception is the function `replicate`(see FunctionsMiscellaneous.h), which has service meaning for the implementation of lambda functions.
|
||||
* Only for its sake is the implementation of the `replicate` method for ColumnArray(ColumnConst).
|
||||
*/
|
||||
MutableColumnPtr replicateConst(const Offsets & replicate_offsets) const;
|
||||
ColumnPtr replicateConst(const Offsets & replicate_offsets) const;
|
||||
|
||||
/** The following is done by simply replicating of nested columns.
|
||||
*/
|
||||
MutableColumnPtr replicateTuple(const Offsets & replicate_offsets) const;
|
||||
MutableColumnPtr replicateNullable(const Offsets & replicate_offsets) const;
|
||||
MutableColumnPtr replicateGeneric(const Offsets & replicate_offsets) const;
|
||||
ColumnPtr replicateTuple(const Offsets & replicate_offsets) const;
|
||||
ColumnPtr replicateNullable(const Offsets & replicate_offsets) const;
|
||||
ColumnPtr replicateGeneric(const Offsets & replicate_offsets) const;
|
||||
|
||||
|
||||
/// Specializations for the filter function.
|
||||
template <typename T>
|
||||
MutableColumnPtr filterNumber(const Filter & filt, ssize_t result_size_hint) const;
|
||||
ColumnPtr filterNumber(const Filter & filt, ssize_t result_size_hint) const;
|
||||
|
||||
MutableColumnPtr filterString(const Filter & filt, ssize_t result_size_hint) const;
|
||||
MutableColumnPtr filterTuple(const Filter & filt, ssize_t result_size_hint) const;
|
||||
MutableColumnPtr filterNullable(const Filter & filt, ssize_t result_size_hint) const;
|
||||
MutableColumnPtr filterGeneric(const Filter & filt, ssize_t result_size_hint) const;
|
||||
ColumnPtr filterString(const Filter & filt, ssize_t result_size_hint) const;
|
||||
ColumnPtr filterTuple(const Filter & filt, ssize_t result_size_hint) const;
|
||||
ColumnPtr filterNullable(const Filter & filt, ssize_t result_size_hint) const;
|
||||
ColumnPtr filterGeneric(const Filter & filt, ssize_t result_size_hint) const;
|
||||
};
|
||||
|
||||
|
||||
|
@ -25,12 +25,12 @@ ColumnConst::ColumnConst(const ColumnPtr & data_, size_t s)
|
||||
ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnConst::convertToFullColumn() const
|
||||
ColumnPtr ColumnConst::convertToFullColumn() const
|
||||
{
|
||||
return data->replicate(Offsets(1, s));
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnConst::filter(const Filter & filt, ssize_t /*result_size_hint*/) const
|
||||
ColumnPtr ColumnConst::filter(const Filter & filt, ssize_t /*result_size_hint*/) const
|
||||
{
|
||||
if (s != filt.size())
|
||||
throw Exception("Size of filter (" + toString(filt.size()) + ") doesn't match size of column (" + toString(s) + ")",
|
||||
@ -39,7 +39,7 @@ MutableColumnPtr ColumnConst::filter(const Filter & filt, ssize_t /*result_size_
|
||||
return ColumnConst::create(data, countBytesInFilter(filt));
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnConst::replicate(const Offsets & offsets) const
|
||||
ColumnPtr ColumnConst::replicate(const Offsets & offsets) const
|
||||
{
|
||||
if (s != offsets.size())
|
||||
throw Exception("Size of offsets (" + toString(offsets.size()) + ") doesn't match size of column (" + toString(s) + ")",
|
||||
@ -49,7 +49,7 @@ MutableColumnPtr ColumnConst::replicate(const Offsets & offsets) const
|
||||
return ColumnConst::create(data, replicated_size);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnConst::permute(const Permutation & perm, size_t limit) const
|
||||
ColumnPtr ColumnConst::permute(const Permutation & perm, size_t limit) const
|
||||
{
|
||||
if (limit == 0)
|
||||
limit = s;
|
||||
|
@ -29,9 +29,9 @@ private:
|
||||
ColumnConst(const ColumnConst & src) = default;
|
||||
|
||||
public:
|
||||
MutableColumnPtr convertToFullColumn() const;
|
||||
ColumnPtr convertToFullColumn() const;
|
||||
|
||||
MutableColumnPtr convertToFullColumnIfConst() const override
|
||||
ColumnPtr convertToFullColumnIfConst() const override
|
||||
{
|
||||
return convertToFullColumn();
|
||||
}
|
||||
@ -145,9 +145,9 @@ public:
|
||||
data->updateHashWithValue(0, hash);
|
||||
}
|
||||
|
||||
MutableColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||
MutableColumnPtr replicate(const Offsets & offsets) const override;
|
||||
MutableColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||
ColumnPtr replicate(const Offsets & offsets) const override;
|
||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override;
|
||||
|
||||
size_t byteSize() const override
|
||||
|
@ -153,7 +153,7 @@ void ColumnFixedString::insertRangeFrom(const IColumn & src, size_t start, size_
|
||||
memcpy(&chars[old_size], &src_concrete.chars[start * n], length * n);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result_size_hint) const
|
||||
ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result_size_hint) const
|
||||
{
|
||||
size_t col_size = size();
|
||||
if (col_size != filt.size())
|
||||
@ -230,7 +230,7 @@ MutableColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t
|
||||
return std::move(res);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnFixedString::permute(const Permutation & perm, size_t limit) const
|
||||
ColumnPtr ColumnFixedString::permute(const Permutation & perm, size_t limit) const
|
||||
{
|
||||
size_t col_size = size();
|
||||
|
||||
@ -258,7 +258,7 @@ MutableColumnPtr ColumnFixedString::permute(const Permutation & perm, size_t lim
|
||||
return std::move(res);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnFixedString::replicate(const Offsets & offsets) const
|
||||
ColumnPtr ColumnFixedString::replicate(const Offsets & offsets) const
|
||||
{
|
||||
size_t col_size = size();
|
||||
if (col_size != offsets.size())
|
||||
|
@ -104,11 +104,11 @@ public:
|
||||
|
||||
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
|
||||
|
||||
MutableColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override;
|
||||
ColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override;
|
||||
|
||||
MutableColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
|
||||
MutableColumnPtr replicate(const Offsets & offsets) const override;
|
||||
ColumnPtr replicate(const Offsets & offsets) const override;
|
||||
|
||||
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override
|
||||
{
|
||||
|
@ -26,7 +26,7 @@ MutableColumnPtr ColumnFunction::cloneResized(size_t size) const
|
||||
return ColumnFunction::create(size, function, capture);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnFunction::replicate(const Offsets & offsets) const
|
||||
ColumnPtr ColumnFunction::replicate(const Offsets & offsets) const
|
||||
{
|
||||
if (size_ != offsets.size())
|
||||
throw Exception("Size of offsets (" + toString(offsets.size()) + ") doesn't match size of column ("
|
||||
@ -40,7 +40,7 @@ MutableColumnPtr ColumnFunction::replicate(const Offsets & offsets) const
|
||||
return ColumnFunction::create(replicated_size, function, capture);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnFunction::cut(size_t start, size_t length) const
|
||||
ColumnPtr ColumnFunction::cut(size_t start, size_t length) const
|
||||
{
|
||||
ColumnsWithTypeAndName capture = captured_columns;
|
||||
for (auto & column : capture)
|
||||
@ -49,7 +49,7 @@ MutableColumnPtr ColumnFunction::cut(size_t start, size_t length) const
|
||||
return ColumnFunction::create(length, function, capture);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnFunction::filter(const Filter & filt, ssize_t result_size_hint) const
|
||||
ColumnPtr ColumnFunction::filter(const Filter & filt, ssize_t result_size_hint) const
|
||||
{
|
||||
if (size_ != filt.size())
|
||||
throw Exception("Size of filter (" + toString(filt.size()) + ") doesn't match size of column ("
|
||||
@ -68,7 +68,7 @@ MutableColumnPtr ColumnFunction::filter(const Filter & filt, ssize_t result_size
|
||||
return ColumnFunction::create(filtered_size, function, capture);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnFunction::permute(const Permutation & perm, size_t limit) const
|
||||
ColumnPtr ColumnFunction::permute(const Permutation & perm, size_t limit) const
|
||||
{
|
||||
if (limit == 0)
|
||||
limit = size_;
|
||||
|
@ -29,10 +29,10 @@ public:
|
||||
|
||||
size_t size() const override { return size_; }
|
||||
|
||||
MutableColumnPtr cut(size_t start, size_t length) const override;
|
||||
MutableColumnPtr replicate(const Offsets & offsets) const override;
|
||||
MutableColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||
MutableColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
ColumnPtr cut(size_t start, size_t length) const override;
|
||||
ColumnPtr replicate(const Offsets & offsets) const override;
|
||||
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
void insertDefault() override;
|
||||
void popBack(size_t n) override;
|
||||
std::vector<MutableColumnPtr> scatter(IColumn::ColumnIndex num_columns,
|
||||
|
@ -18,8 +18,8 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
ColumnNullable::ColumnNullable(const ColumnPtr & nested_column_, const ColumnPtr & null_map_)
|
||||
: nested_column{nested_column_}, null_map{null_map_}
|
||||
ColumnNullable::ColumnNullable(MutableColumnPtr && nested_column_, MutableColumnPtr && null_map_)
|
||||
: nested_column(std::move(nested_column_)), null_map(std::move(null_map_))
|
||||
{
|
||||
/// ColumnNullable cannot have constant nested column. But constant argument could be passed. Materialize it.
|
||||
if (ColumnPtr nested_column_materialized = getNestedColumn().convertToFullColumnIfConst())
|
||||
@ -44,7 +44,7 @@ void ColumnNullable::updateHashWithValue(size_t n, SipHash & hash) const
|
||||
|
||||
MutableColumnPtr ColumnNullable::cloneResized(size_t new_size) const
|
||||
{
|
||||
ColumnPtr new_nested_col = getNestedColumn().cloneResized(new_size);
|
||||
MutableColumnPtr new_nested_col = getNestedColumn().cloneResized(new_size);
|
||||
auto new_null_map = ColumnUInt8::create();
|
||||
|
||||
if (new_size > 0)
|
||||
@ -59,7 +59,7 @@ MutableColumnPtr ColumnNullable::cloneResized(size_t new_size) const
|
||||
memset(&new_null_map->getData()[count], 1, new_size - count);
|
||||
}
|
||||
|
||||
return ColumnNullable::create(new_nested_col, std::move(new_null_map));
|
||||
return ColumnNullable::create(std::move(new_nested_col), std::move(new_null_map));
|
||||
}
|
||||
|
||||
|
||||
@ -152,14 +152,14 @@ void ColumnNullable::popBack(size_t n)
|
||||
getNullMapColumn().popBack(n);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnNullable::filter(const Filter & filt, ssize_t result_size_hint) const
|
||||
ColumnPtr ColumnNullable::filter(const Filter & filt, ssize_t result_size_hint) const
|
||||
{
|
||||
ColumnPtr filtered_data = getNestedColumn().filter(filt, result_size_hint);
|
||||
ColumnPtr filtered_null_map = getNullMapColumn().filter(filt, result_size_hint);
|
||||
return ColumnNullable::create(filtered_data, filtered_null_map);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnNullable::permute(const Permutation & perm, size_t limit) const
|
||||
ColumnPtr ColumnNullable::permute(const Permutation & perm, size_t limit) const
|
||||
{
|
||||
ColumnPtr permuted_data = getNestedColumn().permute(perm, limit);
|
||||
ColumnPtr permuted_null_map = getNullMapColumn().permute(perm, limit);
|
||||
@ -384,7 +384,7 @@ void ColumnNullable::getExtremes(Field & min, Field & max) const
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnNullable::replicate(const Offsets & offsets) const
|
||||
ColumnPtr ColumnNullable::replicate(const Offsets & offsets) const
|
||||
{
|
||||
ColumnPtr replicated_data = getNestedColumn().replicate(offsets);
|
||||
ColumnPtr replicated_null_map = getNullMapColumn().replicate(offsets);
|
||||
|
@ -23,10 +23,22 @@ class ColumnNullable final : public COWPtrHelper<IColumn, ColumnNullable>
|
||||
private:
|
||||
friend class COWPtrHelper<IColumn, ColumnNullable>;
|
||||
|
||||
ColumnNullable(const ColumnPtr & nested_column_, const ColumnPtr & null_map_);
|
||||
ColumnNullable(MutableColumnPtr && nested_column_, MutableColumnPtr && null_map_);
|
||||
ColumnNullable(const ColumnNullable &) = default;
|
||||
|
||||
public:
|
||||
/** Create immutable column using immutable arguments. This arguments may be shared with other columns.
|
||||
* Use IColumn::mutate in order to make mutable column and mutate shared nested columns.
|
||||
*/
|
||||
using Base = COWPtrHelper<IColumn, ColumnNullable>;
|
||||
static Ptr create(const ColumnPtr & nested_column_, const ColumnPtr & null_map_)
|
||||
{
|
||||
return ColumnNullable::create(nested_column_->assumeMutable(), null_map_->assumeMutable());
|
||||
}
|
||||
|
||||
template <typename ... Args, typename = typename std::enable_if<IsMutableColumns<Args ...>::value>::type>
|
||||
static MutablePtr create(Args &&... args) { return Base::create(std::forward<Args>(args)...); }
|
||||
|
||||
const char * getFamilyName() const override { return "Nullable"; }
|
||||
std::string getName() const override { return "Nullable(" + nested_column->getName() + ")"; }
|
||||
MutableColumnPtr cloneResized(size_t size) const override;
|
||||
@ -50,14 +62,14 @@ public:
|
||||
}
|
||||
|
||||
void popBack(size_t n) override;
|
||||
MutableColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||
MutableColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
int compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override;
|
||||
void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override;
|
||||
void reserve(size_t n) override;
|
||||
size_t byteSize() const override;
|
||||
size_t allocatedBytes() const override;
|
||||
MutableColumnPtr replicate(const Offsets & replicate_offsets) const override;
|
||||
ColumnPtr replicate(const Offsets & replicate_offsets) const override;
|
||||
void updateHashWithValue(size_t n, SipHash & hash) const override;
|
||||
void getExtremes(Field & min, Field & max) const override;
|
||||
|
||||
|
@ -97,7 +97,7 @@ void ColumnString::insertRangeFrom(const IColumn & src, size_t start, size_t len
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnString::filter(const Filter & filt, ssize_t result_size_hint) const
|
||||
ColumnPtr ColumnString::filter(const Filter & filt, ssize_t result_size_hint) const
|
||||
{
|
||||
if (offsets.size() == 0)
|
||||
return ColumnString::create();
|
||||
@ -112,7 +112,7 @@ MutableColumnPtr ColumnString::filter(const Filter & filt, ssize_t result_size_h
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnString::permute(const Permutation & perm, size_t limit) const
|
||||
ColumnPtr ColumnString::permute(const Permutation & perm, size_t limit) const
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
|
||||
@ -208,7 +208,7 @@ void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_directio
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const
|
||||
ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const
|
||||
{
|
||||
size_t col_size = size();
|
||||
if (col_size != replicate_offsets.size())
|
||||
|
@ -89,6 +89,12 @@ public:
|
||||
return StringRef(&chars[offsetAt(n)], sizeAt(n));
|
||||
}
|
||||
|
||||
/// Suppress gcc 7.3.1 warning: '*((void*)&<anonymous> +8)' may be used uninitialized in this function
|
||||
#if !__clang__
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
|
||||
#endif
|
||||
|
||||
void insert(const Field & x) override
|
||||
{
|
||||
const String & s = DB::get<const String &>(x);
|
||||
@ -101,7 +107,11 @@ public:
|
||||
offsets.push_back(new_size);
|
||||
}
|
||||
|
||||
void insertFrom(const IColumn & src_, size_t n) override
|
||||
#if !__clang__
|
||||
#pragma GCC diagnostic pop
|
||||
#endif
|
||||
|
||||
void insertFrom(const IColumn & src_, size_t n) override
|
||||
{
|
||||
const ColumnString & src = static_cast<const ColumnString &>(src_);
|
||||
|
||||
@ -206,9 +216,9 @@ public:
|
||||
|
||||
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
|
||||
|
||||
MutableColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||
|
||||
MutableColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
|
||||
void insertDefault() override
|
||||
{
|
||||
@ -239,7 +249,7 @@ public:
|
||||
/// Sorting with respect of collation.
|
||||
void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const;
|
||||
|
||||
MutableColumnPtr replicate(const Offsets & replicate_offsets) const override;
|
||||
ColumnPtr replicate(const Offsets & replicate_offsets) const override;
|
||||
|
||||
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override
|
||||
{
|
||||
|
@ -31,21 +31,38 @@ std::string ColumnTuple::getName() const
|
||||
return res.str();
|
||||
}
|
||||
|
||||
ColumnTuple::ColumnTuple(const Columns & columns) : columns(columns)
|
||||
ColumnTuple::ColumnTuple(MutableColumns && mutable_columns)
|
||||
{
|
||||
columns.reserve(mutable_columns.size());
|
||||
for (auto & column : mutable_columns)
|
||||
{
|
||||
if (column->isColumnConst())
|
||||
throw Exception{"ColumnTuple cannot have ColumnConst as its element", ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
columns.push_back(std::move(column));
|
||||
}
|
||||
}
|
||||
|
||||
ColumnTuple::Ptr ColumnTuple::create(const Columns & columns)
|
||||
{
|
||||
for (const auto & column : columns)
|
||||
if (column->isColumnConst())
|
||||
throw Exception{"ColumnTuple cannot have ColumnConst as its element", ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
auto column_tuple = ColumnTuple::create(MutableColumns());
|
||||
column_tuple->columns = columns;
|
||||
|
||||
return std::move(column_tuple);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnTuple::cloneEmpty() const
|
||||
{
|
||||
const size_t tuple_size = columns.size();
|
||||
Columns new_columns(tuple_size);
|
||||
MutableColumns new_columns(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
new_columns[i] = columns[i]->cloneEmpty();
|
||||
|
||||
return ColumnTuple::create(new_columns);
|
||||
return ColumnTuple::create(std::move(new_columns));
|
||||
}
|
||||
|
||||
Field ColumnTuple::operator[](size_t n) const
|
||||
@ -140,7 +157,7 @@ void ColumnTuple::insertRangeFrom(const IColumn & src, size_t start, size_t leng
|
||||
start, length);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnTuple::filter(const Filter & filt, ssize_t result_size_hint) const
|
||||
ColumnPtr ColumnTuple::filter(const Filter & filt, ssize_t result_size_hint) const
|
||||
{
|
||||
const size_t tuple_size = columns.size();
|
||||
Columns new_columns(tuple_size);
|
||||
@ -151,7 +168,7 @@ MutableColumnPtr ColumnTuple::filter(const Filter & filt, ssize_t result_size_hi
|
||||
return ColumnTuple::create(new_columns);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnTuple::permute(const Permutation & perm, size_t limit) const
|
||||
ColumnPtr ColumnTuple::permute(const Permutation & perm, size_t limit) const
|
||||
{
|
||||
const size_t tuple_size = columns.size();
|
||||
Columns new_columns(tuple_size);
|
||||
@ -162,7 +179,7 @@ MutableColumnPtr ColumnTuple::permute(const Permutation & perm, size_t limit) co
|
||||
return ColumnTuple::create(new_columns);
|
||||
}
|
||||
|
||||
MutableColumnPtr ColumnTuple::replicate(const Offsets & offsets) const
|
||||
ColumnPtr ColumnTuple::replicate(const Offsets & offsets) const
|
||||
{
|
||||
const size_t tuple_size = columns.size();
|
||||
Columns new_columns(tuple_size);
|
||||
@ -185,10 +202,10 @@ MutableColumns ColumnTuple::scatter(ColumnIndex num_columns, const Selector & se
|
||||
|
||||
for (size_t scattered_idx = 0; scattered_idx < num_columns; ++scattered_idx)
|
||||
{
|
||||
Columns new_columns(tuple_size);
|
||||
MutableColumns new_columns(tuple_size);
|
||||
for (size_t tuple_element_idx = 0; tuple_element_idx < tuple_size; ++tuple_element_idx)
|
||||
new_columns[tuple_element_idx] = std::move(scattered_tuple_elements[tuple_element_idx][scattered_idx]);
|
||||
res[scattered_idx] = ColumnTuple::create(new_columns);
|
||||
res[scattered_idx] = ColumnTuple::create(std::move(new_columns));
|
||||
}
|
||||
|
||||
return res;
|
||||
|
@ -22,10 +22,19 @@ private:
|
||||
template <bool positive>
|
||||
struct Less;
|
||||
|
||||
ColumnTuple(const Columns & columns);
|
||||
explicit ColumnTuple(MutableColumns && columns);
|
||||
ColumnTuple(const ColumnTuple &) = default;
|
||||
|
||||
public:
|
||||
/** Create immutable column using immutable arguments. This arguments may be shared with other columns.
|
||||
* Use IColumn::mutate in order to make mutable column and mutate shared nested columns.
|
||||
*/
|
||||
using Base = COWPtrHelper<IColumn, ColumnTuple>;
|
||||
static Ptr create(const Columns & columns);
|
||||
|
||||
template <typename Arg, typename = typename std::enable_if<std::is_rvalue_reference<Arg &&>::value>::type>
|
||||
static MutablePtr create(Arg && arg) { return Base::create(std::forward<Arg>(arg)); }
|
||||
|
||||
std::string getName() const override;
|
||||
const char * getFamilyName() const override { return "Tuple"; }
|
||||
|
||||
@ -49,9 +58,9 @@ public:
|
||||
const char * deserializeAndInsertFromArena(const char * pos) override;
|
||||
void updateHashWithValue(size_t n, SipHash & hash) const override;
|
||||
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
|
||||
MutableColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||
MutableColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
MutableColumnPtr replicate(const Offsets & offsets) const override;
|
||||
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override;
|
||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override;
|
||||
ColumnPtr replicate(const Offsets & offsets) const override;
|
||||
MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override;
|
||||
void gather(ColumnGathererStream & gatherer_stream) override;
|
||||
int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override;
|
||||
|
@ -146,7 +146,7 @@ void ColumnVector<T>::insertRangeFrom(const IColumn & src, size_t start, size_t
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
MutableColumnPtr ColumnVector<T>::filter(const IColumn::Filter & filt, ssize_t result_size_hint) const
|
||||
ColumnPtr ColumnVector<T>::filter(const IColumn::Filter & filt, ssize_t result_size_hint) const
|
||||
{
|
||||
size_t size = data.size();
|
||||
if (size != filt.size())
|
||||
@ -210,7 +210,7 @@ MutableColumnPtr ColumnVector<T>::filter(const IColumn::Filter & filt, ssize_t r
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
MutableColumnPtr ColumnVector<T>::permute(const IColumn::Permutation & perm, size_t limit) const
|
||||
ColumnPtr ColumnVector<T>::permute(const IColumn::Permutation & perm, size_t limit) const
|
||||
{
|
||||
size_t size = data.size();
|
||||
|
||||
@ -231,7 +231,7 @@ MutableColumnPtr ColumnVector<T>::permute(const IColumn::Permutation & perm, siz
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
MutableColumnPtr ColumnVector<T>::replicate(const IColumn::Offsets & offsets) const
|
||||
ColumnPtr ColumnVector<T>::replicate(const IColumn::Offsets & offsets) const
|
||||
{
|
||||
size_t size = data.size();
|
||||
if (size != offsets.size())
|
||||
|
@ -243,11 +243,11 @@ public:
|
||||
|
||||
void insertRangeFrom(const IColumn & src, size_t start, size_t length) override;
|
||||
|
||||
MutableColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override;
|
||||
ColumnPtr filter(const IColumn::Filter & filt, ssize_t result_size_hint) const override;
|
||||
|
||||
MutableColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const override;
|
||||
ColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const override;
|
||||
|
||||
MutableColumnPtr replicate(const IColumn::Offsets & offsets) const override;
|
||||
ColumnPtr replicate(const IColumn::Offsets & offsets) const override;
|
||||
|
||||
void getExtremes(Field & min, Field & max) const override;
|
||||
|
||||
|
@ -57,7 +57,8 @@ FilterDescription::FilterDescription(const IColumn & column)
|
||||
|
||||
if (const ColumnNullable * nullable_column = typeid_cast<const ColumnNullable *>(&column))
|
||||
{
|
||||
MutableColumnPtr mutable_holder = nullable_column->getNestedColumn().mutate();
|
||||
ColumnPtr nested_column = nullable_column->getNestedColumnPtr();
|
||||
MutableColumnPtr mutable_holder = (*std::move(nested_column)).mutate();
|
||||
|
||||
ColumnUInt8 * concrete_column = typeid_cast<ColumnUInt8 *>(mutable_holder.get());
|
||||
if (!concrete_column)
|
||||
|
@ -45,7 +45,7 @@ public:
|
||||
/** If column isn't constant, returns nullptr (or itself).
|
||||
* If column is constant, transforms constant to full column (if column type allows such tranform) and return it.
|
||||
*/
|
||||
virtual MutablePtr convertToFullColumnIfConst() const { return {}; }
|
||||
virtual Ptr convertToFullColumnIfConst() const { return {}; }
|
||||
|
||||
/// Creates empty column with the same type.
|
||||
virtual MutablePtr cloneEmpty() const { return cloneResized(0); }
|
||||
@ -104,11 +104,11 @@ public:
|
||||
|
||||
/// Removes all elements outside of specified range.
|
||||
/// Is used in LIMIT operation, for example.
|
||||
virtual MutablePtr cut(size_t start, size_t length) const
|
||||
virtual Ptr cut(size_t start, size_t length) const
|
||||
{
|
||||
MutablePtr res = cloneEmpty();
|
||||
res->insertRangeFrom(*this, start, length);
|
||||
return res;
|
||||
return std::move(res);
|
||||
}
|
||||
|
||||
/// Appends new value at the end of column (column's size is increased by 1).
|
||||
@ -171,12 +171,12 @@ public:
|
||||
* otherwise (i.e. < 0), makes reserve() using size of source column.
|
||||
*/
|
||||
using Filter = PaddedPODArray<UInt8>;
|
||||
virtual MutablePtr filter(const Filter & filt, ssize_t result_size_hint) const = 0;
|
||||
virtual Ptr filter(const Filter & filt, ssize_t result_size_hint) const = 0;
|
||||
|
||||
/// Permutes elements using specified permutation. Is used in sortings.
|
||||
/// limit - if it isn't 0, puts only first limit elements in the result.
|
||||
using Permutation = PaddedPODArray<size_t>;
|
||||
virtual MutablePtr permute(const Permutation & perm, size_t limit) const = 0;
|
||||
virtual Ptr permute(const Permutation & perm, size_t limit) const = 0;
|
||||
|
||||
/** Compares (*this)[n] and rhs[m].
|
||||
* Returns negative number, 0, or positive number (*this)[n] is less, equal, greater than rhs[m] respectively.
|
||||
@ -205,7 +205,7 @@ public:
|
||||
*/
|
||||
using Offset = UInt64;
|
||||
using Offsets = PaddedPODArray<Offset>;
|
||||
virtual MutablePtr replicate(const Offsets & offsets) const = 0;
|
||||
virtual Ptr replicate(const Offsets & offsets) const = 0;
|
||||
|
||||
/** Split column to smaller columns. Each value goes to column index, selected by corresponding element of 'selector'.
|
||||
* Selector must contain values from 0 to num_columns - 1.
|
||||
@ -247,10 +247,10 @@ public:
|
||||
virtual void forEachSubcolumn(ColumnCallback) {}
|
||||
|
||||
|
||||
MutablePtr mutate() const
|
||||
MutablePtr mutate() const &&
|
||||
{
|
||||
MutablePtr res = COWPtr<IColumn>::mutate();
|
||||
res->forEachSubcolumn([](Ptr & subcolumn) { subcolumn = subcolumn->mutate(); });
|
||||
res->forEachSubcolumn([](Ptr & subcolumn) { subcolumn = (*std::move(subcolumn)).mutate(); });
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -360,4 +360,16 @@ using MutableColumns = std::vector<MutableColumnPtr>;
|
||||
using ColumnRawPtrs = std::vector<const IColumn *>;
|
||||
//using MutableColumnRawPtrs = std::vector<IColumn *>;
|
||||
|
||||
template <typename ... Args>
|
||||
struct IsMutableColumns;
|
||||
|
||||
template <typename Arg, typename ... Args>
|
||||
struct IsMutableColumns<Arg, Args ...>
|
||||
{
|
||||
static const bool value = std::is_assignable<MutableColumnPtr &&, Arg>::value && IsMutableColumns<Args ...>::value;
|
||||
};
|
||||
|
||||
template <>
|
||||
struct IsMutableColumns<> { static const bool value = true; };
|
||||
|
||||
}
|
||||
|
@ -74,12 +74,12 @@ public:
|
||||
s += length;
|
||||
}
|
||||
|
||||
MutableColumnPtr filter(const Filter & filt, ssize_t /*result_size_hint*/) const override
|
||||
ColumnPtr filter(const Filter & filt, ssize_t /*result_size_hint*/) const override
|
||||
{
|
||||
return cloneDummy(countBytesInFilter(filt));
|
||||
}
|
||||
|
||||
MutableColumnPtr permute(const Permutation & perm, size_t limit) const override
|
||||
ColumnPtr permute(const Permutation & perm, size_t limit) const override
|
||||
{
|
||||
if (s != perm.size())
|
||||
throw Exception("Size of permutation doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
@ -94,7 +94,7 @@ public:
|
||||
res[i] = i;
|
||||
}
|
||||
|
||||
MutableColumnPtr replicate(const Offsets & offsets) const override
|
||||
ColumnPtr replicate(const Offsets & offsets) const override
|
||||
{
|
||||
if (s != offsets.size())
|
||||
throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#if !(defined(__FreeBSD__) || defined(__APPLE__))
|
||||
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
@ -68,10 +68,9 @@
|
||||
* of this shared state.
|
||||
*
|
||||
* Caveats:
|
||||
* - after a call to 'mutate' method, you can still have a reference to immutable ptr somewhere
|
||||
* and it can still become shared. Also it would be better to make 'mutate' method rvalue-qualified.
|
||||
* - after a call to 'mutate' method, you can still have a reference to immutable ptr somewhere.
|
||||
* - as 'mutable_ptr' should be unique, it's refcount is redundant - probably it would be better
|
||||
* to use std::unique_ptr for it, but see above.
|
||||
* to use std::unique_ptr for it somehow.
|
||||
*/
|
||||
template <typename Derived>
|
||||
class COWPtr : public boost::intrusive_ref_counter<Derived>
|
||||
@ -80,12 +79,22 @@ private:
|
||||
Derived * derived() { return static_cast<Derived *>(this); }
|
||||
const Derived * derived() const { return static_cast<const Derived *>(this); }
|
||||
|
||||
template <typename T>
|
||||
class IntrusivePtr : public boost::intrusive_ptr<T>
|
||||
{
|
||||
public:
|
||||
using boost::intrusive_ptr<T>::intrusive_ptr;
|
||||
|
||||
T & operator*() const & { return boost::intrusive_ptr<T>::operator*(); }
|
||||
T && operator*() const && { return const_cast<typename std::remove_const<T>::type &&>(*boost::intrusive_ptr<T>::get()); }
|
||||
};
|
||||
|
||||
protected:
|
||||
template <typename T>
|
||||
class mutable_ptr : public boost::intrusive_ptr<T>
|
||||
class mutable_ptr : public IntrusivePtr<T>
|
||||
{
|
||||
private:
|
||||
using Base = boost::intrusive_ptr<T>;
|
||||
using Base = IntrusivePtr<T>;
|
||||
|
||||
template <typename> friend class COWPtr;
|
||||
template <typename, typename> friend class COWPtrHelper;
|
||||
@ -114,10 +123,10 @@ public:
|
||||
|
||||
protected:
|
||||
template <typename T>
|
||||
class immutable_ptr : public boost::intrusive_ptr<const T>
|
||||
class immutable_ptr : public IntrusivePtr<const T>
|
||||
{
|
||||
private:
|
||||
using Base = boost::intrusive_ptr<const T>;
|
||||
using Base = IntrusivePtr<const T>;
|
||||
|
||||
template <typename> friend class COWPtr;
|
||||
template <typename, typename> friend class COWPtrHelper;
|
||||
|
@ -276,6 +276,8 @@ void ConfigProcessor::doIncludesRecursive(
|
||||
/// Replace the original contents, not add to it.
|
||||
bool replace = attributes->getNamedItem("replace");
|
||||
|
||||
bool included_something = false;
|
||||
|
||||
auto process_include = [&](const Node * include_attr, const std::function<const Node * (const std::string &)> & get_node, const char * error_msg)
|
||||
{
|
||||
std::string name = include_attr->getNodeValue();
|
||||
@ -316,6 +318,8 @@ void ConfigProcessor::doIncludesRecursive(
|
||||
{
|
||||
element->setAttributeNode(dynamic_cast<Attr *>(config->importNode(from_attrs->item(i), true)));
|
||||
}
|
||||
|
||||
included_something = true;
|
||||
}
|
||||
};
|
||||
|
||||
@ -348,11 +352,14 @@ void ConfigProcessor::doIncludesRecursive(
|
||||
}
|
||||
}
|
||||
|
||||
NodeListPtr children = node->childNodes();
|
||||
Node * child = nullptr;
|
||||
for (size_t i = 0; (child = children->item(i)); ++i)
|
||||
if (included_something)
|
||||
doIncludesRecursive(config, include_from, node, zk_node_cache, contributing_zk_paths);
|
||||
else
|
||||
{
|
||||
doIncludesRecursive(config, include_from, child, zk_node_cache, contributing_zk_paths);
|
||||
NodeListPtr children = node->childNodes();
|
||||
Node * child = nullptr;
|
||||
for (size_t i = 0; (child = children->item(i)); ++i)
|
||||
doIncludesRecursive(config, include_from, child, zk_node_cache, contributing_zk_paths);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -125,7 +125,6 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_SETTING = 115;
|
||||
extern const int THERE_IS_NO_DEFAULT_VALUE = 116;
|
||||
extern const int INCORRECT_DATA = 117;
|
||||
extern const int TABLE_METADATA_DOESNT_EXIST = 118;
|
||||
extern const int ENGINE_REQUIRED = 119;
|
||||
extern const int CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE = 120;
|
||||
extern const int UNKNOWN_SET_DATA_VARIANT = 121;
|
||||
|
@ -1,13 +1,12 @@
|
||||
#include <sys/types.h>
|
||||
#include <sys/wait.h>
|
||||
#include <unistd.h>
|
||||
#include <fcntl.h>
|
||||
#include <dlfcn.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ShellCommand.h>
|
||||
#include <IO/WriteBufferFromVector.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <port/unistd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -27,6 +27,9 @@ namespace ErrorCodes
|
||||
class Throttler
|
||||
{
|
||||
public:
|
||||
Throttler(size_t max_speed_, const std::shared_ptr<Throttler> & parent = nullptr)
|
||||
: max_speed(max_speed_), limit_exceeded_exception_message(""), parent(parent) {}
|
||||
|
||||
Throttler(size_t max_speed_, size_t limit_, const char * limit_exceeded_exception_message_,
|
||||
const std::shared_ptr<Throttler> & parent = nullptr)
|
||||
: max_speed(max_speed_), limit(limit_), limit_exceeded_exception_message(limit_exceeded_exception_message_), parent(parent) {}
|
||||
@ -76,6 +79,12 @@ public:
|
||||
parent->add(amount);
|
||||
}
|
||||
|
||||
/// Not thread safe
|
||||
void setParent(const std::shared_ptr<Throttler> & parent_)
|
||||
{
|
||||
parent = parent_;
|
||||
}
|
||||
|
||||
void reset()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
@ -3,6 +3,9 @@
|
||||
#include <Core/Types.h>
|
||||
#include <Common/BitHelpers.h>
|
||||
|
||||
#if __SSE2__
|
||||
#include <emmintrin.h>
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -49,9 +52,29 @@ inline size_t seqLength(const UInt8 first_octet)
|
||||
inline size_t countCodePoints(const UInt8 * data, size_t size)
|
||||
{
|
||||
size_t res = 0;
|
||||
const auto end = data + size;
|
||||
|
||||
/// TODO SIMD implementation looks quite simple.
|
||||
for (auto end = data + size; data < end; ++data) /// Skip UTF-8 continuation bytes.
|
||||
#if __SSE2__
|
||||
const auto bytes_sse = sizeof(__m128i);
|
||||
const auto src_end_sse = (data + size) - (size % bytes_sse);
|
||||
|
||||
const auto align_sse = _mm_set1_epi8(0x40);
|
||||
const auto upper_bound = _mm_set1_epi8(0xBF);
|
||||
|
||||
for (; data < src_end_sse; data += bytes_sse)
|
||||
{
|
||||
const auto chars = _mm_loadu_si128(reinterpret_cast<const __m128i *>(data));
|
||||
|
||||
///Align to zero for the solve two case
|
||||
const auto align_res = _mm_adds_epu8(chars, align_sse);
|
||||
const auto less_than_and_equals = _mm_cmpeq_epi8(_mm_min_epu8(align_res, upper_bound), align_res);
|
||||
|
||||
res += __builtin_popcount(_mm_movemask_epi8(less_than_and_equals));
|
||||
}
|
||||
|
||||
#endif
|
||||
|
||||
for (; data < end; ++data) /// Skip UTF-8 continuation bytes.
|
||||
res += (*data <= 0x7F || *data >= 0xC0);
|
||||
|
||||
return res;
|
||||
|
@ -1,4 +1,3 @@
|
||||
|
||||
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
|
||||
|
||||
add_headers_and_sources(clickhouse_common_zookeeper .)
|
||||
|
@ -29,11 +29,11 @@ public:
|
||||
if (zookeeper->tryGet(path, result_str, &stat))
|
||||
{
|
||||
result = std::stol(result_str) + 1;
|
||||
success = zookeeper->trySet(path, std::to_string(result), stat.version) == ZOK;
|
||||
success = zookeeper->trySet(path, std::to_string(result), stat.version) == ZooKeeperImpl::ZooKeeper::ZOK;
|
||||
}
|
||||
else
|
||||
{
|
||||
success = zookeeper->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == ZOK;
|
||||
success = zookeeper->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == ZooKeeperImpl::ZooKeeper::ZOK;
|
||||
}
|
||||
}
|
||||
while (!success);
|
||||
|
@ -23,32 +23,23 @@ namespace zkutil
|
||||
|
||||
|
||||
/// You should reinitialize ZooKeeper session in case of these errors
|
||||
inline bool isUnrecoverableErrorCode(int32_t zk_return_code)
|
||||
inline bool isHardwareError(int32_t zk_return_code)
|
||||
{
|
||||
return zk_return_code == ZINVALIDSTATE || zk_return_code == ZSESSIONEXPIRED || zk_return_code == ZSESSIONMOVED;
|
||||
}
|
||||
|
||||
/// Errors related with temporary network problems
|
||||
inline bool isTemporaryErrorCode(int32_t zk_return_code)
|
||||
{
|
||||
return zk_return_code == ZCONNECTIONLOSS || zk_return_code == ZOPERATIONTIMEOUT;
|
||||
}
|
||||
|
||||
/// Any error related with network or master election
|
||||
/// In case of these errors you should retry the query or reinitialize ZooKeeper session (see isUnrecoverable())
|
||||
inline bool isHardwareErrorCode(int32_t zk_return_code)
|
||||
{
|
||||
return isUnrecoverableErrorCode(zk_return_code) || isTemporaryErrorCode(zk_return_code);
|
||||
return zk_return_code == ZooKeeperImpl::ZooKeeper::ZINVALIDSTATE
|
||||
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED
|
||||
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZSESSIONMOVED
|
||||
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZCONNECTIONLOSS
|
||||
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZOPERATIONTIMEOUT;
|
||||
}
|
||||
|
||||
/// Valid errors sent from server
|
||||
inline bool isUserError(int32_t zk_return_code)
|
||||
{
|
||||
return zk_return_code == ZNONODE
|
||||
|| zk_return_code == ZBADVERSION
|
||||
|| zk_return_code == ZNOCHILDRENFOREPHEMERALS
|
||||
|| zk_return_code == ZNODEEXISTS
|
||||
|| zk_return_code == ZNOTEMPTY;
|
||||
return zk_return_code == ZooKeeperImpl::ZooKeeper::ZNONODE
|
||||
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZBADVERSION
|
||||
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZNOCHILDRENFOREPHEMERALS
|
||||
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS
|
||||
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY;
|
||||
}
|
||||
|
||||
|
||||
@ -60,12 +51,11 @@ private:
|
||||
: DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION), code(code) { incrementEventCounter(); }
|
||||
|
||||
public:
|
||||
explicit KeeperException(const std::string & msg) : KeeperException(msg, ZOK, 0) {}
|
||||
KeeperException(const std::string & msg, const int32_t code)
|
||||
: KeeperException(msg + " (" + zerror(code) + ")", code, 0) {}
|
||||
explicit KeeperException(const int32_t code) : KeeperException(zerror(code), code, 0) {}
|
||||
: KeeperException(msg + " (" + ZooKeeperImpl::ZooKeeper::errorMessage(code) + ")", code, 0) {}
|
||||
explicit KeeperException(const int32_t code) : KeeperException(ZooKeeperImpl::ZooKeeper::errorMessage(code), code, 0) {}
|
||||
KeeperException(const int32_t code, const std::string & path)
|
||||
: KeeperException(std::string{zerror(code)} + ", path: " + path, code, 0) {}
|
||||
: KeeperException(std::string{ZooKeeperImpl::ZooKeeper::errorMessage(code)} + ", path: " + path, code, 0) {}
|
||||
|
||||
KeeperException(const KeeperException & exc) : DB::Exception(exc), code(exc.code) { incrementEventCounter(); }
|
||||
|
||||
@ -73,23 +63,11 @@ public:
|
||||
const char * className() const throw() override { return "zkutil::KeeperException"; }
|
||||
KeeperException * clone() const override { return new KeeperException(*this); }
|
||||
|
||||
/// You should reinitialize ZooKeeper session in case of these errors
|
||||
bool isUnrecoverable() const
|
||||
{
|
||||
return isUnrecoverableErrorCode(code);
|
||||
}
|
||||
|
||||
/// Errors related with temporary network problems
|
||||
bool isTemporaryError() const
|
||||
{
|
||||
return isTemporaryErrorCode(code);
|
||||
}
|
||||
|
||||
/// Any error related with network or master election
|
||||
/// In case of these errors you should retry the query or reinitialize ZooKeeper session (see isUnrecoverable())
|
||||
/// In case of these errors you should reinitialize ZooKeeper session.
|
||||
bool isHardwareError() const
|
||||
{
|
||||
return isHardwareErrorCode(code);
|
||||
return zkutil::isHardwareError(code);
|
||||
}
|
||||
|
||||
const int32_t code;
|
||||
@ -106,15 +84,20 @@ private:
|
||||
class KeeperMultiException : public KeeperException
|
||||
{
|
||||
public:
|
||||
MultiTransactionInfo info;
|
||||
Requests requests;
|
||||
Responses responses;
|
||||
size_t failed_op_index = 0;
|
||||
|
||||
std::string getPathForFirstFailedOp() const;
|
||||
|
||||
/// If it is user error throws KeeperMultiException else throws ordinary KeeperException
|
||||
/// If it is ZOK does nothing
|
||||
static void check(const MultiTransactionInfo & info);
|
||||
static void check(int code, const Ops & ops, const OpResultsPtr & op_results);
|
||||
static void check(int32_t code, const Requests & requests, const Responses & responses);
|
||||
|
||||
protected:
|
||||
KeeperMultiException(const MultiTransactionInfo & info, size_t failed_op_index);
|
||||
KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses);
|
||||
|
||||
private:
|
||||
size_t getFailedOpIndex(int32_t code, const Responses & responses) const;
|
||||
};
|
||||
|
||||
};
|
||||
|
@ -76,41 +76,9 @@ private:
|
||||
std::string node_path = node->getPath();
|
||||
node_name = node_path.substr(node_path.find_last_of('/') + 1);
|
||||
|
||||
cleanOldEphemeralNodes();
|
||||
|
||||
thread = std::thread(&LeaderElection::threadFunction, this);
|
||||
}
|
||||
|
||||
void cleanOldEphemeralNodes()
|
||||
{
|
||||
if (identifier.empty())
|
||||
return;
|
||||
|
||||
/** If there are nodes with same identifier, remove them.
|
||||
* Such nodes could still be alive after failed attempt of removal,
|
||||
* if it was temporary communication failure, that was continued for more than session timeout,
|
||||
* but ZK session is still alive for unknown reason, and someone still holds that ZK session.
|
||||
* See comments in destructor of EphemeralNodeHolder.
|
||||
*/
|
||||
Strings brothers = zookeeper.getChildren(path);
|
||||
for (const auto & brother : brothers)
|
||||
{
|
||||
if (brother == node_name)
|
||||
continue;
|
||||
|
||||
std::string brother_path = path + "/" + brother;
|
||||
std::string brother_identifier = zookeeper.get(brother_path);
|
||||
|
||||
if (brother_identifier == identifier)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::ObsoleteEphemeralNode);
|
||||
LOG_WARNING(&Logger::get("LeaderElection"), "Found obsolete ephemeral node for identifier "
|
||||
+ identifier + ", removing: " + brother_path);
|
||||
zookeeper.tryRemoveWithRetries(brother_path);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void releaseNode()
|
||||
{
|
||||
shutdown = true;
|
||||
|
@ -11,31 +11,16 @@ bool Lock::tryLock()
|
||||
if (tryCheck() != Status::LOCKED_BY_ME)
|
||||
locked.reset(nullptr);
|
||||
}
|
||||
|
||||
if (!locked)
|
||||
else
|
||||
{
|
||||
size_t attempt;
|
||||
std::string dummy;
|
||||
int32_t code = zookeeper->tryCreate(lock_path, lock_message, zkutil::CreateMode::Ephemeral, dummy);
|
||||
|
||||
/// TODO: ошибка. можно создать эфемерную ноду, но при этом не получить подтверждения даже после нескольких попыток.
|
||||
/// тогда все последующие локи будут неуспешные из-за существования ноды.
|
||||
int32_t code = zookeeper->tryCreateWithRetries(lock_path, lock_message, zkutil::CreateMode::Ephemeral, dummy, &attempt);
|
||||
|
||||
if (code == ZNODEEXISTS)
|
||||
if (code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
|
||||
{
|
||||
if (attempt == 0)
|
||||
locked.reset(nullptr);
|
||||
else
|
||||
{
|
||||
zkutil::Stat stat;
|
||||
zookeeper->get(lock_path, &stat);
|
||||
if (stat.ephemeralOwner == zookeeper->getClientID())
|
||||
locked.reset(new ZooKeeperHandler(zookeeper));
|
||||
else
|
||||
locked.reset(nullptr);
|
||||
}
|
||||
locked.reset(nullptr);
|
||||
}
|
||||
else if (code == ZOK)
|
||||
else if (code == ZooKeeperImpl::ZooKeeper::ZOK)
|
||||
{
|
||||
locked.reset(new ZooKeeperHandler(zookeeper));
|
||||
}
|
||||
@ -52,34 +37,8 @@ void Lock::unlock()
|
||||
if (locked)
|
||||
{
|
||||
auto zookeeper = zookeeper_holder->getZooKeeper();
|
||||
try
|
||||
{
|
||||
if (tryCheck() == Status::LOCKED_BY_ME)
|
||||
{
|
||||
size_t attempt;
|
||||
int32_t code = zookeeper->tryRemoveEphemeralNodeWithRetries(lock_path, -1, &attempt);
|
||||
|
||||
if (attempt)
|
||||
{
|
||||
if (code != ZOK)
|
||||
throw zkutil::KeeperException(code);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (code == ZNONODE)
|
||||
LOG_ERROR(log, "Node " << lock_path << " has been already removed. Probably due to network error.");
|
||||
else if (code != ZOK)
|
||||
throw zkutil::KeeperException(code);
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
{
|
||||
/// если сессия находится в невостанавливаемом состоянии, то эфемерные ноды нам больше не принадлежат
|
||||
/// и лок через таймаут будет отпущен
|
||||
if (!e.isUnrecoverable())
|
||||
throw;
|
||||
}
|
||||
if (tryCheck() == Status::LOCKED_BY_ME)
|
||||
zookeeper->remove(lock_path, -1);
|
||||
locked.reset(nullptr);
|
||||
}
|
||||
}
|
||||
@ -97,47 +56,17 @@ Lock::Status Lock::tryCheck() const
|
||||
else
|
||||
{
|
||||
if (stat.ephemeralOwner == zookeeper->getClientID())
|
||||
{
|
||||
lock_status = LOCKED_BY_ME;
|
||||
}
|
||||
else
|
||||
{
|
||||
lock_status = LOCKED_BY_OTHER;
|
||||
}
|
||||
}
|
||||
|
||||
if (locked && lock_status != LOCKED_BY_ME)
|
||||
LOG_WARNING(log, "Lock is lost. It is normal if session was reinitialized. Path: " << lock_path << "/" << lock_message);
|
||||
LOG_WARNING(log, "Lock is lost. It is normal if session was expired. Path: " << lock_path << "/" << lock_message);
|
||||
|
||||
return lock_status;
|
||||
}
|
||||
|
||||
std::string Lock::status2String(Status status)
|
||||
{
|
||||
if (status >= END)
|
||||
throw zkutil::KeeperException("Wrong status code: " + std::to_string(status));
|
||||
static const char * names[] = {"Unlocked", "Locked by me", "Locked by other"};
|
||||
return names[status];
|
||||
}
|
||||
|
||||
void Lock::unlockOrMoveIfFailed(std::vector<zkutil::Lock> & failed_to_unlock_locks)
|
||||
{
|
||||
try
|
||||
{
|
||||
unlock();
|
||||
}
|
||||
catch (const zkutil::KeeperException & e)
|
||||
{
|
||||
if (e.isTemporaryError())
|
||||
{
|
||||
LOG_WARNING(log, "Fail to unlock lock. Move lock to vector to remove later. Path: " << getPath());
|
||||
failed_to_unlock_locks.emplace_back(std::move(*this));
|
||||
}
|
||||
else
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void Lock::unlockAssumeLockNodeRemovedManually()
|
||||
{
|
||||
locked.reset(nullptr);
|
||||
|
@ -51,15 +51,12 @@ namespace zkutil
|
||||
UNLOCKED,
|
||||
LOCKED_BY_ME,
|
||||
LOCKED_BY_OTHER,
|
||||
END
|
||||
};
|
||||
std::string status2String(Status status);
|
||||
|
||||
/// проверяет создана ли эфемерная нода и кто ее владелец.
|
||||
Status tryCheck() const;
|
||||
|
||||
void unlock();
|
||||
void unlockOrMoveIfFailed(std::vector<zkutil::Lock> & failed_to_unlock_locks);
|
||||
void unlockAssumeLockNodeRemovedManually();
|
||||
|
||||
bool tryLock();
|
||||
|
@ -3,184 +3,17 @@
|
||||
#include <future>
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
#include <zookeeper.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperImpl.h>
|
||||
#include <Poco/Event.h>
|
||||
|
||||
|
||||
namespace zkutil
|
||||
{
|
||||
|
||||
using ACLPtr = const ACL_vector *;
|
||||
using Stat = ::Stat;
|
||||
class ZooKeeper;
|
||||
|
||||
|
||||
struct Op
|
||||
{
|
||||
public:
|
||||
Op() : data(new zoo_op_t) {}
|
||||
virtual ~Op() {}
|
||||
|
||||
virtual std::shared_ptr<Op> clone() const = 0;
|
||||
|
||||
virtual std::string getPath() const = 0;
|
||||
|
||||
virtual std::string describe() const = 0;
|
||||
|
||||
std::unique_ptr<zoo_op_t> data;
|
||||
|
||||
struct Remove;
|
||||
struct Create;
|
||||
struct SetData;
|
||||
struct Check;
|
||||
};
|
||||
|
||||
using OpPtr = std::shared_ptr<Op>;
|
||||
|
||||
|
||||
struct Op::Remove : public Op
|
||||
{
|
||||
Remove(const std::string & path_, int32_t version_) :
|
||||
path(path_), version(version_)
|
||||
{
|
||||
zoo_delete_op_init(data.get(), path.c_str(), version);
|
||||
}
|
||||
|
||||
OpPtr clone() const override
|
||||
{
|
||||
return std::make_shared<Remove>(path, version);
|
||||
}
|
||||
|
||||
std::string getPath() const override { return path; }
|
||||
|
||||
std::string describe() const override { return "command: remove, path: " + path; }
|
||||
|
||||
private:
|
||||
std::string path;
|
||||
int32_t version;
|
||||
};
|
||||
|
||||
struct Op::Create : public Op
|
||||
{
|
||||
Create(const std::string & path_pattern_, const std::string & value_, ACLPtr acl_, int32_t flags_);
|
||||
|
||||
OpPtr clone() const override
|
||||
{
|
||||
return std::make_shared<Create>(path_pattern, value, acl, flags);
|
||||
}
|
||||
|
||||
std::string getPathCreated() { return created_path.data(); }
|
||||
|
||||
std::string getPath() const override { return path_pattern; }
|
||||
|
||||
std::string describe() const override
|
||||
{
|
||||
return "command: create"
|
||||
", path: " + path_pattern +
|
||||
", value: " + value;
|
||||
}
|
||||
|
||||
private:
|
||||
std::string path_pattern;
|
||||
std::string value;
|
||||
ACLPtr acl;
|
||||
int32_t flags;
|
||||
std::vector<char> created_path;
|
||||
};
|
||||
|
||||
struct Op::SetData : public Op
|
||||
{
|
||||
SetData(const std::string & path_, const std::string & value_, int32_t version_) :
|
||||
path(path_), value(value_), version(version_)
|
||||
{
|
||||
zoo_set_op_init(data.get(), path.c_str(), value.c_str(), value.size(), version, &stat);
|
||||
}
|
||||
|
||||
OpPtr clone() const override
|
||||
{
|
||||
return std::make_shared<SetData>(path, value, version);
|
||||
}
|
||||
|
||||
std::string getPath() const override { return path; }
|
||||
|
||||
std::string describe() const override
|
||||
{
|
||||
return
|
||||
"command: set"
|
||||
", path: " + path +
|
||||
", value: " + value +
|
||||
", version: " + std::to_string(data->set_op.version);
|
||||
}
|
||||
|
||||
private:
|
||||
std::string path;
|
||||
std::string value;
|
||||
int32_t version;
|
||||
Stat stat;
|
||||
};
|
||||
|
||||
struct Op::Check : public Op
|
||||
{
|
||||
Check(const std::string & path_, int32_t version_) :
|
||||
path(path_), version(version_)
|
||||
{
|
||||
zoo_check_op_init(data.get(), path.c_str(), version);
|
||||
}
|
||||
|
||||
OpPtr clone() const override
|
||||
{
|
||||
return std::make_shared<Check>(path, version);
|
||||
}
|
||||
|
||||
std::string getPath() const override { return path; }
|
||||
|
||||
std::string describe() const override { return "command: check, path: " + path; }
|
||||
|
||||
private:
|
||||
std::string path;
|
||||
int32_t version;
|
||||
};
|
||||
|
||||
using Ops = std::vector<OpPtr>;
|
||||
|
||||
|
||||
/// C++ version of zoo_op_result_t
|
||||
struct OpResult
|
||||
{
|
||||
int err;
|
||||
std::string value;
|
||||
std::unique_ptr<Stat> stat;
|
||||
|
||||
/// ZooKeeper is required for correct chroot path prefixes handling
|
||||
explicit OpResult(const zoo_op_result_t & op_result, const ZooKeeper * zookeeper = nullptr);
|
||||
};
|
||||
using OpResults = std::vector<OpResult>;
|
||||
using OpResultsPtr = std::shared_ptr<OpResults>;
|
||||
using Stat = ZooKeeperImpl::ZooKeeper::Stat;
|
||||
using Strings = std::vector<std::string>;
|
||||
|
||||
|
||||
/// Simple structure to handle transaction execution results
|
||||
struct MultiTransactionInfo
|
||||
{
|
||||
Ops ops;
|
||||
int32_t code = ZOK;
|
||||
OpResultsPtr op_results;
|
||||
|
||||
MultiTransactionInfo() = default;
|
||||
|
||||
MultiTransactionInfo(int32_t code_, const Ops & ops_, const OpResultsPtr & op_results_)
|
||||
: ops(ops_), code(code_), op_results(op_results_) {}
|
||||
|
||||
bool empty() const
|
||||
{
|
||||
return ops.empty();
|
||||
}
|
||||
|
||||
/// Returns failed op if zkutil::isUserError(code) is true
|
||||
const Op & getFailedOp() const;
|
||||
};
|
||||
|
||||
|
||||
namespace CreateMode
|
||||
{
|
||||
extern const int Persistent;
|
||||
@ -191,23 +24,39 @@ namespace CreateMode
|
||||
|
||||
using EventPtr = std::shared_ptr<Poco::Event>;
|
||||
|
||||
class ZooKeeper;
|
||||
|
||||
/// Callback to call when the watch fires.
|
||||
/// Because callbacks are called in the single "completion" thread internal to libzookeeper,
|
||||
/// they must execute as quickly as possible (preferably just set some notification).
|
||||
/// Parameters:
|
||||
/// zookeeper - zookeeper session to which the fired watch belongs
|
||||
/// type - event type, one of the *_EVENT constants from zookeeper.h
|
||||
/// state - session connection state, one of the *_STATE constants from zookeeper.h
|
||||
/// path - znode path to which the change happened. if event == ZOO_SESSION_EVENT it is either NULL or empty string.
|
||||
using WatchCallback = std::function<void(ZooKeeper & zookeeper, int type, int state, const char * path)>;
|
||||
using WatchCallback = ZooKeeperImpl::ZooKeeper::WatchCallback;
|
||||
|
||||
using Request = ZooKeeperImpl::ZooKeeper::Request;
|
||||
using Response = ZooKeeperImpl::ZooKeeper::Response;
|
||||
|
||||
/// Returns first op which code != ZOK or throws an exception
|
||||
/// ZooKeeper client sets correct OP codes if the transaction fails because of logical (user) errors like ZNODEEXISTS
|
||||
/// If it is failed because of network error, for example, OP codes is not set.
|
||||
/// Therefore you should make zkutil::isUserError() check before the function invocation.
|
||||
size_t getFailedOpIndex(const OpResultsPtr & op_results, int32_t transaction_return_code);
|
||||
using RequestPtr = ZooKeeperImpl::ZooKeeper::RequestPtr;
|
||||
using ResponsePtr = ZooKeeperImpl::ZooKeeper::ResponsePtr;
|
||||
|
||||
using Requests = ZooKeeperImpl::ZooKeeper::Requests;
|
||||
using Responses = ZooKeeperImpl::ZooKeeper::Responses;
|
||||
|
||||
using CreateRequest = ZooKeeperImpl::ZooKeeper::CreateRequest;
|
||||
using RemoveRequest = ZooKeeperImpl::ZooKeeper::RemoveRequest;
|
||||
using ExistsRequest = ZooKeeperImpl::ZooKeeper::ExistsRequest;
|
||||
using GetRequest = ZooKeeperImpl::ZooKeeper::GetRequest;
|
||||
using SetRequest = ZooKeeperImpl::ZooKeeper::SetRequest;
|
||||
using ListRequest = ZooKeeperImpl::ZooKeeper::ListRequest;
|
||||
using CheckRequest = ZooKeeperImpl::ZooKeeper::CheckRequest;
|
||||
|
||||
using CreateResponse = ZooKeeperImpl::ZooKeeper::CreateResponse;
|
||||
using RemoveResponse = ZooKeeperImpl::ZooKeeper::RemoveResponse;
|
||||
using ExistsResponse = ZooKeeperImpl::ZooKeeper::ExistsResponse;
|
||||
using GetResponse = ZooKeeperImpl::ZooKeeper::GetResponse;
|
||||
using SetResponse = ZooKeeperImpl::ZooKeeper::SetResponse;
|
||||
using ListResponse = ZooKeeperImpl::ZooKeeper::ListResponse;
|
||||
using CheckResponse = ZooKeeperImpl::ZooKeeper::CheckResponse;
|
||||
|
||||
RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode);
|
||||
RequestPtr makeRemoveRequest(const std::string & path, int version);
|
||||
RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version);
|
||||
RequestPtr makeCheckRequest(const std::string & path, int version);
|
||||
|
||||
}
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -8,10 +8,10 @@
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
#include <string>
|
||||
#include <unistd.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <port/unistd.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -29,14 +29,10 @@ namespace zkutil
|
||||
{
|
||||
|
||||
const UInt32 DEFAULT_SESSION_TIMEOUT = 30000;
|
||||
const UInt32 MEDIUM_SESSION_TIMEOUT = 120000;
|
||||
const UInt32 BIG_SESSION_TIMEOUT = 600000;
|
||||
|
||||
/// Preferred size of multi() command (in number of ops)
|
||||
constexpr size_t MULTI_BATCH_SIZE = 100;
|
||||
|
||||
struct WatchContext;
|
||||
struct MultiTransactionInfo;
|
||||
|
||||
/// ZooKeeper session. The interface is substantially different from the usual libzookeeper API.
|
||||
///
|
||||
@ -77,27 +73,15 @@ public:
|
||||
*/
|
||||
ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name);
|
||||
|
||||
~ZooKeeper();
|
||||
|
||||
/// Creates a new session with the same parameters. This method can be used for reconnecting
|
||||
/// after the session has expired.
|
||||
/// This object remains unchanged, and the new session is returned.
|
||||
Ptr startNewSession() const;
|
||||
|
||||
/// Returns true, if the session has expired forever.
|
||||
/// This is possible only if the connection has been established, then lost and re-established
|
||||
/// again, but too late.
|
||||
/// In contrast, if, for instance, the server name or port is misconfigured, connection
|
||||
/// attempts will continue indefinitely, expired() will return false and all method calls
|
||||
/// will raise ConnectionLoss exception.
|
||||
/// Also returns true if is_dirty flag is set - a request to close the session ASAP.
|
||||
/// Returns true, if the session has expired.
|
||||
bool expired();
|
||||
|
||||
ACLPtr getDefaultACL();
|
||||
|
||||
void setDefaultACL(ACLPtr new_acl);
|
||||
|
||||
/// Create a znode. ACL set by setDefaultACL is used (full access to everybody by default).
|
||||
/// Create a znode.
|
||||
/// Throw an exception if something went wrong.
|
||||
std::string create(const std::string & path, const std::string & data, int32_t mode);
|
||||
|
||||
@ -108,12 +92,9 @@ public:
|
||||
/// In case of other errors throws an exception.
|
||||
int32_t tryCreate(const std::string & path, const std::string & data, int32_t mode, std::string & path_created);
|
||||
int32_t tryCreate(const std::string & path, const std::string & data, int32_t mode);
|
||||
int32_t tryCreateWithRetries(const std::string & path, const std::string & data, int32_t mode,
|
||||
std::string & path_created, size_t * attempt = nullptr);
|
||||
|
||||
/// Create a Persistent node.
|
||||
/// Does nothing if the node already exists.
|
||||
/// Retries on ConnectionLoss or OperationTimeout.
|
||||
void createIfNotExists(const std::string & path, const std::string & data);
|
||||
|
||||
/// Creates all non-existent ancestors of the given path with empty contents.
|
||||
@ -123,42 +104,11 @@ public:
|
||||
/// Remove the node if the version matches. (if version == -1, remove any version).
|
||||
void remove(const std::string & path, int32_t version = -1);
|
||||
|
||||
/// Removes the node. In case of network errors tries to remove again.
|
||||
/// ZNONODE error for the second and the following tries is ignored.
|
||||
void removeWithRetries(const std::string & path, int32_t version = -1);
|
||||
|
||||
/// Doesn't throw in the following cases:
|
||||
/// * The node doesn't exist
|
||||
/// * Versions don't match
|
||||
/// * The node has children.
|
||||
int32_t tryRemove(const std::string & path, int32_t version = -1);
|
||||
/// Retries in case of network errors, returns ZNONODE if the node is already removed.
|
||||
int32_t tryRemoveWithRetries(const std::string & path, int32_t version = -1, size_t * attempt = nullptr);
|
||||
|
||||
/// The same, but sets is_dirty flag if all removal attempts were unsuccessful.
|
||||
/// This is needed because the session might still exist after all retries,
|
||||
/// even if more time than session_timeout has passed.
|
||||
/// So we do not rely on the ephemeral node being deleted and set is_dirty to
|
||||
/// try and close the session ASAP.
|
||||
/** Ridiculously Long Delay to Expire
|
||||
When disconnects do happen, the common case should be a very* quick
|
||||
reconnect to another server, but an extended network outage may
|
||||
introduce a long delay before a client can reconnect to the ZooKeep‐
|
||||
er service. Some developers wonder why the ZooKeeper client li‐
|
||||
brary doesn’t simply decide at some point (perhaps twice the session
|
||||
timeout) that enough is enough and kill the session itself.
|
||||
There are two answers to this. First, ZooKeeper leaves this kind of
|
||||
policy decision up to the developer. Developers can easily implement
|
||||
such a policy by closing the handle themselves. Second, when a Zoo‐
|
||||
Keeper ensemble goes down, time freezes. Thus, when the ensemble is
|
||||
brought back up, session timeouts are restarted. If processes using
|
||||
ZooKeeper hang in there, they may find out that the long timeout was
|
||||
due to an extended ensemble failure that has recovered and pick right
|
||||
up where they left off without any additional startup delay.
|
||||
|
||||
ZooKeeper: Distributed Process Coordination p118
|
||||
*/
|
||||
int32_t tryRemoveEphemeralNodeWithRetries(const std::string & path, int32_t version = -1, size_t * attempt = nullptr);
|
||||
|
||||
bool exists(const std::string & path, Stat * stat = nullptr, const EventPtr & watch = nullptr);
|
||||
bool existsWatch(const std::string & path, Stat * stat, const WatchCallback & watch_callback);
|
||||
@ -195,16 +145,14 @@ public:
|
||||
|
||||
/// Performs several operations in a transaction.
|
||||
/// Throws on every error.
|
||||
OpResultsPtr multi(const Ops & ops);
|
||||
Responses multi(const Requests & requests);
|
||||
/// Throws only if some operation has returned an "unexpected" error
|
||||
/// - an error that would cause the corresponding try- method to throw.
|
||||
int32_t tryMulti(const Ops & ops, OpResultsPtr * out_results = nullptr);
|
||||
/// Use only with read-only operations.
|
||||
int32_t tryMultiWithRetries(const Ops & ops, OpResultsPtr * out_results = nullptr, size_t * attempt = nullptr);
|
||||
int32_t tryMulti(const Requests & requests, Responses & responses);
|
||||
/// Throws nothing, just alias of multiImpl
|
||||
int32_t tryMultiNoThrow(const Ops & ops, OpResultsPtr * out_op_results = nullptr, MultiTransactionInfo * out_info = nullptr)
|
||||
int32_t tryMultiNoThrow(const Requests & requests, Responses & responses)
|
||||
{
|
||||
return multiImpl(ops, out_op_results, out_info);
|
||||
return multiImpl(requests, responses);
|
||||
}
|
||||
|
||||
Int64 getClientID();
|
||||
@ -237,182 +185,47 @@ public:
|
||||
///
|
||||
/// Future should not be destroyed before the result is gotten.
|
||||
|
||||
template <typename Result, typename... TaskParams>
|
||||
class Future
|
||||
{
|
||||
friend class ZooKeeper;
|
||||
private:
|
||||
using Task = std::packaged_task<Result (TaskParams...)>;
|
||||
using TaskPtr = std::unique_ptr<Task>;
|
||||
using TaskPtrPtr = std::unique_ptr<TaskPtr>;
|
||||
std::future<ZooKeeperImpl::ZooKeeper::GetResponse> asyncGet(const std::string & path);
|
||||
|
||||
/// Everything is complicated.
|
||||
///
|
||||
/// In libzookeeper async interface a function (e.g. zoo_aget)
|
||||
/// accepts a pointer to a standalone callback function and void* pointer to the context
|
||||
/// which is then passed to the callback.
|
||||
/// The caller is responsible for ensuring that the context lives until the callback
|
||||
/// is finished and we can't simply pass ownership of the context into function object.
|
||||
/// Instead, we save the context in a Future object and return it to the caller.
|
||||
/// The context will live until the Future lives.
|
||||
/// Context data is wrapped in an unique_ptr so that its address (which is passed to
|
||||
/// libzookeeper) remains unchanged after the Future is returned from the function.
|
||||
///
|
||||
/// The second problem is that after std::promise has been fulfilled, and the user
|
||||
/// has gotten the result from std::future, the Future object can be destroyed
|
||||
/// before the std::promise::set_value() call that fulfils the promise completes in another
|
||||
/// thread.
|
||||
/// See http://stackoverflow.com/questions/10843304/race-condition-in-pthread-once
|
||||
/// To combat this we use the second unique_ptr. Inside the callback, the void* context
|
||||
/// is cast to unique_ptr and moved into the local unique_ptr to prolong the lifetime of
|
||||
/// the context data.
|
||||
std::future<ZooKeeperImpl::ZooKeeper::GetResponse> asyncTryGet(const std::string & path);
|
||||
|
||||
TaskPtrPtr task;
|
||||
std::future<Result> future;
|
||||
std::future<ZooKeeperImpl::ZooKeeper::ExistsResponse> asyncExists(const std::string & path);
|
||||
|
||||
template <typename... Args>
|
||||
Future(Args &&... args) :
|
||||
task(std::make_unique<TaskPtr>(std::make_unique<Task>(std::forward<Args>(args)...))),
|
||||
future((*task)->get_future()) {}
|
||||
std::future<ZooKeeperImpl::ZooKeeper::ListResponse> asyncGetChildren(const std::string & path);
|
||||
|
||||
public:
|
||||
Result get()
|
||||
{
|
||||
return future.get();
|
||||
}
|
||||
|
||||
Future(Future &&) = default;
|
||||
Future & operator= (Future &&) = default;
|
||||
|
||||
~Future()
|
||||
{
|
||||
/// If nobody has waited for the result, we must wait for it before the object is
|
||||
/// destroyed, because the object contents can still be used in the callback.
|
||||
if (future.valid())
|
||||
future.wait();
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
struct ValueAndStat
|
||||
{
|
||||
std::string value;
|
||||
Stat stat;
|
||||
};
|
||||
|
||||
using GetFuture = Future<ValueAndStat, int, const char *, int, const Stat *>;
|
||||
GetFuture asyncGet(const std::string & path);
|
||||
|
||||
|
||||
struct ValueAndStatAndExists
|
||||
{
|
||||
std::string value;
|
||||
Stat stat;
|
||||
bool exists;
|
||||
};
|
||||
|
||||
using TryGetFuture = Future<ValueAndStatAndExists, int, const char *, int, const Stat *>;
|
||||
TryGetFuture asyncTryGet(const std::string & path);
|
||||
|
||||
|
||||
struct StatAndExists
|
||||
{
|
||||
Stat stat;
|
||||
bool exists;
|
||||
};
|
||||
|
||||
using ExistsFuture = Future<StatAndExists, int, const Stat *>;
|
||||
ExistsFuture asyncExists(const std::string & path);
|
||||
|
||||
|
||||
using GetChildrenFuture = Future<Strings, int, const String_vector *>;
|
||||
GetChildrenFuture asyncGetChildren(const std::string & path);
|
||||
|
||||
|
||||
using RemoveFuture = Future<void, int>;
|
||||
RemoveFuture asyncRemove(const std::string & path, int32_t version = -1);
|
||||
std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> asyncRemove(const std::string & path, int32_t version = -1);
|
||||
|
||||
/// Doesn't throw in the following cases:
|
||||
/// * The node doesn't exist
|
||||
/// * The versions do not match
|
||||
/// * The node has children
|
||||
using TryRemoveFuture = Future<int32_t, int>;
|
||||
TryRemoveFuture asyncTryRemove(const std::string & path, int32_t version = -1);
|
||||
std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> asyncTryRemove(const std::string & path, int32_t version = -1);
|
||||
|
||||
struct OpResultsAndCode
|
||||
{
|
||||
OpResultsPtr results;
|
||||
std::shared_ptr<Ops> ops_ptr;
|
||||
int code;
|
||||
};
|
||||
std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> asyncMulti(const Requests & ops);
|
||||
|
||||
using MultiFuture = Future<OpResultsAndCode, int>;
|
||||
MultiFuture asyncMulti(const Ops & ops);
|
||||
/// Like the previous one but don't throw any exceptions on future.get()
|
||||
MultiFuture tryAsyncMulti(const Ops & ops);
|
||||
|
||||
std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> tryAsyncMulti(const Requests & ops);
|
||||
|
||||
static std::string error2string(int32_t code);
|
||||
|
||||
/// Max size of node contents in bytes.
|
||||
/// In 3.4.5 max node size is 1Mb.
|
||||
static const size_t MAX_NODE_SIZE = 1048576;
|
||||
|
||||
/// Length of the suffix that ZooKeeper adds to sequential nodes.
|
||||
/// In fact it is smaller, but round it up for convenience.
|
||||
static const size_t SEQUENTIAL_SUFFIX_SIZE = 64;
|
||||
|
||||
|
||||
zhandle_t * getHandle() { return impl; }
|
||||
|
||||
private:
|
||||
friend struct WatchContext;
|
||||
friend class EphemeralNodeHolder;
|
||||
friend struct OpResult;
|
||||
|
||||
void init(const std::string & hosts, const std::string & identity, int32_t session_timeout_ms, const std::string & chroot);
|
||||
void init(const std::string & hosts_, const std::string & identity_, int32_t session_timeout_ms_, const std::string & chroot_);
|
||||
|
||||
void removeChildrenRecursive(const std::string & path);
|
||||
void tryRemoveChildrenRecursive(const std::string & path);
|
||||
|
||||
static WatchCallback callbackForEvent(const EventPtr & event);
|
||||
WatchContext * createContext(WatchCallback && callback);
|
||||
static void destroyContext(WatchContext * context);
|
||||
static void processCallback(zhandle_t * zh, int type, int state, const char * path, void * watcher_ctx);
|
||||
|
||||
template <typename T>
|
||||
int32_t retry(T && operation, size_t * attempt = nullptr)
|
||||
{
|
||||
int32_t code = operation();
|
||||
if (attempt)
|
||||
*attempt = 0;
|
||||
for (size_t i = 0; (i < retry_num) && (code == ZOPERATIONTIMEOUT || code == ZCONNECTIONLOSS); ++i)
|
||||
{
|
||||
if (attempt)
|
||||
*attempt = i;
|
||||
|
||||
/// If the connection has been lost, wait timeout/3 hoping for connection re-establishment.
|
||||
static const int MAX_SLEEP_TIME = 10;
|
||||
if (code == ZCONNECTIONLOSS)
|
||||
usleep(std::min(session_timeout_ms * 1000u / 3, MAX_SLEEP_TIME * 1000u * 1000u));
|
||||
|
||||
LOG_WARNING(log, "Error on attempt " << i << ": " << error2string(code) << ". Retry");
|
||||
code = operation();
|
||||
}
|
||||
|
||||
return code;
|
||||
}
|
||||
|
||||
/// The following methods don't throw exceptions but return error codes.
|
||||
int32_t createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created);
|
||||
int32_t removeImpl(const std::string & path, int32_t version = -1);
|
||||
int32_t removeImpl(const std::string & path, int32_t version);
|
||||
int32_t getImpl(const std::string & path, std::string & res, Stat * stat, WatchCallback watch_callback);
|
||||
int32_t setImpl(const std::string & path, const std::string & data, int32_t version = -1, Stat * stat = nullptr);
|
||||
int32_t setImpl(const std::string & path, const std::string & data, int32_t version, Stat * stat);
|
||||
int32_t getChildrenImpl(const std::string & path, Strings & res, Stat * stat, WatchCallback watch_callback);
|
||||
int32_t multiImpl(const Ops & ops, OpResultsPtr * out_op_results = nullptr, MultiTransactionInfo * out_info = nullptr);
|
||||
int32_t multiImpl(const Requests & requests, Responses & responses);
|
||||
int32_t existsImpl(const std::string & path, Stat * stat_, WatchCallback watch_callback);
|
||||
|
||||
MultiFuture asyncMultiImpl(const zkutil::Ops & ops_, bool throw_exception);
|
||||
std::unique_ptr<ZooKeeperImpl::ZooKeeper> impl;
|
||||
|
||||
std::string hosts;
|
||||
std::string identity;
|
||||
@ -420,19 +233,8 @@ private:
|
||||
std::string chroot;
|
||||
|
||||
std::mutex mutex;
|
||||
ACLPtr default_acl;
|
||||
zhandle_t * impl;
|
||||
|
||||
std::unordered_set<WatchContext *> watch_context_store;
|
||||
|
||||
/// Retries number in case of OperationTimeout or ConnectionLoss errors.
|
||||
static constexpr size_t retry_num = 3;
|
||||
Logger * log = nullptr;
|
||||
|
||||
/// If true, there were unsuccessfull attempts to remove ephemeral nodes.
|
||||
/// It is better to close the session to remove ephemeral nodes with certainty
|
||||
/// instead of continuing to use re-established session.
|
||||
bool is_dirty = false;
|
||||
};
|
||||
|
||||
|
||||
@ -476,11 +278,7 @@ public:
|
||||
{
|
||||
try
|
||||
{
|
||||
/// Important: if the ZooKeeper is temporarily unavailable, repeated attempts to
|
||||
/// delete the node are made.
|
||||
/// Otherwise it is possible that EphemeralNodeHolder is destroyed,
|
||||
/// but the session has recovered and the node in ZooKeeper remains for the long time.
|
||||
zookeeper.tryRemoveEphemeralNodeWithRetries(path);
|
||||
zookeeper.tryRemove(path);
|
||||
}
|
||||
catch (const KeeperException & e)
|
||||
{
|
||||
|
1325
dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp
Normal file
1325
dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp
Normal file
File diff suppressed because it is too large
Load Diff
548
dbms/src/Common/ZooKeeper/ZooKeeperImpl.h
Normal file
548
dbms/src/Common/ZooKeeper/ZooKeeperImpl.h
Normal file
@ -0,0 +1,548 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/ReadBufferFromPocoSocket.h>
|
||||
#include <IO/WriteBufferFromPocoSocket.h>
|
||||
|
||||
#include <Poco/Net/StreamSocket.h>
|
||||
#include <Poco/Net/SocketAddress.h>
|
||||
|
||||
#include <map>
|
||||
#include <mutex>
|
||||
#include <chrono>
|
||||
#include <vector>
|
||||
#include <memory>
|
||||
#include <thread>
|
||||
#include <atomic>
|
||||
#include <cstdint>
|
||||
#include <optional>
|
||||
#include <functional>
|
||||
|
||||
|
||||
namespace ZooKeeperImpl
|
||||
{
|
||||
|
||||
using namespace DB;
|
||||
|
||||
|
||||
/** Usage scenario:
|
||||
* - create an object and issue commands;
|
||||
* - you provide callbacks for your commands; callbacks are invoked in internal thread and must be cheap:
|
||||
* for example, just signal a condvar / fulfull a promise.
|
||||
* - you also may provide callbacks for watches; they are also invoked in internal thread and must be cheap.
|
||||
* - whenever you receive SessionExpired exception of method isValid returns false,
|
||||
* the ZooKeeper instance is no longer usable - you may only destroy it and probably create another.
|
||||
* - whenever session is expired or ZooKeeper instance is destroying, all callbacks are notified with special event.
|
||||
* - data for callbacks must be alive when ZooKeeper instance is alive.
|
||||
*/
|
||||
class ZooKeeper
|
||||
{
|
||||
public:
|
||||
using Addresses = std::vector<Poco::Net::SocketAddress>;
|
||||
|
||||
struct ACL
|
||||
{
|
||||
static constexpr int32_t Read = 1;
|
||||
static constexpr int32_t Write = 2;
|
||||
static constexpr int32_t Create = 4;
|
||||
static constexpr int32_t Delete = 8;
|
||||
static constexpr int32_t Admin = 16;
|
||||
static constexpr int32_t All = 0x1F;
|
||||
|
||||
int32_t permissions;
|
||||
String scheme;
|
||||
String id;
|
||||
|
||||
void write(WriteBuffer & out) const;
|
||||
};
|
||||
using ACLs = std::vector<ACL>;
|
||||
|
||||
struct Stat
|
||||
{
|
||||
int64_t czxid;
|
||||
int64_t mzxid;
|
||||
int64_t ctime;
|
||||
int64_t mtime;
|
||||
int32_t version;
|
||||
int32_t cversion;
|
||||
int32_t aversion;
|
||||
int64_t ephemeralOwner;
|
||||
int32_t dataLength;
|
||||
int32_t numChildren;
|
||||
int64_t pzxid;
|
||||
|
||||
void read(ReadBuffer & in);
|
||||
};
|
||||
|
||||
using XID = int32_t;
|
||||
using OpNum = int32_t;
|
||||
|
||||
|
||||
struct Response
|
||||
{
|
||||
int32_t error = 0;
|
||||
virtual ~Response() {}
|
||||
virtual void readImpl(ReadBuffer &) = 0;
|
||||
|
||||
virtual void removeRootPath(const String & /* root_path */) {}
|
||||
};
|
||||
|
||||
using ResponsePtr = std::shared_ptr<Response>;
|
||||
using Responses = std::vector<ResponsePtr>;
|
||||
using ResponseCallback = std::function<void(const Response &)>;
|
||||
|
||||
struct Request
|
||||
{
|
||||
XID xid = 0;
|
||||
bool has_watch = false;
|
||||
|
||||
virtual ~Request() {};
|
||||
virtual OpNum getOpNum() const = 0;
|
||||
|
||||
/// Writes length, xid, op_num, then the rest.
|
||||
void write(WriteBuffer & out) const;
|
||||
virtual void writeImpl(WriteBuffer &) const = 0;
|
||||
|
||||
virtual ResponsePtr makeResponse() const = 0;
|
||||
|
||||
virtual void addRootPath(const String & /* root_path */) {};
|
||||
virtual String getPath() const = 0;
|
||||
};
|
||||
|
||||
using RequestPtr = std::shared_ptr<Request>;
|
||||
using Requests = std::vector<RequestPtr>;
|
||||
|
||||
struct HeartbeatRequest final : Request
|
||||
{
|
||||
OpNum getOpNum() const override { return 11; }
|
||||
void writeImpl(WriteBuffer &) const override {}
|
||||
ResponsePtr makeResponse() const override;
|
||||
String getPath() const override { return {}; }
|
||||
};
|
||||
|
||||
struct HeartbeatResponse final : Response
|
||||
{
|
||||
void readImpl(ReadBuffer &) override {}
|
||||
};
|
||||
|
||||
struct WatchResponse final : Response
|
||||
{
|
||||
int32_t type = 0;
|
||||
int32_t state = 0;
|
||||
String path;
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
void removeRootPath(const String & root_path) override;
|
||||
};
|
||||
|
||||
using WatchCallback = std::function<void(const WatchResponse &)>;
|
||||
|
||||
struct AuthRequest final : Request
|
||||
{
|
||||
int32_t type = 0; /// ignored by the server
|
||||
String scheme;
|
||||
String data;
|
||||
|
||||
OpNum getOpNum() const override { return 100; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
String getPath() const override { return {}; }
|
||||
};
|
||||
|
||||
struct AuthResponse final : Response
|
||||
{
|
||||
void readImpl(ReadBuffer &) override {};
|
||||
};
|
||||
|
||||
struct CloseRequest final : Request
|
||||
{
|
||||
OpNum getOpNum() const override { return -11; }
|
||||
void writeImpl(WriteBuffer &) const override {}
|
||||
ResponsePtr makeResponse() const override;
|
||||
String getPath() const override { return {}; }
|
||||
};
|
||||
|
||||
struct CloseResponse final : Response
|
||||
{
|
||||
void readImpl(ReadBuffer &) override;
|
||||
};
|
||||
|
||||
struct CreateRequest final : Request
|
||||
{
|
||||
String path;
|
||||
String data;
|
||||
bool is_ephemeral = false;
|
||||
bool is_sequential = false;
|
||||
ACLs acls;
|
||||
|
||||
OpNum getOpNum() const override { return 1; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct CreateResponse final : Response
|
||||
{
|
||||
String path_created;
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
void removeRootPath(const String & root_path) override;
|
||||
};
|
||||
|
||||
struct RemoveRequest final : Request
|
||||
{
|
||||
String path;
|
||||
int32_t version = -1;
|
||||
|
||||
OpNum getOpNum() const override { return 2; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct RemoveResponse final : Response
|
||||
{
|
||||
void readImpl(ReadBuffer &) override {}
|
||||
};
|
||||
|
||||
struct ExistsRequest final : Request
|
||||
{
|
||||
String path;
|
||||
|
||||
OpNum getOpNum() const override { return 3; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct ExistsResponse final : Response
|
||||
{
|
||||
Stat stat;
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
};
|
||||
|
||||
struct GetRequest final : Request
|
||||
{
|
||||
String path;
|
||||
|
||||
OpNum getOpNum() const override { return 4; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct GetResponse final : Response
|
||||
{
|
||||
String data;
|
||||
Stat stat;
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
};
|
||||
|
||||
struct SetRequest final : Request
|
||||
{
|
||||
String path;
|
||||
String data;
|
||||
int32_t version = -1;
|
||||
|
||||
OpNum getOpNum() const override { return 5; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct SetResponse final : Response
|
||||
{
|
||||
Stat stat;
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
};
|
||||
|
||||
struct ListRequest final : Request
|
||||
{
|
||||
String path;
|
||||
|
||||
OpNum getOpNum() const override { return 12; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct ListResponse final : Response
|
||||
{
|
||||
std::vector<String> names;
|
||||
Stat stat;
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
};
|
||||
|
||||
struct CheckRequest final : Request
|
||||
{
|
||||
String path;
|
||||
int32_t version = -1;
|
||||
|
||||
OpNum getOpNum() const override { return 13; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return path; }
|
||||
};
|
||||
|
||||
struct CheckResponse final : Response
|
||||
{
|
||||
void readImpl(ReadBuffer &) override {};
|
||||
};
|
||||
|
||||
struct MultiRequest final : Request
|
||||
{
|
||||
Requests requests;
|
||||
|
||||
OpNum getOpNum() const override { return 14; }
|
||||
void writeImpl(WriteBuffer &) const override;
|
||||
ResponsePtr makeResponse() const override;
|
||||
void addRootPath(const String & root_path) override;
|
||||
String getPath() const override { return {}; }
|
||||
};
|
||||
|
||||
struct MultiResponse final : Response
|
||||
{
|
||||
Responses responses;
|
||||
|
||||
MultiResponse(const Requests & requests);
|
||||
|
||||
void readImpl(ReadBuffer &) override;
|
||||
void removeRootPath(const String & root_path) override;
|
||||
};
|
||||
|
||||
/// This response may be received only as an element of responses in MultiResponse.
|
||||
struct ErrorResponse final : Response
|
||||
{
|
||||
void readImpl(ReadBuffer &) override;
|
||||
};
|
||||
|
||||
|
||||
/** Connection to addresses is performed in order. If you want, shuffle them manually.
|
||||
* Operation timeout couldn't be greater than session timeout.
|
||||
* Operation timeout applies independently for network read, network write, waiting for events and synchronization.
|
||||
*/
|
||||
ZooKeeper(
|
||||
const Addresses & addresses,
|
||||
const String & root_path,
|
||||
const String & auth_scheme,
|
||||
const String & auth_data,
|
||||
Poco::Timespan session_timeout,
|
||||
Poco::Timespan connection_timeout,
|
||||
Poco::Timespan operation_timeout);
|
||||
|
||||
~ZooKeeper();
|
||||
|
||||
|
||||
/// If expired, you can only destroy the object. All other methods will throw exception.
|
||||
bool isExpired() const { return expired; }
|
||||
|
||||
/// Useful to check owner of ephemeral node.
|
||||
int64_t getSessionID() const { return session_id; }
|
||||
|
||||
|
||||
using CreateCallback = std::function<void(const CreateResponse &)>;
|
||||
using RemoveCallback = std::function<void(const RemoveResponse &)>;
|
||||
using ExistsCallback = std::function<void(const ExistsResponse &)>;
|
||||
using GetCallback = std::function<void(const GetResponse &)>;
|
||||
using SetCallback = std::function<void(const SetResponse &)>;
|
||||
using ListCallback = std::function<void(const ListResponse &)>;
|
||||
using CheckCallback = std::function<void(const CheckResponse &)>;
|
||||
using MultiCallback = std::function<void(const MultiResponse &)>;
|
||||
|
||||
|
||||
void create(
|
||||
const String & path,
|
||||
const String & data,
|
||||
bool is_ephemeral,
|
||||
bool is_sequential,
|
||||
const ACLs & acls,
|
||||
CreateCallback callback);
|
||||
|
||||
void remove(
|
||||
const String & path,
|
||||
int32_t version,
|
||||
RemoveCallback callback);
|
||||
|
||||
void exists(
|
||||
const String & path,
|
||||
ExistsCallback callback,
|
||||
WatchCallback watch);
|
||||
|
||||
void get(
|
||||
const String & path,
|
||||
GetCallback callback,
|
||||
WatchCallback watch);
|
||||
|
||||
void set(
|
||||
const String & path,
|
||||
const String & data,
|
||||
int32_t version,
|
||||
SetCallback callback);
|
||||
|
||||
void list(
|
||||
const String & path,
|
||||
ListCallback callback,
|
||||
WatchCallback watch);
|
||||
|
||||
void check(
|
||||
const String & path,
|
||||
int32_t version,
|
||||
CheckCallback callback);
|
||||
|
||||
void multi(
|
||||
const Requests & requests,
|
||||
MultiCallback callback);
|
||||
|
||||
|
||||
enum Error
|
||||
{
|
||||
ZOK = 0,
|
||||
|
||||
/** System and server-side errors.
|
||||
* This is never thrown by the server, it shouldn't be used other than
|
||||
* to indicate a range. Specifically error codes greater than this
|
||||
* value, but lesser than ZAPIERROR, are system errors.
|
||||
*/
|
||||
ZSYSTEMERROR = -1,
|
||||
|
||||
ZRUNTIMEINCONSISTENCY = -2, /// A runtime inconsistency was found
|
||||
ZDATAINCONSISTENCY = -3, /// A data inconsistency was found
|
||||
ZCONNECTIONLOSS = -4, /// Connection to the server has been lost
|
||||
ZMARSHALLINGERROR = -5, /// Error while marshalling or unmarshalling data
|
||||
ZUNIMPLEMENTED = -6, /// Operation is unimplemented
|
||||
ZOPERATIONTIMEOUT = -7, /// Operation timeout
|
||||
ZBADARGUMENTS = -8, /// Invalid arguments
|
||||
ZINVALIDSTATE = -9, /// Invliad zhandle state
|
||||
|
||||
/** API errors.
|
||||
* This is never thrown by the server, it shouldn't be used other than
|
||||
* to indicate a range. Specifically error codes greater than this
|
||||
* value are API errors.
|
||||
*/
|
||||
ZAPIERROR = -100,
|
||||
|
||||
ZNONODE = -101, /// Node does not exist
|
||||
ZNOAUTH = -102, /// Not authenticated
|
||||
ZBADVERSION = -103, /// Version conflict
|
||||
ZNOCHILDRENFOREPHEMERALS = -108, /// Ephemeral nodes may not have children
|
||||
ZNODEEXISTS = -110, /// The node already exists
|
||||
ZNOTEMPTY = -111, /// The node has children
|
||||
ZSESSIONEXPIRED = -112, /// The session has been expired by the server
|
||||
ZINVALIDCALLBACK = -113, /// Invalid callback specified
|
||||
ZINVALIDACL = -114, /// Invalid ACL specified
|
||||
ZAUTHFAILED = -115, /// Client authentication failed
|
||||
ZCLOSING = -116, /// ZooKeeper is closing
|
||||
ZNOTHING = -117, /// (not error) no server responses to process
|
||||
ZSESSIONMOVED = -118 /// Session moved to another server, so operation is ignored
|
||||
};
|
||||
|
||||
static const char * errorMessage(int32_t code);
|
||||
|
||||
/// For watches.
|
||||
enum State
|
||||
{
|
||||
EXPIRED_SESSION = -112,
|
||||
AUTH_FAILED = -113,
|
||||
CONNECTING = 1,
|
||||
ASSOCIATING = 2,
|
||||
CONNECTED = 3,
|
||||
NOTCONNECTED = 999
|
||||
};
|
||||
|
||||
enum Event
|
||||
{
|
||||
CREATED = 1,
|
||||
DELETED = 2,
|
||||
CHANGED = 3,
|
||||
CHILD = 4,
|
||||
SESSION = -1,
|
||||
NOTWATCHING = -2
|
||||
};
|
||||
|
||||
private:
|
||||
String root_path;
|
||||
ACLs default_acls;
|
||||
|
||||
Poco::Timespan session_timeout;
|
||||
Poco::Timespan operation_timeout;
|
||||
|
||||
Poco::Net::StreamSocket socket;
|
||||
std::optional<ReadBufferFromPocoSocket> in;
|
||||
std::optional<WriteBufferFromPocoSocket> out;
|
||||
|
||||
int64_t session_id = 0;
|
||||
std::atomic<XID> xid {1};
|
||||
|
||||
using clock = std::chrono::steady_clock;
|
||||
|
||||
struct RequestInfo
|
||||
{
|
||||
RequestPtr request;
|
||||
ResponseCallback callback;
|
||||
WatchCallback watch;
|
||||
clock::time_point time;
|
||||
};
|
||||
|
||||
using RequestsQueue = ConcurrentBoundedQueue<RequestPtr>;
|
||||
|
||||
RequestsQueue requests{1};
|
||||
void pushRequest(RequestInfo && request);
|
||||
|
||||
using Operations = std::map<XID, RequestInfo>;
|
||||
|
||||
Operations operations;
|
||||
std::mutex operations_mutex;
|
||||
|
||||
using WatchCallbacks = std::vector<WatchCallback>;
|
||||
using Watches = std::map<String /* path */, WatchCallbacks>;
|
||||
|
||||
Watches watches;
|
||||
std::mutex watches_mutex;
|
||||
|
||||
std::thread send_thread;
|
||||
std::thread receive_thread;
|
||||
|
||||
std::atomic<bool> expired {false};
|
||||
|
||||
void connect(
|
||||
const Addresses & addresses,
|
||||
Poco::Timespan connection_timeout);
|
||||
|
||||
void sendHandshake();
|
||||
void receiveHandshake();
|
||||
|
||||
void sendAuth(const String & scheme, const String & data);
|
||||
|
||||
void receiveEvent();
|
||||
|
||||
void sendThread();
|
||||
void receiveThread();
|
||||
|
||||
void close();
|
||||
|
||||
/// Call all remaining callbacks and watches, passing errors to them.
|
||||
void finalize(bool error_send, bool error_receive);
|
||||
|
||||
template <typename T>
|
||||
void write(const T &);
|
||||
|
||||
template <typename T>
|
||||
void read(T &);
|
||||
};
|
||||
|
||||
};
|
@ -41,20 +41,18 @@ std::optional<std::string> ZooKeeperNodeCache::get(const std::string & path)
|
||||
if (nonexistent_nodes.count(path))
|
||||
return std::nullopt;
|
||||
|
||||
auto watch_callback = [context=context](zkutil::ZooKeeper & zookeeper, int type, int state, const char * path)
|
||||
auto watch_callback = [context=context](const ZooKeeperImpl::ZooKeeper::WatchResponse & response)
|
||||
{
|
||||
if (!(type != ZOO_SESSION_EVENT || state == ZOO_EXPIRED_SESSION_STATE))
|
||||
if (!(response.type != ZooKeeperImpl::ZooKeeper::SESSION || response.state == ZooKeeperImpl::ZooKeeper::EXPIRED_SESSION))
|
||||
return;
|
||||
|
||||
bool changed = false;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(context->mutex);
|
||||
if (&zookeeper != context->zookeeper.get())
|
||||
return;
|
||||
|
||||
if (type != ZOO_SESSION_EVENT)
|
||||
changed = context->invalidated_paths.emplace(path).second;
|
||||
else if (state == ZOO_EXPIRED_SESSION_STATE)
|
||||
if (response.type != ZooKeeperImpl::ZooKeeper::SESSION)
|
||||
changed = context->invalidated_paths.emplace(response.path).second;
|
||||
else if (response.state == ZooKeeperImpl::ZooKeeper::EXPIRED_SESSION)
|
||||
{
|
||||
context->zookeeper = nullptr;
|
||||
context->invalidated_paths.clear();
|
||||
|
@ -1,6 +1,9 @@
|
||||
add_executable(zkutil_test_commands zkutil_test_commands.cpp)
|
||||
target_link_libraries(zkutil_test_commands clickhouse_common_zookeeper)
|
||||
|
||||
add_executable(zkutil_test_commands_new_lib zkutil_test_commands_new_lib.cpp)
|
||||
target_link_libraries(zkutil_test_commands_new_lib clickhouse_common_zookeeper)
|
||||
|
||||
add_executable(zkutil_test_lock zkutil_test_lock.cpp)
|
||||
target_link_libraries(zkutil_test_lock clickhouse_common_zookeeper)
|
||||
|
||||
@ -15,3 +18,6 @@ target_link_libraries(zkutil_zookeeper_holder clickhouse_common_zookeeper)
|
||||
|
||||
add_executable (zk_many_watches_reconnect zk_many_watches_reconnect.cpp)
|
||||
target_link_libraries (zk_many_watches_reconnect clickhouse_common_zookeeper clickhouse_common_config)
|
||||
|
||||
add_executable (zookeeper_impl zookeeper_impl.cpp)
|
||||
target_link_libraries (zookeeper_impl clickhouse_common_zookeeper)
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <iostream>
|
||||
@ -31,25 +32,24 @@ TEST(zkutil, multi_nice_exception_msg)
|
||||
{
|
||||
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181");
|
||||
|
||||
auto acl = zookeeper->getDefaultACL();
|
||||
zkutil::Ops ops;
|
||||
zkutil::Requests ops;
|
||||
|
||||
ASSERT_NO_THROW(
|
||||
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
|
||||
|
||||
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "_", acl, zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent));
|
||||
zookeeper->multi(ops);
|
||||
);
|
||||
|
||||
try
|
||||
{
|
||||
ops.clear();
|
||||
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/c", "_", acl, zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(new zkutil::Op::Remove("/clickhouse_test/zkutil_multi/c", -1));
|
||||
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "BadBoy", acl, zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/b", "_", acl, zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/c", "_", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest("/clickhouse_test/zkutil_multi/c", -1));
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "BadBoy", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/b", "_", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent));
|
||||
|
||||
zookeeper->multi(ops);
|
||||
FAIL();
|
||||
@ -69,8 +69,7 @@ TEST(zkutil, multi_nice_exception_msg)
|
||||
TEST(zkutil, multi_async)
|
||||
{
|
||||
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181");
|
||||
auto acl = zookeeper->getDefaultACL();
|
||||
zkutil::Ops ops;
|
||||
zkutil::Requests ops;
|
||||
|
||||
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
|
||||
|
||||
@ -81,30 +80,29 @@ TEST(zkutil, multi_async)
|
||||
|
||||
{
|
||||
ops.clear();
|
||||
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "", acl, zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "", acl, zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "", zkutil::CreateMode::Persistent));
|
||||
|
||||
auto fut = zookeeper->tryAsyncMulti(ops);
|
||||
ops.clear();
|
||||
|
||||
auto res = fut.get();
|
||||
ASSERT_TRUE(res.code == ZOK);
|
||||
ASSERT_EQ(res.results->size(), 2);
|
||||
ASSERT_EQ(res.ops_ptr->size(), 2);
|
||||
ASSERT_TRUE(res.error == ZooKeeperImpl::ZooKeeper::ZOK);
|
||||
ASSERT_EQ(res.responses.size(), 2);
|
||||
}
|
||||
|
||||
EXPECT_ANY_THROW
|
||||
(
|
||||
std::vector<zkutil::ZooKeeper::MultiFuture> futures;
|
||||
std::vector<std::future<ZooKeeperImpl::ZooKeeper::MultiResponse>> futures;
|
||||
|
||||
for (size_t i = 0; i < 10000; ++i)
|
||||
{
|
||||
ops.clear();
|
||||
ops.emplace_back(new zkutil::Op::Remove("/clickhouse_test/zkutil_multi", -1));
|
||||
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "_", acl, zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(new zkutil::Op::Check("/clickhouse_test/zkutil_multi", -1));
|
||||
ops.emplace_back(new zkutil::Op::SetData("/clickhouse_test/zkutil_multi", "xxx", 42));
|
||||
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest("/clickhouse_test/zkutil_multi", -1));
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCheckRequest("/clickhouse_test/zkutil_multi", -1));
|
||||
ops.emplace_back(zkutil::makeSetRequest("/clickhouse_test/zkutil_multi", "xxx", 42));
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent));
|
||||
|
||||
futures.emplace_back(zookeeper->asyncMulti(ops));
|
||||
}
|
||||
@ -118,16 +116,15 @@ TEST(zkutil, multi_async)
|
||||
|
||||
{
|
||||
ops.clear();
|
||||
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "_", acl, zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent));
|
||||
|
||||
auto fut = zookeeper->tryAsyncMulti(ops);
|
||||
ops.clear();
|
||||
|
||||
auto res = fut.get();
|
||||
ASSERT_TRUE(res.code == ZNODEEXISTS);
|
||||
ASSERT_EQ(res.results->size(), 2);
|
||||
ASSERT_EQ(res.ops_ptr->size(), 2);
|
||||
ASSERT_TRUE(res.error == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS);
|
||||
ASSERT_EQ(res.responses.size(), 2);
|
||||
}
|
||||
}
|
||||
|
||||
@ -135,9 +132,9 @@ TEST(zkutil, multi_async)
|
||||
TEST(zkutil, multi_async_libzookeeper_segfault)
|
||||
{
|
||||
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181", "", 1000);
|
||||
zkutil::Ops ops;
|
||||
zkutil::Requests ops;
|
||||
|
||||
ops.emplace_back(new zkutil::Op::Check("/clickhouse_test/zkutil_multi", 0));
|
||||
ops.emplace_back(zkutil::makeCheckRequest("/clickhouse_test/zkutil_multi", 0));
|
||||
|
||||
/// Uncomment to test
|
||||
//auto cmd = ShellCommand::execute("sudo service zookeeper restart");
|
||||
@ -146,7 +143,7 @@ TEST(zkutil, multi_async_libzookeeper_segfault)
|
||||
auto future = zookeeper->asyncMulti(ops);
|
||||
auto res = future.get();
|
||||
|
||||
EXPECT_TRUE(zkutil::isUnrecoverableErrorCode(res.code));
|
||||
EXPECT_TRUE(zkutil::isHardwareError(res.error));
|
||||
}
|
||||
|
||||
|
||||
@ -159,21 +156,20 @@ TEST(zkutil, multi_create_sequential)
|
||||
zookeeper->createAncestors("/clickhouse_test/");
|
||||
|
||||
zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181", "", zkutil::DEFAULT_SESSION_TIMEOUT, "/clickhouse_test");
|
||||
auto acl = zookeeper->getDefaultACL();
|
||||
zkutil::Ops ops;
|
||||
zkutil::Requests ops;
|
||||
|
||||
String base_path = "/zkutil/multi_create_sequential";
|
||||
zookeeper->tryRemoveRecursive(base_path);
|
||||
zookeeper->createAncestors(base_path + "/");
|
||||
|
||||
String sequential_node_prefix = base_path + "/queue-";
|
||||
ops.emplace_back(new zkutil::Op::Create(sequential_node_prefix, "", acl, zkutil::CreateMode::EphemeralSequential));
|
||||
zkutil::OpResultsPtr results = zookeeper->multi(ops);
|
||||
zkutil::OpResult & sequential_node_result_op = results->at(0);
|
||||
ops.emplace_back(zkutil::makeCreateRequest(sequential_node_prefix, "", zkutil::CreateMode::EphemeralSequential));
|
||||
auto results = zookeeper->multi(ops);
|
||||
const auto & sequential_node_result_op = typeid_cast<const zkutil::CreateResponse &>(*results.at(0));
|
||||
|
||||
EXPECT_FALSE(sequential_node_result_op.value.empty());
|
||||
EXPECT_GT(sequential_node_result_op.value.length(), sequential_node_prefix.length());
|
||||
EXPECT_EQ(sequential_node_result_op.value.substr(0, sequential_node_prefix.length()), sequential_node_prefix);
|
||||
EXPECT_FALSE(sequential_node_result_op.path_created.empty());
|
||||
EXPECT_GT(sequential_node_result_op.path_created.length(), sequential_node_prefix.length());
|
||||
EXPECT_EQ(sequential_node_result_op.path_created.substr(0, sequential_node_prefix.length()), sequential_node_prefix);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -33,18 +33,18 @@ int main(int argc, char ** argv)
|
||||
while (true)
|
||||
{
|
||||
{
|
||||
zkutil::Ops ops;
|
||||
ops.emplace_back(std::make_shared<zkutil::Op::Create>("/test/zk_expiration_test", "hello", zk.getDefaultACL(), zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(std::make_shared<zkutil::Op::Remove>("/test/zk_expiration_test", -1));
|
||||
zkutil::Requests ops;
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/test/zk_expiration_test", "hello", zkutil::CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest("/test/zk_expiration_test", -1));
|
||||
|
||||
zkutil::MultiTransactionInfo info;
|
||||
zk.tryMultiNoThrow(ops, nullptr, &info);
|
||||
zkutil::Responses responses;
|
||||
int32_t code = zk.tryMultiNoThrow(ops, responses);
|
||||
|
||||
std::cout << time(nullptr) - time0 << "s: " << zkutil::ZooKeeper::error2string(info.code) << std::endl;
|
||||
std::cout << time(nullptr) - time0 << "s: " << zkutil::ZooKeeper::error2string(code) << std::endl;
|
||||
try
|
||||
{
|
||||
if (info.code != ZOK)
|
||||
std::cout << "Path: " << info.getFailedOp().getPath() << std::endl;
|
||||
if (code)
|
||||
std::cout << "Path: " << zkutil::KeeperMultiException(code, ops, responses).getPathForFirstFailedOp() << std::endl;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -23,12 +23,12 @@ try
|
||||
{
|
||||
while (true)
|
||||
{
|
||||
std::vector<zkutil::ZooKeeper::TryGetFuture> futures;
|
||||
std::vector<std::future<zkutil::GetResponse>> futures;
|
||||
for (auto & node : nodes)
|
||||
futures.push_back(zookeeper.asyncTryGet("/tmp/" + node));
|
||||
futures.push_back(zookeeper.asyncGet("/tmp/" + node));
|
||||
|
||||
for (auto & future : futures)
|
||||
std::cerr << (future.get().value.empty() ? ',' : '.');
|
||||
std::cerr << (future.get().data.empty() ? ',' : '.');
|
||||
}
|
||||
});
|
||||
}
|
||||
@ -41,5 +41,5 @@ try
|
||||
catch (const Poco::Exception & e)
|
||||
{
|
||||
std::cout << e.message() << std::endl;
|
||||
throw;
|
||||
return 1;
|
||||
}
|
||||
|
@ -1,41 +1,54 @@
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <iostream>
|
||||
#include <unistd.h>
|
||||
#include <port/unistd.h>
|
||||
|
||||
|
||||
using namespace zkutil;
|
||||
|
||||
int main()
|
||||
int main(int argc, char ** argv)
|
||||
try
|
||||
{
|
||||
try
|
||||
if (argc < 2)
|
||||
{
|
||||
ZooKeeper zk("mtfilter01t:2181,metrika-test:2181,mtweb01t:2181", "", 5000);
|
||||
Strings children;
|
||||
|
||||
std::cout << "create path" << std::endl;
|
||||
zk.create("/test", "old", zkutil::CreateMode::Persistent);
|
||||
zkutil::Stat stat;
|
||||
zkutil::EventPtr watch = std::make_shared<Poco::Event>();
|
||||
|
||||
std::cout << "get path" << std::endl;
|
||||
zk.get("/test", &stat, watch);
|
||||
std::cout << "set path" << std::endl;
|
||||
zk.set("/test", "new");
|
||||
watch->wait();
|
||||
std::cout << "watch happened" << std::endl;
|
||||
std::cout << "remove path" << std::endl;
|
||||
zk.remove("/test");
|
||||
|
||||
Ops ops;
|
||||
ops.emplace_back(std::make_unique<Op::Create>("/test", "multi1", zk.getDefaultACL(), CreateMode::Persistent));
|
||||
ops.emplace_back(std::make_unique<Op::SetData>("/test", "multi2", -1));
|
||||
ops.emplace_back(std::make_unique<Op::Remove>("/test", -1));
|
||||
std::cout << "multi" << std::endl;
|
||||
OpResultsPtr res = zk.multi(ops);
|
||||
std::cout << "path created: " << dynamic_cast<Op::Create &>(*ops[0]).getPathCreated() << std::endl;
|
||||
}
|
||||
catch (KeeperException & e)
|
||||
{
|
||||
std::cerr << "KeeperException " << e.what() << " " << e.message() << std::endl;
|
||||
std::cerr << "Usage: ./zkutil_test_commands host:port,host:port...\n";
|
||||
return 1;
|
||||
}
|
||||
|
||||
ZooKeeper zk(argv[1], "", 5000);
|
||||
|
||||
std::cout << "create path" << std::endl;
|
||||
zk.create("/test", "old", zkutil::CreateMode::Persistent);
|
||||
zkutil::Stat stat;
|
||||
zkutil::EventPtr watch = std::make_shared<Poco::Event>();
|
||||
|
||||
std::cout << "get path" << std::endl;
|
||||
zk.get("/test", &stat, watch);
|
||||
std::cout << "set path" << std::endl;
|
||||
zk.set("/test", "new");
|
||||
watch->wait();
|
||||
std::cout << "watch happened" << std::endl;
|
||||
std::cout << "remove path" << std::endl;
|
||||
|
||||
std::cout << "list path" << std::endl;
|
||||
Strings children = zk.getChildren("/");
|
||||
for (const auto & name : children)
|
||||
std::cerr << "\t" << name << "\n";
|
||||
|
||||
zk.remove("/test");
|
||||
|
||||
zkutil::Requests ops;
|
||||
ops.emplace_back(zkutil::makeCreateRequest("/test", "multi1", CreateMode::Persistent));
|
||||
ops.emplace_back(zkutil::makeSetRequest("/test", "multi2", -1));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest("/test", -1));
|
||||
std::cout << "multi" << std::endl;
|
||||
zkutil::Responses res = zk.multi(ops);
|
||||
std::cout << "path created: " << typeid_cast<const CreateResponse &>(*res[0]).path_created << std::endl;
|
||||
|
||||
return 0;
|
||||
}
|
||||
catch (KeeperException & e)
|
||||
{
|
||||
std::cerr << "KeeperException " << e.what() << " " << e.message() << std::endl;
|
||||
return 1;
|
||||
}
|
||||
|
199
dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp
Normal file
199
dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp
Normal file
@ -0,0 +1,199 @@
|
||||
#include <Poco/ConsoleChannel.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/Event.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperImpl.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <iostream>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
|
||||
|
||||
using namespace ZooKeeperImpl;
|
||||
|
||||
|
||||
int main(int argc, char ** argv)
|
||||
try
|
||||
{
|
||||
if (argc < 2)
|
||||
{
|
||||
std::cerr << "Usage: ./zkutil_test_commands_new_lib host:port,host:port...\n";
|
||||
return 1;
|
||||
}
|
||||
|
||||
Poco::AutoPtr<Poco::ConsoleChannel> channel(new Poco::ConsoleChannel(std::cerr));
|
||||
Poco::Logger::root().setChannel(channel);
|
||||
Poco::Logger::root().setLevel("trace");
|
||||
|
||||
std::string addresses_arg = argv[1];
|
||||
std::vector<std::string> addresses_strings;
|
||||
boost::split(addresses_strings, addresses_arg, boost::is_any_of(","));
|
||||
ZooKeeper::Addresses addresses;
|
||||
addresses.reserve(addresses_strings.size());
|
||||
for (const auto & address_string : addresses_strings)
|
||||
addresses.emplace_back(address_string);
|
||||
|
||||
ZooKeeper zk(addresses, {}, {}, {}, {5, 0}, {0, 50000}, {0, 50000});
|
||||
|
||||
Poco::Event event(true);
|
||||
|
||||
std::cout << "create\n";
|
||||
|
||||
zk.create("/test", "old", false, false, {},
|
||||
[&](const ZooKeeper::CreateResponse & response)
|
||||
{
|
||||
if (response.error)
|
||||
std::cerr << "Error (create) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
|
||||
else
|
||||
std::cerr << "Created path: " << response.path_created << '\n';
|
||||
|
||||
//event.set();
|
||||
});
|
||||
|
||||
//event.wait();
|
||||
|
||||
std::cout << "get\n";
|
||||
|
||||
zk.get("/test",
|
||||
[&](const ZooKeeper::GetResponse & response)
|
||||
{
|
||||
if (response.error)
|
||||
std::cerr << "Error (get) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
|
||||
else
|
||||
std::cerr << "Value: " << response.data << '\n';
|
||||
|
||||
//event.set();
|
||||
},
|
||||
[](const ZooKeeper::WatchResponse & response)
|
||||
{
|
||||
if (response.error)
|
||||
std::cerr << "Watch (get) on /test, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
|
||||
else
|
||||
std::cerr << "Watch (get) on /test, path: " << response.path << ", type: " << response.type << '\n';
|
||||
});
|
||||
|
||||
//event.wait();
|
||||
|
||||
std::cout << "set\n";
|
||||
|
||||
zk.set("/test", "new", -1,
|
||||
[&](const ZooKeeper::SetResponse & response)
|
||||
{
|
||||
if (response.error)
|
||||
std::cerr << "Error (set) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
|
||||
else
|
||||
std::cerr << "Set\n";
|
||||
|
||||
//event.set();
|
||||
});
|
||||
|
||||
//event.wait();
|
||||
|
||||
std::cout << "list\n";
|
||||
|
||||
zk.list("/",
|
||||
[&](const ZooKeeper::ListResponse & response)
|
||||
{
|
||||
if (response.error)
|
||||
std::cerr << "Error (list) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
|
||||
else
|
||||
{
|
||||
std::cerr << "Children:\n";
|
||||
for (const auto & name : response.names)
|
||||
std::cerr << name << "\n";
|
||||
}
|
||||
|
||||
//event.set();
|
||||
},
|
||||
[](const ZooKeeper::WatchResponse & response)
|
||||
{
|
||||
if (response.error)
|
||||
std::cerr << "Watch (list) on /, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
|
||||
else
|
||||
std::cerr << "Watch (list) on /, path: " << response.path << ", type: " << response.type << '\n';
|
||||
});
|
||||
|
||||
//event.wait();
|
||||
|
||||
std::cout << "exists\n";
|
||||
|
||||
zk.exists("/test",
|
||||
[&](const ZooKeeper::ExistsResponse & response)
|
||||
{
|
||||
if (response.error)
|
||||
std::cerr << "Error (exists) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
|
||||
else
|
||||
std::cerr << "Exists\n";
|
||||
|
||||
//event.set();
|
||||
},
|
||||
[](const ZooKeeper::WatchResponse & response)
|
||||
{
|
||||
if (response.error)
|
||||
std::cerr << "Watch (exists) on /test, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
|
||||
else
|
||||
std::cerr << "Watch (exists) on /test, path: " << response.path << ", type: " << response.type << '\n';
|
||||
});
|
||||
|
||||
//event.wait();
|
||||
|
||||
std::cout << "remove\n";
|
||||
|
||||
zk.remove("/test", -1, [&](const ZooKeeper::RemoveResponse & response)
|
||||
{
|
||||
if (response.error)
|
||||
std::cerr << "Error (remove) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
|
||||
else
|
||||
std::cerr << "Removed\n";
|
||||
|
||||
//event.set();
|
||||
});
|
||||
|
||||
//event.wait();
|
||||
|
||||
std::cout << "multi\n";
|
||||
|
||||
ZooKeeper::Requests ops;
|
||||
|
||||
{
|
||||
ZooKeeper::CreateRequest create_request;
|
||||
create_request.path = "/test";
|
||||
create_request.data = "multi1";
|
||||
ops.emplace_back(std::make_shared<ZooKeeper::CreateRequest>(std::move(create_request)));
|
||||
}
|
||||
|
||||
{
|
||||
ZooKeeper::SetRequest set_request;
|
||||
set_request.path = "/test";
|
||||
set_request.data = "multi2";
|
||||
ops.emplace_back(std::make_shared<ZooKeeper::SetRequest>(std::move(set_request)));
|
||||
}
|
||||
|
||||
{
|
||||
ZooKeeper::RemoveRequest remove_request;
|
||||
remove_request.path = "/test";
|
||||
ops.emplace_back(std::make_shared<ZooKeeper::RemoveRequest>(std::move(remove_request)));
|
||||
}
|
||||
|
||||
zk.multi(ops, [&](const ZooKeeper::MultiResponse & response)
|
||||
{
|
||||
if (response.error)
|
||||
std::cerr << "Error (multi) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
|
||||
else
|
||||
{
|
||||
for (const auto & elem : response.responses)
|
||||
if (elem->error)
|
||||
std::cerr << "Error (elem) " << elem->error << ": " << ZooKeeper::errorMessage(elem->error) << '\n';
|
||||
|
||||
std::cerr << "Created path: " << typeid_cast<const ZooKeeper::CreateResponse &>(*response.responses[0]).path_created << '\n';
|
||||
}
|
||||
|
||||
event.set();
|
||||
});
|
||||
|
||||
event.wait();
|
||||
return 0;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
std::cerr << DB::getCurrentExceptionMessage(__PRETTY_FUNCTION__) << '\n';
|
||||
return 1;
|
||||
}
|
26
dbms/src/Common/ZooKeeper/tests/zookeeper_impl.cpp
Normal file
26
dbms/src/Common/ZooKeeper/tests/zookeeper_impl.cpp
Normal file
@ -0,0 +1,26 @@
|
||||
#include <Common/ZooKeeper/ZooKeeperImpl.h>
|
||||
#include <iostream>
|
||||
|
||||
|
||||
int main()
|
||||
try
|
||||
{
|
||||
ZooKeeperImpl::ZooKeeper zookeeper({Poco::Net::SocketAddress{"localhost:2181"}}, "", "", "", {30, 0}, {0, 50000}, {0, 50000});
|
||||
|
||||
zookeeper.create("/test", "hello", false, false, {}, [](const ZooKeeperImpl::ZooKeeper::CreateResponse & response)
|
||||
{
|
||||
if (response.error)
|
||||
std::cerr << "Error " << response.error << ": " << ZooKeeperImpl::ZooKeeper::errorMessage(response.error) << "\n";
|
||||
else
|
||||
std::cerr << "Path created: " << response.path_created << "\n";
|
||||
});
|
||||
|
||||
sleep(100);
|
||||
|
||||
return 0;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
DB::tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
return 1;
|
||||
}
|
@ -1,13 +1,10 @@
|
||||
#include <sys/stat.h>
|
||||
#include <unistd.h>
|
||||
|
||||
#include <string>
|
||||
#include <iostream>
|
||||
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <Poco/File.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <port/unistd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,12 +1,10 @@
|
||||
#include <time.h>
|
||||
#include <unistd.h>
|
||||
#include <port/unistd.h>
|
||||
#include <sys/types.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Core/Types.h>
|
||||
|
||||
#ifdef __APPLE__
|
||||
#include <common/apple_rt.h>
|
||||
#endif
|
||||
|
@ -7,15 +7,13 @@
|
||||
#include <Common/CompactArray.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
|
||||
#include <boost/filesystem.hpp>
|
||||
|
||||
#include <string>
|
||||
#include <iostream>
|
||||
#include <fstream>
|
||||
#include <stdexcept>
|
||||
#include <unistd.h>
|
||||
#include <cstdlib>
|
||||
#include <port/unistd.h>
|
||||
|
||||
namespace fs = boost::filesystem;
|
||||
|
||||
|
@ -312,7 +312,7 @@ MutableColumns Block::mutateColumns() const
|
||||
size_t num_columns = data.size();
|
||||
MutableColumns columns(num_columns);
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
columns[i] = data[i].column ? data[i].column->mutate() : data[i].type->createColumn();
|
||||
columns[i] = data[i].column ? (*std::move(data[i].column)).mutate() : data[i].type->createColumn();
|
||||
return columns;
|
||||
}
|
||||
|
||||
|
@ -109,7 +109,7 @@ public:
|
||||
/** Get empty columns with the same types as in block. */
|
||||
MutableColumns cloneEmptyColumns() const;
|
||||
|
||||
/** Get columns from block for mutation. */
|
||||
/** Get columns from block for mutation. Columns in block will be nullptr. */
|
||||
MutableColumns mutateColumns() const;
|
||||
|
||||
/** Replace columns in a block */
|
||||
|
@ -60,6 +60,7 @@
|
||||
#define DBMS_MIN_REVISION_WITH_QUOTA_KEY_IN_CLIENT_INFO 54060
|
||||
#define DBMS_MIN_REVISION_WITH_TABLES_STATUS 54226
|
||||
#define DBMS_MIN_REVISION_WITH_TIME_ZONE_PARAMETER_IN_DATETIME_DATA_TYPE 54337
|
||||
#define DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME 54372
|
||||
|
||||
/// Version of ClickHouse TCP protocol. Set to git tag with latest protocol change.
|
||||
#define DBMS_TCP_PROTOCOL_VERSION 54226
|
||||
@ -74,8 +75,18 @@
|
||||
#define DEFAULT_HTTP_READ_BUFFER_TIMEOUT 1800
|
||||
#define DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT 1
|
||||
|
||||
#define ALWAYS_INLINE __attribute__((__always_inline__))
|
||||
#define NO_INLINE __attribute__((__noinline__))
|
||||
// more aliases: https://mailman.videolan.org/pipermail/x264-devel/2014-May/010660.html
|
||||
|
||||
#if defined(_MSC_VER)
|
||||
#define ALWAYS_INLINE __forceinline
|
||||
#define NO_INLINE static __declspec(noinline)
|
||||
#define MAY_ALIAS
|
||||
#else
|
||||
#define ALWAYS_INLINE __attribute__((__always_inline__))
|
||||
#define NO_INLINE __attribute__((__noinline__))
|
||||
#define MAY_ALIAS __attribute__((__may_alias__))
|
||||
#endif
|
||||
|
||||
|
||||
#define PLATFORM_NOT_SUPPORTED "The only supported platforms are x86_64 and AArch64 (work in progress)"
|
||||
|
||||
|
@ -198,14 +198,14 @@ public:
|
||||
template <typename T> T & get()
|
||||
{
|
||||
using TWithoutRef = std::remove_reference_t<T>;
|
||||
TWithoutRef * __attribute__((__may_alias__)) ptr = reinterpret_cast<TWithoutRef*>(&storage);
|
||||
TWithoutRef * MAY_ALIAS ptr = reinterpret_cast<TWithoutRef*>(&storage);
|
||||
return *ptr;
|
||||
};
|
||||
|
||||
template <typename T> const T & get() const
|
||||
{
|
||||
using TWithoutRef = std::remove_reference_t<T>;
|
||||
const TWithoutRef * __attribute__((__may_alias__)) ptr = reinterpret_cast<const TWithoutRef*>(&storage);
|
||||
const TWithoutRef * MAY_ALIAS ptr = reinterpret_cast<const TWithoutRef*>(&storage);
|
||||
return *ptr;
|
||||
};
|
||||
|
||||
@ -340,7 +340,7 @@ private:
|
||||
void createConcrete(T && x)
|
||||
{
|
||||
using JustT = std::decay_t<T>;
|
||||
JustT * __attribute__((__may_alias__)) ptr = reinterpret_cast<JustT *>(&storage);
|
||||
JustT * MAY_ALIAS ptr = reinterpret_cast<JustT *>(&storage);
|
||||
new (ptr) JustT(std::forward<T>(x));
|
||||
which = TypeToEnum<JustT>::value;
|
||||
}
|
||||
@ -350,7 +350,7 @@ private:
|
||||
void assignConcrete(T && x)
|
||||
{
|
||||
using JustT = std::decay_t<T>;
|
||||
JustT * __attribute__((__may_alias__)) ptr = reinterpret_cast<JustT *>(&storage);
|
||||
JustT * MAY_ALIAS ptr = reinterpret_cast<JustT *>(&storage);
|
||||
*ptr = std::forward<T>(x);
|
||||
}
|
||||
|
||||
@ -398,7 +398,7 @@ private:
|
||||
|
||||
void create(const char * data, size_t size)
|
||||
{
|
||||
String * __attribute__((__may_alias__)) ptr = reinterpret_cast<String*>(&storage);
|
||||
String * MAY_ALIAS ptr = reinterpret_cast<String*>(&storage);
|
||||
new (ptr) String(data, size);
|
||||
which = Types::String;
|
||||
}
|
||||
@ -434,7 +434,7 @@ private:
|
||||
template <typename T>
|
||||
void destroy()
|
||||
{
|
||||
T * __attribute__((__may_alias__)) ptr = reinterpret_cast<T*>(&storage);
|
||||
T * MAY_ALIAS ptr = reinterpret_cast<T*>(&storage);
|
||||
ptr->~T();
|
||||
}
|
||||
};
|
||||
|
@ -116,7 +116,7 @@ namespace Protocol
|
||||
};
|
||||
|
||||
/// Whether the ssl must be used.
|
||||
enum class Encryption
|
||||
enum class Secure
|
||||
{
|
||||
Disable = 0,
|
||||
Enable = 1,
|
||||
|
@ -76,7 +76,7 @@ Block ColumnGathererStream::readImpl()
|
||||
return Block();
|
||||
|
||||
output_block = Block{column.cloneEmpty()};
|
||||
MutableColumnPtr output_column = output_block.getByPosition(0).column->mutate();
|
||||
MutableColumnPtr output_column = output_block.getByPosition(0).column->assumeMutable();
|
||||
output_column->gather(*this);
|
||||
if (!output_column->empty())
|
||||
output_block.getByPosition(0).column = std::move(output_column);
|
||||
|
@ -221,7 +221,7 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, std::
|
||||
throw Exception("Logical error in MergingSortedBlockInputStream", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
merged_columns[i] = source_blocks[source_num]->getByPosition(i).column->mutate();
|
||||
merged_columns[i] = (*std::move(source_blocks[source_num]->getByPosition(i).column)).mutate();
|
||||
|
||||
// std::cerr << "copied columns\n";
|
||||
|
||||
@ -233,7 +233,7 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, std::
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
{
|
||||
auto & column = merged_columns[i];
|
||||
column = column->cut(0, merged_rows);
|
||||
column = (*column->cut(0, merged_rows)).mutate();
|
||||
}
|
||||
|
||||
cancel(false);
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <sys/ioctl.h>
|
||||
#include <unistd.h>
|
||||
|
||||
#include <port/unistd.h>
|
||||
#include <DataStreams/PrettyBlockOutputStream.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <DataStreams/PushingToViewsBlockOutputStream.h>
|
||||
#include <DataStreams/SquashingBlockInputStream.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
|
||||
|
||||
@ -35,7 +36,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream(
|
||||
auto & materialized_view = dynamic_cast<const StorageMaterializedView &>(*dependent_table);
|
||||
|
||||
auto query = materialized_view.getInnerQuery();
|
||||
auto out = std::make_shared<PushingToViewsBlockOutputStream>(
|
||||
BlockOutputStreamPtr out = std::make_shared<PushingToViewsBlockOutputStream>(
|
||||
database_table.first, database_table.second, dependent_table, *views_context, ASTPtr());
|
||||
views.emplace_back(ViewInfo{std::move(query), database_table.first, database_table.second, std::move(out)});
|
||||
}
|
||||
@ -66,8 +67,19 @@ void PushingToViewsBlockOutputStream::write(const Block & block)
|
||||
{
|
||||
BlockInputStreamPtr from = std::make_shared<OneBlockInputStream>(block);
|
||||
InterpreterSelectQuery select(view.query, *views_context, {}, QueryProcessingStage::Complete, 0, from);
|
||||
BlockInputStreamPtr data = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
|
||||
copyData(*data, *view.out);
|
||||
BlockInputStreamPtr in = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
|
||||
/// Squashing is needed here because the materialized view query can generate a lot of blocks
|
||||
/// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY
|
||||
/// and two-level aggregation is triggered).
|
||||
in = std::make_shared<SquashingBlockInputStream>(
|
||||
in, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes);
|
||||
|
||||
in->readPrefix();
|
||||
|
||||
while (Block result_block = in->read())
|
||||
view.out->write(result_block);
|
||||
|
||||
in->readSuffix();
|
||||
}
|
||||
catch (Exception & ex)
|
||||
{
|
||||
|
@ -29,18 +29,27 @@ public:
|
||||
{
|
||||
if (output)
|
||||
output->flush();
|
||||
|
||||
for (auto & view : views)
|
||||
view.out->flush();
|
||||
}
|
||||
|
||||
void writePrefix() override
|
||||
{
|
||||
if (output)
|
||||
output->writePrefix();
|
||||
|
||||
for (auto & view : views)
|
||||
view.out->writePrefix();
|
||||
}
|
||||
|
||||
void writeSuffix() override
|
||||
{
|
||||
if (output)
|
||||
output->writeSuffix();
|
||||
|
||||
for (auto & view : views)
|
||||
view.out->writeSuffix();
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -62,7 +62,7 @@ void SquashingTransform::append(Block && block)
|
||||
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
MutableColumnPtr mutable_column = accumulated_block.getByPosition(i).column->mutate();
|
||||
MutableColumnPtr mutable_column = (*std::move(accumulated_block.getByPosition(i).column)).mutate();
|
||||
mutable_column->insertRangeFrom(*block.getByPosition(i).column, 0, rows);
|
||||
accumulated_block.getByPosition(i).column = std::move(mutable_column);
|
||||
}
|
||||
|
@ -259,11 +259,11 @@ Block SummingSortedBlockInputStream::readImpl()
|
||||
if (checkDataType<DataTypeTuple>(desc.function->getReturnType().get()))
|
||||
{
|
||||
size_t tuple_size = desc.column_numbers.size();
|
||||
Columns tuple_columns(tuple_size);
|
||||
MutableColumns tuple_columns(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
tuple_columns[i] = header.safeGetByPosition(desc.column_numbers[i]).column;
|
||||
tuple_columns[i] = header.safeGetByPosition(desc.column_numbers[i]).column->assumeMutable();
|
||||
|
||||
desc.merged_column = ColumnTuple::create(tuple_columns);
|
||||
desc.merged_column = ColumnTuple::create(std::move(tuple_columns));
|
||||
}
|
||||
else
|
||||
desc.merged_column = header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty();
|
||||
|
@ -329,10 +329,10 @@ void DataTypeTuple::deserializeBinaryBulkWithMultipleStreams(
|
||||
MutableColumnPtr DataTypeTuple::createColumn() const
|
||||
{
|
||||
size_t size = elems.size();
|
||||
Columns tuple_columns(size);
|
||||
MutableColumns tuple_columns(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
tuple_columns[i] = elems[i]->createColumn();
|
||||
return ColumnTuple::create(tuple_columns);
|
||||
return ColumnTuple::create(std::move(tuple_columns));
|
||||
}
|
||||
|
||||
Field DataTypeTuple::getDefault() const
|
||||
|
@ -3,15 +3,21 @@
|
||||
#include <Interpreters/ExternalDictionaries.h>
|
||||
#include <Storages/StorageDictionary.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
DatabaseDictionary::DatabaseDictionary(const String & name_, const Context & context)
|
||||
@ -85,7 +91,7 @@ StoragePtr DatabaseDictionary::tryGetTable(
|
||||
|
||||
DatabaseIteratorPtr DatabaseDictionary::getIterator(const Context & /*context*/)
|
||||
{
|
||||
return std::make_unique<DatabaseSnaphotIterator>(loadTables());
|
||||
return std::make_unique<DatabaseSnapshotIterator>(loadTables());
|
||||
}
|
||||
|
||||
bool DatabaseDictionary::empty(const Context & /*context*/) const
|
||||
@ -153,11 +159,54 @@ time_t DatabaseDictionary::getTableMetadataModificationTime(
|
||||
return static_cast<time_t>(0);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseDictionary::getCreateQuery(
|
||||
const Context &,
|
||||
const String &) const
|
||||
ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const Context & context,
|
||||
const String & table_name, bool throw_on_error) const
|
||||
{
|
||||
throw Exception("There is no CREATE TABLE query for DatabaseDictionary tables", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
String query;
|
||||
{
|
||||
WriteBufferFromString buffer(query);
|
||||
|
||||
const auto & dictionaries = context.getExternalDictionaries();
|
||||
auto dictionary = throw_on_error ? dictionaries.getDictionary(table_name)
|
||||
: dictionaries.tryGetDictionary(table_name);
|
||||
|
||||
auto names_and_types = StorageDictionary::getNamesAndTypes(dictionary->getStructure());
|
||||
buffer << "CREATE TABLE " << backQuoteIfNeed(name) << '.' << backQuoteIfNeed(table_name) << " (";
|
||||
buffer << StorageDictionary::generateNamesAndTypesDescription(names_and_types.begin(), names_and_types.end());
|
||||
buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")";
|
||||
}
|
||||
|
||||
ParserCreateQuery parser;
|
||||
const char * pos = query.data();
|
||||
std::string error_message;
|
||||
auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message,
|
||||
/* hilite = */ false, "", /* allow_multi_statements = */ false);
|
||||
|
||||
if (!ast && throw_on_error)
|
||||
throw Exception(error_message, ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
return ast;
|
||||
}
|
||||
|
||||
ASTPtr DatabaseDictionary::getCreateTableQuery(const Context & context, const String & table_name) const
|
||||
{
|
||||
return getCreateTableQueryImpl(context, table_name, true);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseDictionary::tryGetCreateTableQuery(const Context & context, const String & table_name) const
|
||||
{
|
||||
return getCreateTableQueryImpl(context, table_name, false);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseDictionary::getCreateDatabaseQuery(const Context & /*context*/) const
|
||||
{
|
||||
String query;
|
||||
{
|
||||
WriteBufferFromString buffer(query);
|
||||
buffer << "CREATE DATABASE " << backQuoteIfNeed(name) << " ENGINE = Dictionary";
|
||||
}
|
||||
ParserCreateQuery parser;
|
||||
return parseQuery(parser, query.data(), query.data() + query.size(), "");
|
||||
}
|
||||
|
||||
void DatabaseDictionary::shutdown()
|
||||
|
@ -22,16 +22,6 @@ class ExternalDictionaries;
|
||||
*/
|
||||
class DatabaseDictionary : public IDatabase
|
||||
{
|
||||
private:
|
||||
const String name;
|
||||
mutable std::mutex mutex;
|
||||
const ExternalDictionaries & external_dictionaries;
|
||||
std::unordered_set<String> deleted_tables;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
Tables loadTables();
|
||||
|
||||
public:
|
||||
DatabaseDictionary(const String & name_, const Context & context);
|
||||
|
||||
@ -86,12 +76,30 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
ASTPtr getCreateQuery(
|
||||
ASTPtr getCreateTableQuery(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
ASTPtr tryGetCreateTableQuery(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
|
||||
void shutdown() override;
|
||||
void drop() override;
|
||||
|
||||
private:
|
||||
const String name;
|
||||
mutable std::mutex mutex;
|
||||
const ExternalDictionaries & external_dictionaries;
|
||||
std::unordered_set<String> deleted_tables;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
Tables loadTables();
|
||||
|
||||
ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -8,75 +8,22 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
}
|
||||
|
||||
DatabaseMemory::DatabaseMemory(String name_)
|
||||
: DatabaseWithOwnTablesBase(std::move(name_))
|
||||
, log(&Logger::get("DatabaseMemory(" + name + ")"))
|
||||
{}
|
||||
|
||||
void DatabaseMemory::loadTables(
|
||||
Context & /*context*/,
|
||||
ThreadPool * /*thread_pool*/,
|
||||
bool /*has_force_restore_data_flag*/)
|
||||
{
|
||||
log = &Logger::get("DatabaseMemory(" + name + ")");
|
||||
|
||||
/// Nothing to load.
|
||||
}
|
||||
|
||||
bool DatabaseMemory::isTableExist(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return tables.find(table_name) != tables.end();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseMemory::tryGetTable(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it == tables.end())
|
||||
return {};
|
||||
return it->second;
|
||||
}
|
||||
|
||||
DatabaseIteratorPtr DatabaseMemory::getIterator(const Context & /*context*/)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return std::make_unique<DatabaseSnaphotIterator>(tables);
|
||||
}
|
||||
|
||||
bool DatabaseMemory::empty(const Context & /*context*/) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return tables.empty();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseMemory::detachTable(const String & table_name)
|
||||
{
|
||||
StoragePtr res;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it == tables.end())
|
||||
throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
res = it->second;
|
||||
tables.erase(it);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void DatabaseMemory::attachTable(const String & table_name, const StoragePtr & table)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
if (!tables.emplace(table_name, table).second)
|
||||
throw Exception("Table " + name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
|
||||
void DatabaseMemory::createTable(
|
||||
const Context & /*context*/,
|
||||
const String & table_name,
|
||||
@ -118,31 +65,17 @@ time_t DatabaseMemory::getTableMetadataModificationTime(
|
||||
return static_cast<time_t>(0);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseMemory::getCreateQuery(
|
||||
ASTPtr DatabaseMemory::getCreateTableQuery(
|
||||
const Context &,
|
||||
const String &) const
|
||||
{
|
||||
throw Exception("There is no CREATE TABLE query for DatabaseMemory tables", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
}
|
||||
|
||||
void DatabaseMemory::shutdown()
|
||||
ASTPtr DatabaseMemory::getCreateDatabaseQuery(
|
||||
const Context &) const
|
||||
{
|
||||
/// You can not hold a lock during shutdown.
|
||||
/// Because inside `shutdown` function tables can work with database, and mutex is not recursive.
|
||||
|
||||
Tables tables_snapshot;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
tables_snapshot = tables;
|
||||
}
|
||||
|
||||
for (const auto & kv: tables_snapshot)
|
||||
{
|
||||
kv.second->shutdown();
|
||||
}
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
tables.clear();
|
||||
throw Exception("There is no CREATE DATABASE query for DatabaseMemory", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
}
|
||||
|
||||
void DatabaseMemory::drop()
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <mutex>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
@ -16,18 +14,10 @@ namespace DB
|
||||
* All tables are created by calling code.
|
||||
* TODO: Maybe DatabaseRuntime is more suitable class name.
|
||||
*/
|
||||
class DatabaseMemory : public IDatabase
|
||||
class DatabaseMemory : public DatabaseWithOwnTablesBase
|
||||
{
|
||||
protected:
|
||||
const String name;
|
||||
mutable std::mutex mutex;
|
||||
Tables tables;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
public:
|
||||
|
||||
DatabaseMemory(const String & name_) : name(name_) {}
|
||||
DatabaseMemory(String name_);
|
||||
|
||||
String getEngineName() const override { return "Memory"; }
|
||||
|
||||
@ -36,18 +26,6 @@ public:
|
||||
ThreadPool * thread_pool,
|
||||
bool has_force_restore_data_flag) override;
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
|
||||
DatabaseIteratorPtr getIterator(const Context & context) override;
|
||||
|
||||
bool isTableExist(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
StoragePtr tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
void createTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
@ -58,9 +36,6 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & table) override;
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
void renameTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
@ -77,12 +52,15 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
ASTPtr getCreateQuery(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
ASTPtr getCreateTableQuery(const Context & context, const String & table_name) const override;
|
||||
ASTPtr tryGetCreateTableQuery(const Context &, const String &) const override { return nullptr; }
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
|
||||
void shutdown() override;
|
||||
void drop() override;
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -25,12 +25,12 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int TABLE_METADATA_DOESNT_EXIST;
|
||||
extern const int CANNOT_CREATE_TABLE_FROM_METADATA;
|
||||
extern const int INCORRECT_FILE_NAME;
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
|
||||
@ -45,6 +45,12 @@ namespace detail
|
||||
{
|
||||
return base_path + (endsWith(base_path, "/") ? "" : "/") + escapeForFileName(table_name) + ".sql";
|
||||
}
|
||||
|
||||
String getDatabaseMetadataPath(const String & base_path)
|
||||
{
|
||||
return (endsWith(base_path, "/") ? base_path.substr(0, base_path.size() - 1) : base_path) + ".sql";
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static void loadTable(
|
||||
@ -94,8 +100,11 @@ static void loadTable(
|
||||
}
|
||||
|
||||
|
||||
DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context)
|
||||
: DatabaseMemory(name_), metadata_path(metadata_path_), data_path(context.getPath() + "data/" + escapeForFileName(name_) + "/")
|
||||
DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context)
|
||||
: DatabaseWithOwnTablesBase(std::move(name_))
|
||||
, metadata_path(metadata_path_)
|
||||
, data_path(context.getPath() + "data/" + escapeForFileName(name) + "/")
|
||||
, log(&Logger::get("DatabaseOrdinary (" + name + ")"))
|
||||
{
|
||||
Poco::File(data_path).createDirectories();
|
||||
}
|
||||
@ -106,8 +115,6 @@ void DatabaseOrdinary::loadTables(
|
||||
ThreadPool * thread_pool,
|
||||
bool has_force_restore_data_flag)
|
||||
{
|
||||
log = &Logger::get("DatabaseOrdinary (" + name + ")");
|
||||
|
||||
using FileNames = std::vector<std::string>;
|
||||
FileNames file_names;
|
||||
|
||||
@ -329,19 +336,42 @@ void DatabaseOrdinary::removeTable(
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
static ASTPtr getCreateQueryImpl(const String & path, const String & table_name)
|
||||
static ASTPtr getQueryFromMetadata(const String & metadata_path, bool throw_on_error = true)
|
||||
{
|
||||
String table_metadata_path = detail::getTableMetadataPath(path, table_name);
|
||||
if (!Poco::File(metadata_path).exists())
|
||||
return nullptr;
|
||||
|
||||
String query;
|
||||
|
||||
{
|
||||
ReadBufferFromFile in(table_metadata_path, 4096);
|
||||
ReadBufferFromFile in(metadata_path, 4096);
|
||||
readStringUntilEOF(query, in);
|
||||
}
|
||||
|
||||
ParserCreateQuery parser;
|
||||
return parseQuery(parser, query.data(), query.data() + query.size(), "in file " + table_metadata_path);
|
||||
const char * pos = query.data();
|
||||
std::string error_message;
|
||||
auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message, /* hilite = */ false,
|
||||
"in file " + metadata_path, /* allow_multi_statements = */ false);
|
||||
|
||||
if (!ast && throw_on_error)
|
||||
throw Exception(error_message, ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
return ast;
|
||||
}
|
||||
|
||||
static ASTPtr getCreateQueryFromMetadata(const String & metadata_path, const String & database, bool throw_on_error)
|
||||
{
|
||||
ASTPtr ast = getQueryFromMetadata(metadata_path, throw_on_error);
|
||||
|
||||
if (ast)
|
||||
{
|
||||
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
||||
ast_create_query.attach = false;
|
||||
ast_create_query.database = database;
|
||||
}
|
||||
|
||||
return ast;
|
||||
}
|
||||
|
||||
|
||||
@ -378,7 +408,9 @@ void DatabaseOrdinary::renameTable(
|
||||
throw Exception{e};
|
||||
}
|
||||
|
||||
ASTPtr ast = getCreateQueryImpl(metadata_path, table_name);
|
||||
ASTPtr ast = getQueryFromMetadata(detail::getTableMetadataPath(metadata_path, table_name));
|
||||
if (!ast)
|
||||
throw Exception("There is no metadata file for table " + table_name, ErrorCodes::FILE_DOESNT_EXIST);
|
||||
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
||||
ast_create_query.table = to_table_name;
|
||||
|
||||
@ -405,28 +437,51 @@ time_t DatabaseOrdinary::getTableMetadataModificationTime(
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
ASTPtr DatabaseOrdinary::getCreateQuery(
|
||||
const Context & context,
|
||||
const String & table_name) const
|
||||
ASTPtr DatabaseOrdinary::getCreateTableQueryImpl(const Context & context,
|
||||
const String & table_name, bool throw_on_error) const
|
||||
{
|
||||
ASTPtr ast;
|
||||
try
|
||||
{
|
||||
ast = getCreateQueryImpl(metadata_path, table_name);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
/// Handle system.* tables for which there are no table.sql files
|
||||
if (e.code() == ErrorCodes::FILE_DOESNT_EXIST && tryGetTable(context, table_name) != nullptr)
|
||||
throw Exception("There is no CREATE TABLE query for table " + table_name, ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
|
||||
throw;
|
||||
auto table_metadata_path = detail::getTableMetadataPath(metadata_path, table_name);
|
||||
ast = getCreateQueryFromMetadata(table_metadata_path, name, throw_on_error);
|
||||
if (!ast && throw_on_error)
|
||||
{
|
||||
/// Handle system.* tables for which there are no table.sql files.
|
||||
bool has_table = tryGetTable(context, table_name) != nullptr;
|
||||
|
||||
auto msg = has_table
|
||||
? "There is no CREATE TABLE query for table "
|
||||
: "There is no metadata file for table ";
|
||||
|
||||
throw Exception(msg + table_name, ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
}
|
||||
|
||||
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
|
||||
ast_create_query.attach = false;
|
||||
ast_create_query.database = name;
|
||||
return ast;
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOrdinary::getCreateTableQuery(const Context & context, const String & table_name) const
|
||||
{
|
||||
return getCreateTableQueryImpl(context, table_name, true);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOrdinary::tryGetCreateTableQuery(const Context & context, const String & table_name) const
|
||||
{
|
||||
return getCreateTableQueryImpl(context, table_name, false);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOrdinary::getCreateDatabaseQuery(const Context & /*context*/) const
|
||||
{
|
||||
ASTPtr ast;
|
||||
|
||||
auto database_metadata_path = detail::getDatabaseMetadataPath(metadata_path);
|
||||
ast = getCreateQueryFromMetadata(database_metadata_path, name, true);
|
||||
if (!ast)
|
||||
{
|
||||
/// Handle databases (such as default) for which there are no database.sql files.
|
||||
String query = "CREATE DATABASE " + backQuoteIfNeed(name) + " ENGINE = Ordinary";
|
||||
ParserCreateQuery parser;
|
||||
ast = parseQuery(parser, query.data(), query.data() + query.size(), "");
|
||||
}
|
||||
|
||||
return ast;
|
||||
}
|
||||
@ -458,7 +513,6 @@ void DatabaseOrdinary::drop()
|
||||
/// No additional removal actions are required.
|
||||
}
|
||||
|
||||
|
||||
void DatabaseOrdinary::alterTable(
|
||||
const Context & context,
|
||||
const String & name,
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -10,14 +10,10 @@ namespace DB
|
||||
* It stores tables list in filesystem using list of .sql files,
|
||||
* that contain declaration of table represented by SQL ATTACH TABLE query.
|
||||
*/
|
||||
class DatabaseOrdinary : public DatabaseMemory
|
||||
class DatabaseOrdinary : public DatabaseWithOwnTablesBase
|
||||
{
|
||||
protected:
|
||||
const String metadata_path;
|
||||
const String data_path;
|
||||
|
||||
public:
|
||||
DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context);
|
||||
DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context);
|
||||
|
||||
String getEngineName() const override { return "Ordinary"; }
|
||||
|
||||
@ -52,10 +48,16 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
ASTPtr getCreateQuery(
|
||||
ASTPtr getCreateTableQuery(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
ASTPtr tryGetCreateTableQuery(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
|
||||
String getDataPath() const override;
|
||||
String getMetadataPath() const override;
|
||||
String getTableMetadataPath(const String & table_name) const override;
|
||||
@ -64,7 +66,13 @@ public:
|
||||
void drop() override;
|
||||
|
||||
private:
|
||||
const String metadata_path;
|
||||
const String data_path;
|
||||
Poco::Logger * log;
|
||||
|
||||
void startupTables(ThreadPool * thread_pool);
|
||||
|
||||
ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -16,6 +16,9 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
@ -80,4 +83,90 @@ std::pair<String, StoragePtr> createTableFromDefinition(
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
bool DatabaseWithOwnTablesBase::isTableExist(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return tables.find(table_name) != tables.end();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseWithOwnTablesBase::tryGetTable(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it == tables.end())
|
||||
return {};
|
||||
return it->second;
|
||||
}
|
||||
|
||||
DatabaseIteratorPtr DatabaseWithOwnTablesBase::getIterator(const Context & /*context*/)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return std::make_unique<DatabaseSnapshotIterator>(tables);
|
||||
}
|
||||
|
||||
bool DatabaseWithOwnTablesBase::empty(const Context & /*context*/) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return tables.empty();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
|
||||
{
|
||||
StoragePtr res;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it == tables.end())
|
||||
throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
res = it->second;
|
||||
tables.erase(it);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
if (!tables.emplace(table_name, table).second)
|
||||
throw Exception("Table " + name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
|
||||
void DatabaseWithOwnTablesBase::shutdown()
|
||||
{
|
||||
/// You can not hold a lock during shutdown.
|
||||
/// Because inside `shutdown` function tables can work with database, and mutex is not recursive.
|
||||
|
||||
Tables tables_snapshot;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
tables_snapshot = tables;
|
||||
}
|
||||
|
||||
for (const auto & kv: tables_snapshot)
|
||||
{
|
||||
kv.second->shutdown();
|
||||
}
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
tables.clear();
|
||||
}
|
||||
|
||||
DatabaseWithOwnTablesBase::~DatabaseWithOwnTablesBase()
|
||||
{
|
||||
try
|
||||
{
|
||||
shutdown();
|
||||
}
|
||||
catch(...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -36,17 +36,17 @@ std::pair<String, StoragePtr> createTableFromDefinition(
|
||||
|
||||
|
||||
/// Copies list of tables and iterates through such snapshot.
|
||||
class DatabaseSnaphotIterator : public IDatabaseIterator
|
||||
class DatabaseSnapshotIterator : public IDatabaseIterator
|
||||
{
|
||||
private:
|
||||
Tables tables;
|
||||
Tables::iterator it;
|
||||
|
||||
public:
|
||||
DatabaseSnaphotIterator(Tables & tables_)
|
||||
DatabaseSnapshotIterator(Tables & tables_)
|
||||
: tables(tables_), it(tables.begin()) {}
|
||||
|
||||
DatabaseSnaphotIterator(Tables && tables_)
|
||||
DatabaseSnapshotIterator(Tables && tables_)
|
||||
: tables(tables_), it(tables.begin()) {}
|
||||
|
||||
void next() override
|
||||
@ -70,4 +70,38 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
/// A base class for databases that manage their own list of tables.
|
||||
class DatabaseWithOwnTablesBase : public IDatabase
|
||||
{
|
||||
public:
|
||||
bool isTableExist(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
StoragePtr tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & table) override;
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
DatabaseIteratorPtr getIterator(const Context & context) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
virtual ~DatabaseWithOwnTablesBase() override;
|
||||
|
||||
protected:
|
||||
String name;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
Tables tables;
|
||||
|
||||
DatabaseWithOwnTablesBase(String name_) : name(std::move(name_)) { }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -122,9 +122,15 @@ public:
|
||||
const String & name) = 0;
|
||||
|
||||
/// Get the CREATE TABLE query for the table. It can also provide information for detached tables for which there is metadata.
|
||||
virtual ASTPtr getCreateQuery(
|
||||
const Context & context,
|
||||
const String & name) const = 0;
|
||||
virtual ASTPtr tryGetCreateTableQuery(const Context & context, const String & name) const = 0;
|
||||
|
||||
virtual ASTPtr getCreateTableQuery(const Context & context, const String & name) const
|
||||
{
|
||||
return tryGetCreateTableQuery(context, name);
|
||||
}
|
||||
|
||||
/// Get the CREATE DATABASE query for current database.
|
||||
virtual ASTPtr getCreateDatabaseQuery(const Context & context) const = 0;
|
||||
|
||||
/// Returns path for persistent data storage if the database supports it, empty string otherwise
|
||||
virtual String getDataPath() const { return {}; }
|
||||
|
@ -21,13 +21,15 @@ namespace ErrorCodes
|
||||
static const size_t MAX_CONNECTIONS = 16;
|
||||
|
||||
static ConnectionPoolWithFailoverPtr createPool(
|
||||
const std::string & host, UInt16 port, const std::string & db,
|
||||
const std::string & host, UInt16 port, bool secure, const std::string & db,
|
||||
const std::string & user, const std::string & password, const Context & context)
|
||||
{
|
||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(context.getSettingsRef());
|
||||
ConnectionPoolPtrs pools;
|
||||
pools.emplace_back(std::make_shared<ConnectionPool>(
|
||||
MAX_CONNECTIONS, host, port, db, user, password, timeouts, "ClickHouseDictionarySource"));
|
||||
MAX_CONNECTIONS, host, port, db, user, password, timeouts, "ClickHouseDictionarySource",
|
||||
Protocol::Compression::Enable,
|
||||
secure ? Protocol::Secure::Enable : Protocol::Secure::Disable));
|
||||
return std::make_shared<ConnectionPoolWithFailover>(pools, LoadBalancing::RANDOM);
|
||||
}
|
||||
|
||||
@ -41,6 +43,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
|
||||
dict_struct{dict_struct_},
|
||||
host{config.getString(config_prefix + ".host")},
|
||||
port(config.getInt(config_prefix + ".port")),
|
||||
secure(config.getBool(config_prefix + ".secure", false)),
|
||||
user{config.getString(config_prefix + ".user", "")},
|
||||
password{config.getString(config_prefix + ".password", "")},
|
||||
db{config.getString(config_prefix + ".db", "")},
|
||||
@ -50,7 +53,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
|
||||
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
|
||||
sample_block{sample_block}, context(context),
|
||||
is_local{isLocalAddress({ host, port }, config.getInt("tcp_port", 0))},
|
||||
pool{is_local ? nullptr : createPool(host, port, db, user, password, context)},
|
||||
pool{is_local ? nullptr : createPool(host, port, secure, db, user, password, context)},
|
||||
load_all_query{query_builder.composeLoadAllQuery()}
|
||||
{}
|
||||
|
||||
@ -58,14 +61,16 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
|
||||
ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionarySource & other)
|
||||
: update_time{other.update_time},
|
||||
dict_struct{other.dict_struct},
|
||||
host{other.host}, port{other.port}, user{other.user}, password{other.password},
|
||||
host{other.host}, port{other.port},
|
||||
secure{other.secure},
|
||||
user{other.user}, password{other.password},
|
||||
db{other.db}, table{other.table},
|
||||
where{other.where},
|
||||
update_field{other.update_field},
|
||||
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
|
||||
sample_block{other.sample_block}, context(other.context),
|
||||
is_local{other.is_local},
|
||||
pool{is_local ? nullptr : createPool(host, port, db, user, password, context)},
|
||||
pool{is_local ? nullptr : createPool(host, port, secure, db, user, password, context)},
|
||||
load_all_query{other.load_all_query}
|
||||
{}
|
||||
|
||||
|
@ -53,6 +53,7 @@ private:
|
||||
const DictionaryStructure dict_struct;
|
||||
const std::string host;
|
||||
const UInt16 port;
|
||||
const bool secure;
|
||||
const std::string user;
|
||||
const std::string password;
|
||||
const std::string db;
|
||||
|
@ -289,7 +289,7 @@ void ComplexKeyHashedDictionary::updateData()
|
||||
for (const auto attribute_idx : ext::range(0, keys_size + attributes_size))
|
||||
{
|
||||
const IColumn & update_column = *block.getByPosition(attribute_idx).column.get();
|
||||
MutableColumnPtr saved_column = saved_block->getByPosition(attribute_idx).column->mutate();
|
||||
MutableColumnPtr saved_column = saved_block->getByPosition(attribute_idx).column->assumeMutable();
|
||||
saved_column->insertRangeFrom(update_column, 0, update_column.size());
|
||||
}
|
||||
}
|
||||
|
@ -66,7 +66,7 @@ void ExternalResultDescription::init(const Block & sample_block_)
|
||||
/// If default value for column was not provided, use default from data type.
|
||||
if (sample_columns.back()->empty())
|
||||
{
|
||||
MutableColumnPtr mutable_column = sample_columns.back()->mutate();
|
||||
MutableColumnPtr mutable_column = (*std::move(sample_columns.back())).mutate();
|
||||
column.type->insertDefaultInto(*mutable_column);
|
||||
sample_columns.back() = std::move(mutable_column);
|
||||
}
|
||||
|
@ -316,7 +316,7 @@ void FlatDictionary::updateData()
|
||||
for (const auto attribute_idx : ext::range(0, attributes.size() + 1))
|
||||
{
|
||||
const IColumn & update_column = *block.getByPosition(attribute_idx).column.get();
|
||||
MutableColumnPtr saved_column = saved_block->getByPosition(attribute_idx).column->mutate();
|
||||
MutableColumnPtr saved_column = saved_block->getByPosition(attribute_idx).column->assumeMutable();
|
||||
saved_column->insertRangeFrom(update_column, 0, update_column.size());
|
||||
}
|
||||
}
|
||||
|
@ -310,7 +310,7 @@ void HashedDictionary::updateData()
|
||||
for (const auto attribute_idx : ext::range(0, attributes.size() + 1))
|
||||
{
|
||||
const IColumn & update_column = *block.getByPosition(attribute_idx).column.get();
|
||||
MutableColumnPtr saved_column = saved_block->getByPosition(attribute_idx).column->mutate();
|
||||
MutableColumnPtr saved_column = saved_block->getByPosition(attribute_idx).column->assumeMutable();
|
||||
saved_column->insertRangeFrom(update_column, 0, update_column.size());
|
||||
}
|
||||
}
|
||||
|
@ -179,7 +179,7 @@ struct IndexIdentity
|
||||
/// For countEqual.
|
||||
struct IndexCount
|
||||
{
|
||||
using ResultType = UInt32;
|
||||
using ResultType = UInt64;
|
||||
static bool apply(size_t, ResultType & current) { ++current; return true; }
|
||||
};
|
||||
|
||||
|
@ -592,7 +592,7 @@ private:
|
||||
const ColumnPtr & result_column = temporary_block.getByPosition(3).column;
|
||||
if (result_column->isColumnNullable())
|
||||
{
|
||||
MutableColumnPtr mutable_result_column = result_column->mutate();
|
||||
MutableColumnPtr mutable_result_column = (*std::move(result_column)).mutate();
|
||||
static_cast<ColumnNullable &>(*mutable_result_column).applyNullMap(static_cast<const ColumnNullable &>(*arg_cond.column));
|
||||
block.getByPosition(result).column = std::move(mutable_result_column);
|
||||
return true;
|
||||
@ -744,7 +744,8 @@ private:
|
||||
{
|
||||
if (arg_else.column->isColumnNullable())
|
||||
{
|
||||
auto result_column = arg_else.column->mutate();
|
||||
auto arg_else_column = arg_else.column;
|
||||
auto result_column = (*std::move(arg_else_column)).mutate();
|
||||
static_cast<ColumnNullable &>(*result_column).applyNullMap(static_cast<const ColumnUInt8 &>(*arg_cond.column));
|
||||
block.getByPosition(result).column = std::move(result_column);
|
||||
}
|
||||
@ -785,7 +786,8 @@ private:
|
||||
|
||||
if (arg_then.column->isColumnNullable())
|
||||
{
|
||||
auto result_column = arg_then.column->mutate();
|
||||
auto arg_then_column = arg_then.column;
|
||||
auto result_column = (*std::move(arg_then_column)).mutate();
|
||||
static_cast<ColumnNullable &>(*result_column).applyNegatedNullMap(static_cast<const ColumnUInt8 &>(*arg_cond.column));
|
||||
block.getByPosition(result).column = std::move(result_column);
|
||||
}
|
||||
|
@ -936,7 +936,7 @@ public:
|
||||
}
|
||||
|
||||
/// Put all the necessary columns multiplied by the sizes of arrays into the block.
|
||||
auto replicated_column_function_ptr = column_function->replicate(column_first_array->getOffsets());
|
||||
auto replicated_column_function_ptr = (*column_function->replicate(column_first_array->getOffsets())).mutate();
|
||||
auto * replicated_column_function = typeid_cast<ColumnFunction *>(replicated_column_function_ptr.get());
|
||||
replicated_column_function->appendArguments(arrays);
|
||||
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user