Merge branch 'master' of github.com:yandex/ClickHouse into ubsan-fixes-4

This commit is contained in:
Alexey Milovidov 2018-12-28 00:23:13 +03:00
commit ab85175e7f
34 changed files with 274 additions and 73 deletions

View File

@ -25,11 +25,6 @@ endif ()
# Write compile_commands.json
set(CMAKE_EXPORT_COMPILE_COMMANDS 1)
set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "")
set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "")
include (cmake/limit_jobs.cmake)
include (cmake/find_ccache.cmake)
if (NOT CMAKE_BUILD_TYPE OR CMAKE_BUILD_TYPE STREQUAL "None")

View File

@ -8,23 +8,24 @@ endif ()
if (NOT ZLIB_FOUND)
if (NOT MSVC)
set (INTERNAL_ZLIB_NAME "zlib-ng")
set (INTERNAL_ZLIB_NAME "zlib-ng" CACHE INTERNAL "")
else ()
set (INTERNAL_ZLIB_NAME "zlib")
set (INTERNAL_ZLIB_NAME "zlib" CACHE INTERNAL "")
if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}")
message (WARNING "Will use standard zlib, please clone manually:\n git clone https://github.com/madler/zlib.git ${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}")
endif ()
endif ()
set (USE_INTERNAL_ZLIB_LIBRARY 1)
set (ZLIB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}" "${ClickHouse_BINARY_DIR}/contrib/${INTERNAL_ZLIB_NAME}") # generated zconf.h
set (ZLIB_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/${INTERNAL_ZLIB_NAME}" "${ClickHouse_BINARY_DIR}/contrib/${INTERNAL_ZLIB_NAME}" CACHE INTERNAL "") # generated zconf.h
set (ZLIB_INCLUDE_DIRS ${ZLIB_INCLUDE_DIR}) # for poco
set (ZLIB_INCLUDE_DIRECTORIES ${ZLIB_INCLUDE_DIR}) # for protobuf
set (ZLIB_FOUND 1) # for poco
if (USE_STATIC_LIBRARIES)
set (ZLIB_LIBRARIES zlibstatic)
set (ZLIB_LIBRARIES zlibstatic CACHE INTERNAL "")
else ()
set (ZLIB_LIBRARIES zlib)
set (ZLIB_LIBRARIES zlib CACHE INTERNAL "")
endif ()
endif ()
message (STATUS "Using zlib: ${ZLIB_INCLUDE_DIR} : ${ZLIB_LIBRARIES}")
message (STATUS "Using ${INTERNAL_ZLIB_NAME}: ${ZLIB_INCLUDE_DIR} : ${ZLIB_LIBRARIES}")

View File

@ -6,30 +6,32 @@
cmake_host_system_information(RESULT AVAILABLE_PHYSICAL_MEMORY QUERY AVAILABLE_PHYSICAL_MEMORY) # Not available under freebsd
option(PARALLEL_COMPILE_JOBS "Define the maximum number of concurrent compilation jobs" "")
if (NOT PARALLEL_COMPILE_JOBS AND AVAILABLE_PHYSICAL_MEMORY)
math(EXPR PARALLEL_COMPILE_JOBS ${AVAILABLE_PHYSICAL_MEMORY}/2500) # ~2.5gb max per one compiler
if (NOT PARALLEL_COMPILE_JOBS AND AVAILABLE_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY)
math(EXPR PARALLEL_COMPILE_JOBS ${AVAILABLE_PHYSICAL_MEMORY}/${MAX_COMPILER_MEMORY})
if (NOT PARALLEL_COMPILE_JOBS)
set (PARALLEL_COMPILE_JOBS 1)
endif ()
endif ()
if (PARALLEL_COMPILE_JOBS)
set_property(GLOBAL APPEND PROPERTY JOB_POOLS compile_job_pool=${PARALLEL_COMPILE_JOBS})
set(CMAKE_JOB_POOL_COMPILE compile_job_pool)
set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR})
string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE})
set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS})
endif ()
option(PARALLEL_LINK_JOBS "Define the maximum number of concurrent link jobs" "")
if (NOT PARALLEL_LINK_JOBS AND AVAILABLE_PHYSICAL_MEMORY)
math(EXPR PARALLEL_LINK_JOBS ${AVAILABLE_PHYSICAL_MEMORY}/4000) # ~4gb max per one linker
if (NOT PARALLEL_LINK_JOBS AND AVAILABLE_PHYSICAL_MEMORY AND MAX_LINKER_MEMORY)
math(EXPR PARALLEL_LINK_JOBS ${AVAILABLE_PHYSICAL_MEMORY}/${MAX_LINKER_MEMORY})
if (NOT PARALLEL_LINK_JOBS)
set (PARALLEL_LINK_JOBS 1)
endif ()
endif ()
if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS)
message(STATUS "Have ${AVAILABLE_PHYSICAL_MEMORY} megabytes of memory. Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS}")
message(STATUS "${CMAKE_CURRENT_SOURCE_DIR}: Have ${AVAILABLE_PHYSICAL_MEMORY} megabytes of memory. Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS}")
endif ()
if (LLVM_PARALLEL_LINK_JOBS)
set_property(GLOBAL APPEND PROPERTY JOB_POOLS link_job_pool=${PARALLEL_LINK_JOBS})
set(CMAKE_JOB_POOL_LINK link_job_pool)
set(CMAKE_JOB_POOL_LINK link_job_pool${CMAKE_CURRENT_SOURCE_DIR})
string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK})
set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_LINK}=${PARALLEL_LINK_JOBS})
endif ()

View File

@ -206,6 +206,7 @@ if (USE_INTERNAL_HDFS3_LIBRARY)
if (USE_INTERNAL_PROTOBUF_LIBRARY)
set(protobuf_BUILD_TESTS OFF CACHE INTERNAL "" FORCE)
set(protobuf_BUILD_SHARED_LIBS OFF CACHE INTERNAL "" FORCE)
set(protobuf_WITH_ZLIB 0 CACHE INTERNAL "" FORCE) # actually will use zlib, but skip find
add_subdirectory(protobuf/cmake)
endif ()
add_subdirectory(libhdfs3-cmake)

View File

@ -2,15 +2,23 @@ if (USE_INCLUDE_WHAT_YOU_USE)
set (CMAKE_CXX_INCLUDE_WHAT_YOU_USE ${IWYU_PATH})
endif ()
include(${CMAKE_CURRENT_SOURCE_DIR}/cmake/find_vectorclass.cmake)
set (MAX_COMPILER_MEMORY 2500 CACHE INTERNAL "")
if (MAKE_STATIC_LIBRARIES)
set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "")
else()
set (MAX_LINKER_MEMORY 2500 CACHE INTERNAL "")
endif ()
include (../cmake/limit_jobs.cmake)
include(cmake/find_vectorclass.cmake)
set (CONFIG_VERSION ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config_version.h)
set (CONFIG_COMMON ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config.h)
include (cmake/version.cmake)
message (STATUS "Will build ${VERSION_FULL}")
configure_file (${CMAKE_CURRENT_SOURCE_DIR}/src/Common/config.h.in ${CONFIG_COMMON})
configure_file (${CMAKE_CURRENT_SOURCE_DIR}/src/Common/config_version.h.in ${CONFIG_VERSION})
configure_file (src/Common/config.h.in ${CONFIG_COMMON})
configure_file (src/Common/config_version.h.in ${CONFIG_VERSION})
if (NOT MSVC)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra")
@ -53,7 +61,7 @@ add_subdirectory (src)
set(dbms_headers)
set(dbms_sources)
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
include(../cmake/dbms_glob_sources.cmake)
add_headers_and_sources(clickhouse_common_io src/Common)
add_headers_and_sources(clickhouse_common_io src/Common/HashTable)

View File

@ -666,6 +666,12 @@ private:
const bool test_mode = config().has("testmode");
if (config().has("multiquery"))
{
{ /// disable logs if expects errors
TestHint test_hint(test_mode, text);
if (test_hint.clientError() || test_hint.serverError())
process("SET send_logs_level = 'none'");
}
/// Several queries separated by ';'.
/// INSERT data is ended by the end of line, not ';'.

View File

@ -5,6 +5,7 @@
#include <iostream>
#include <Core/Types.h>
#include <Common/Exception.h>
#include <Parsers/Lexer.h>
namespace DB
@ -27,28 +28,29 @@ public:
if (!enabled_)
return;
/// TODO: This is absolutely wrong. Fragment may be contained inside string literal.
size_t pos = query.find("--");
String full_comment;
Lexer lexer(query.data(), query.data() + query.size());
if (pos != String::npos && query.find("--", pos + 2) != String::npos)
return; /// It's not last comment. Hint belongs to commented query. /// TODO Absolutely wrong: there maybe the following comment for the next query.
if (pos != String::npos)
for (Token token = lexer.nextToken(); !token.isEnd(); token = lexer.nextToken())
{
/// TODO: This is also wrong. Comment may already have ended by line break.
pos = query.find('{', pos + 2);
if (token.type == TokenType::Comment)
full_comment += String(token.begin, token.begin + token.size()) + ' ';
}
if (pos != String::npos)
if (!full_comment.empty())
{
String hint = query.substr(pos + 1);
/// TODO: And this is wrong for the same reason.
pos = hint.find('}');
hint.resize(pos);
size_t pos_start = full_comment.find('{', 0);
if (pos_start != String::npos)
{
size_t pos_end = full_comment.find('}', pos_start);
if (pos_end != String::npos)
{
String hint(full_comment.begin() + pos_start + 1, full_comment.begin() + pos_end);
parse(hint);
}
}
}
}
/// @returns true if it's possible to continue without reconnect
bool checkActual(int & actual_server_error, int & actual_client_error,

View File

@ -447,6 +447,11 @@ XMLDocumentPtr ConfigProcessor::processConfig(
merge(config, with);
contributing_files.push_back(merge_file);
}
catch (Exception & e)
{
e.addMessage("while merging config '" + path + "' with '" + merge_file + "'");
throw;
}
catch (Poco::Exception & e)
{
throw Poco::Exception("Failed to merge config with '" + merge_file + "': " + e.displayText());
@ -479,6 +484,11 @@ XMLDocumentPtr ConfigProcessor::processConfig(
doIncludesRecursive(config, include_from, getRootNode(config.get()), zk_node_cache, zk_changed_event, contributing_zk_paths);
}
catch (Exception & e)
{
e.addMessage("while preprocessing config '" + path + "'");
throw;
}
catch (Poco::Exception & e)
{
throw Poco::Exception("Failed to preprocess config '" + path + "': " + e.displayText(), e);

View File

@ -81,7 +81,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
std::lock_guard<std::mutex> lock(reload_mutex);
FilesChangesTracker new_files = getNewFileList();
if (force || new_files.isDifferOrNewerThan(files))
if (force || need_reload_from_zk || new_files.isDifferOrNewerThan(files))
{
ConfigProcessor config_processor(path);
ConfigProcessor::LoadedConfig loaded_config;
@ -94,6 +94,17 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
loaded_config = config_processor.loadConfigWithZooKeeperIncludes(
zk_node_cache, zk_changed_event, fallback_to_preprocessed);
}
catch (const Coordination::Exception & e)
{
if (Coordination::isHardwareError(e.code))
need_reload_from_zk = true;
if (throw_on_error)
throw;
tryLogCurrentException(log, "ZooKeeper error when loading config from `" + path + "'");
return;
}
catch (...)
{
if (throw_on_error)
@ -110,7 +121,10 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac
* When file has been written (and contain valid data), we don't load new data since modification time remains the same.
*/
if (!loaded_config.loaded_from_preprocessed)
{
files = std::move(new_files);
need_reload_from_zk = false;
}
try
{

View File

@ -75,6 +75,7 @@ private:
std::string preprocessed_dir;
FilesChangesTracker files;
zkutil::ZooKeeperNodeCache zk_node_cache;
bool need_reload_from_zk = false;
zkutil::EventPtr zk_changed_event = std::make_shared<Poco::Event>();
Updater updater;

View File

@ -7,6 +7,7 @@
#include <Poco/URI.h>
#include <hdfs/hdfs.h>
#include <IO/BufferWithOwnMemory.h>
#include <string>
#ifndef O_DIRECT
#define O_DIRECT 00040000
@ -52,7 +53,7 @@ namespace DB
if (fs == nullptr)
{
throw Exception("Unable to connect to HDFS: " + String(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR);
throw Exception("Unable to connect to HDFS: " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR);
}
fin = hdfsOpenFile(fs, path.c_str(), O_RDONLY, 0, 0, 0);
@ -77,7 +78,7 @@ namespace DB
int bytes_read = hdfsRead(fs, fin, internal_buffer.begin(), internal_buffer.size());
if (bytes_read < 0)
{
throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + String(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR);
throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR);
}
if (bytes_read)

View File

@ -70,18 +70,20 @@ public:
else
{
String addr(str, 0, pos - str.c_str());
mask_address = toIPv6(Poco::Net::IPAddress(addr));
auto real_address = Poco::Net::IPAddress(addr);
String str_mask(str, addr.length() + 1, str.length() - addr.length() - 1);
if (isDigits(str_mask))
{
UInt8 prefix_bits = parse<UInt8>(pos + 1);
construct(prefix_bits);
construct(prefix_bits, real_address.family() == Poco::Net::AddressFamily::IPv4);
}
else
{
subnet_mask = netmaskToIPv6(Poco::Net::IPAddress(str_mask));
}
mask_address = toIPv6(real_address);
}
}
@ -97,9 +99,9 @@ private:
subnet_mask = Poco::Net::IPAddress(128, Poco::Net::IPAddress::IPv6);
}
void construct(UInt8 prefix_bits)
void construct(UInt8 prefix_bits, bool is_ipv4)
{
prefix_bits = mask_address.family() == Poco::Net::IPAddress::IPv4 ? prefix_bits + 96 : prefix_bits;
prefix_bits = is_ipv4 ? prefix_bits + 96 : prefix_bits;
subnet_mask = Poco::Net::IPAddress(prefix_bits, Poco::Net::IPAddress::IPv6);
}

View File

@ -97,7 +97,7 @@ class ClickHouseCluster:
cmd += " client"
return cmd
def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False):
def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False, ipv4_address=None, ipv6_address=None):
"""Add an instance to the cluster.
name - the name of the instance directory and the value of the 'instance' macro in ClickHouse.
@ -116,7 +116,8 @@ class ClickHouseCluster:
instance = ClickHouseInstance(
self, self.base_dir, name, config_dir, main_configs, user_configs, macros, with_zookeeper,
self.zookeeper_config_path, with_mysql, with_kafka, self.base_configs_dir, self.server_bin_path,
clickhouse_path_dir, with_odbc_drivers, hostname=hostname, env_variables=env_variables, image=image, stay_alive=stay_alive)
clickhouse_path_dir, with_odbc_drivers, hostname=hostname, env_variables=env_variables, image=image,
stay_alive=stay_alive, ipv4_address=ipv4_address, ipv6_address=ipv6_address)
self.instances[name] = instance
self.base_cmd.extend(['--file', instance.docker_compose_path])
@ -332,7 +333,7 @@ CLICKHOUSE_START_COMMAND = "clickhouse server --config-file=/etc/clickhouse-serv
CLICKHOUSE_STAY_ALIVE_COMMAND = 'bash -c "{} --daemon; tail -f /dev/null"'.format(CLICKHOUSE_START_COMMAND)
DOCKER_COMPOSE_TEMPLATE = '''
version: '2'
version: '2.2'
services:
{name}:
image: {image}
@ -344,9 +345,27 @@ services:
- {logs_dir}:/var/log/clickhouse-server/
{odbc_ini_path}
entrypoint: {entrypoint_cmd}
cap_add:
- SYS_PTRACE
depends_on: {depends_on}
env_file:
- {env_file}
{networks}
{app_net}
{ipv4_address}
{ipv6_address}
networks:
app_net:
driver: bridge
enable_ipv6: true
ipam:
driver: default
config:
- subnet: 10.5.0.0/12
gateway: 10.5.1.1
- subnet: 2001:3984:3989::/64
gateway: 2001:3984:3989::1
'''
class ClickHouseInstance:
@ -354,7 +373,8 @@ class ClickHouseInstance:
def __init__(
self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros,
with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, base_configs_dir, server_bin_path,
clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False):
clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test",
stay_alive=False, ipv4_address=None, ipv6_address=None):
self.name = name
self.base_cmd = cluster.base_cmd[:]
@ -391,6 +411,8 @@ class ClickHouseInstance:
self.default_timeout = 20.0 # 20 sec
self.image = image
self.stay_alive = stay_alive
self.ipv4_address = ipv4_address
self.ipv6_address = ipv6_address
# Connects to the instance via clickhouse-client, sends a query (1st argument) and returns the answer
def query(self, sql, stdin=None, timeout=None, settings=None, user=None, ignore_error=False):
@ -609,6 +631,18 @@ class ClickHouseInstance:
if self.stay_alive:
entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND
ipv4_address = ipv6_address = ""
if self.ipv4_address is None and self.ipv6_address is None:
networks = ""
app_net = ""
else:
networks = "networks:"
app_net = "app_net:"
if self.ipv4_address is not None:
ipv4_address = "ipv4_address: " + self.ipv4_address
if self.ipv6_address is not None:
ipv6_address = "ipv6_address: " + self.ipv6_address
with open(self.docker_compose_path, 'w') as docker_compose:
docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format(
image=self.image,
@ -623,6 +657,10 @@ class ClickHouseInstance:
env_file=env_file,
odbc_ini_path=odbc_ini_path,
entrypoint_cmd=entrypoint_cmd,
networks=networks,
app_net=app_net,
ipv4_address=ipv4_address,
ipv6_address=ipv6_address,
))

View File

@ -1,4 +1,4 @@
version: '2'
version: '2.2'
services:
hdfs1:
image: sequenceiq/hadoop-docker:2.7.0

View File

@ -1,4 +1,4 @@
version: '2'
version: '2.2'
services:
kafka_zookeeper:

View File

@ -1,4 +1,4 @@
version: '2'
version: '2.2'
services:
mysql1:
image: mysql:5.7

View File

@ -1,4 +1,4 @@
version: '2'
version: '2.2'
services:
postgres1:
image: postgres

View File

@ -1,4 +1,4 @@
version: '2'
version: '2.2'
services:
zoo1:
image: zookeeper

View File

@ -0,0 +1 @@
<yandex><listen_host>::</listen_host></yandex>

View File

@ -0,0 +1,27 @@
<yandex>
<profiles>
<default>
<max_memory_usage>10000000000</max_memory_usage>
<use_uncompressed_cache>0</use_uncompressed_cache>
<load_balancing>random</load_balancing>
</default>
</profiles>
<users>
<default>
<password></password>
<networks incl="networks" replace="replace">
<ip>10.5.173.1</ip>
<ip>10.5.172.0/24</ip>
<ip>10.5.175.0/255.255.255.0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</default>
</users>
<quotas>
<default>
</default>
</quotas>
</yandex>

View File

@ -0,0 +1,26 @@
<yandex>
<profiles>
<default>
<max_memory_usage>10000000000</max_memory_usage>
<use_uncompressed_cache>0</use_uncompressed_cache>
<load_balancing>random</load_balancing>
</default>
</profiles>
<users>
<default>
<password></password>
<networks incl="networks" replace="replace">
<ip>2001:3984:3989:0:0:0:1:1111</ip>
<ip>2001:3984:3989:0:0:0:0:0/112</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</default>
</users>
<quotas>
<default>
</default>
</quotas>
</yandex>

View File

@ -0,0 +1,74 @@
import time
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
node_ipv4 = cluster.add_instance('node_ipv4', config_dir="configs", user_configs=['configs/users_ipv4.xml'], ipv4_address='10.5.172.77')
client_ipv4_ok = cluster.add_instance('client_ipv4_ok', config_dir="configs", ipv4_address='10.5.172.10')
client_ipv4_ok_direct = cluster.add_instance('client_ipv4_ok_direct', config_dir="configs", ipv4_address='10.5.173.1')
client_ipv4_ok_full_mask = cluster.add_instance('client_ipv4_ok_full_mask', config_dir="configs", ipv4_address='10.5.175.77')
client_ipv4_bad = cluster.add_instance('client_ipv4_bad', config_dir="configs", ipv4_address='10.5.173.10')
node_ipv6 = cluster.add_instance('node_ipv6', config_dir="configs", main_configs=["configs/config_ipv6.xml"], user_configs=['configs/users_ipv6.xml'], ipv6_address='2001:3984:3989::1:1000')
client_ipv6_ok = cluster.add_instance('client_ipv6_ok', config_dir="configs", ipv6_address='2001:3984:3989::5555')
client_ipv6_ok_direct = cluster.add_instance('client_ipv6_ok_direct', config_dir="configs", ipv6_address='2001:3984:3989::1:1111')
client_ipv6_bad = cluster.add_instance('client_ipv6_bad', config_dir="configs", ipv6_address='2001:3984:3989::1:1112')
@pytest.fixture(scope="module")
def setup_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_ipv4(setup_cluster):
try:
client_ipv4_ok.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 10.5.172.77 --query 'select 1'"], privileged=True, user='root')
except Exception as ex:
assert False, "allowed client with 10.5.172.10 cannot connect to server with allowed mask '10.5.172.0/24'"
try:
client_ipv4_ok_direct.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 10.5.172.77 --query 'select 1'"], privileged=True, user='root')
except Exception as ex:
assert False, "allowed client with 10.5.173.1 cannot connect to server with allowed ip '10.5.173.1'"
try:
client_ipv4_ok_full_mask.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 10.5.172.77 --query 'select 1'"], privileged=True, user='root')
except Exception as ex:
assert False, "allowed client with 10.5.175.77 cannot connect to server with allowed ip '10.5.175.0/255.255.255.0'"
try:
client_ipv4_bad.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 10.5.172.77 --query 'select 1'"], privileged=True, user='root')
assert False, "restricted client with 10.5.173.10 can connect to server with allowed mask '10.5.172.0/24'"
except AssertionError:
raise
except Exception as ex:
print ex
def test_ipv6(setup_cluster):
try:
client_ipv6_ok.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 2001:3984:3989::1:1000 --query 'select 1'"], privileged=True, user='root')
except Exception as ex:
print ex
assert False, "allowed client with 2001:3984:3989:0:0:0:1:1111 cannot connect to server with allowed mask '2001:3984:3989:0:0:0:0:0/112'"
try:
client_ipv6_ok_direct.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 2001:3984:3989:0:0:0:1:1000 --query 'select 1'"], privileged=True, user='root')
except Exception as ex:
assert False, "allowed client with 2001:3984:3989:0:0:0:1:1111 cannot connect to server with allowed ip '2001:3984:3989:0:0:0:1:1111'"
try:
client_ipv6_bad.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host 2001:3984:3989:0:0:0:1:1000 --query 'select 1'"], privileged=True, user='root')
assert False, "restricted client with 2001:3984:3989:0:0:0:1:1112 can connect to server with allowed mask '2001:3984:3989:0:0:0:0:0/112'"
except AssertionError:
raise
except Exception as ex:
print ex

View File

@ -1,5 +1,3 @@
SET send_logs_level = 'none';
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.decimal;

View File

@ -1,5 +1,3 @@
SET send_logs_level = 'none';
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.decimal;

View File

@ -1,5 +1,3 @@
SET send_logs_level = 'none';
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.decimal;

View File

@ -1,5 +1,3 @@
SET send_logs_level = 'none';
SELECT toDecimal32('1.1', 1), toDecimal32('1.1', 2), toDecimal32('1.1', 8);
SELECT toDecimal32('1.1', 0); -- { serverError 69 }
SELECT toDecimal32(1.1, 0), toDecimal32(1.1, 1), toDecimal32(1.1, 2), toDecimal32(1.1, 8);

View File

@ -1,5 +1,3 @@
SET send_logs_level = 'none';
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.decimal;

View File

@ -1,5 +1,3 @@
SET send_logs_level = 'none';
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.decimal;

View File

@ -1,5 +1,3 @@
SET send_logs_level = 'none';
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.decimal;

View File

@ -1,5 +1,3 @@
SET send_logs_level = 'none';
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log(x), 4) AS y, exp(y);
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log2(x), 4) AS y, exp2(y);
SELECT toDecimal32('42.42', 4) AS x, toDecimal32(log10(x), 4) AS y, exp10(y);

View File

@ -1,5 +1,3 @@
SET send_logs_level = 'none';
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.decimal;

View File

@ -3,6 +3,13 @@ if (NOT NO_WERROR)
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Werror")
endif ()
if (MAKE_STATIC_LIBRARIES)
set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "")
else()
set (MAX_LINKER_MEMORY 2500 CACHE INTERNAL "")
endif ()
include (../cmake/limit_jobs.cmake)
# Utils used in package
add_subdirectory (config-processor)
add_subdirectory (report)

View File

@ -52,6 +52,9 @@ inc="-I. \
-I./libs/libdaemon/include \
-I./libs/consistent-hashing \
-I./libs/consistent-hashing-sumbur \
-I./contrib/libhdfs3/include \
-I./contrib/base64/include \
-I./contrib/lz4/lib \
-I./dbms/src \
-I${BUILD_DIR}/dbms/src"