Build fixes (#3181)

This commit is contained in:
proller 2018-09-28 17:53:20 +03:00 committed by GitHub
parent 6f96dfc7da
commit ec11bef633
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 138 additions and 77 deletions

View File

@ -1,18 +1,11 @@
# Freebsd: /usr/local/include/libcpuid/libcpuid_types.h:61:29: error: conflicting declaration 'typedef long long int int64_t'
# TODO: test new libcpuid - maybe already fixed
if (NOT ARCH_ARM)
if (OS_FREEBSD)
set (DEFAULT_USE_INTERNAL_CPUID_LIBRARY 1)
else ()
set (DEFAULT_USE_INTERNAL_CPUID_LIBRARY ${NOT_UNBUNDLED})
endif ()
option (USE_INTERNAL_CPUID_LIBRARY "Set to FALSE to use system cpuid library instead of bundled" ${DEFAULT_USE_INTERNAL_CPUID_LIBRARY})
option (USE_INTERNAL_CPUID_LIBRARY "Set to FALSE to use system cpuid library instead of bundled" ${NOT_UNBUNDLED})
endif ()
#if (USE_INTERNAL_CPUID_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libcpuid/include/cpuid/libcpuid.h")
# message (WARNING "submodule contrib/libcpuid is missing. to fix try run: \n git submodule update --init --recursive")
# set (USE_INTERNAL_CPUID_LIBRARY 0)
# set (MISSING_INTERNAL_CPUID_LIBRARY 1)
#endif ()
if (NOT USE_INTERNAL_CPUID_LIBRARY)
@ -21,7 +14,13 @@ if (NOT USE_INTERNAL_CPUID_LIBRARY)
endif ()
if (CPUID_LIBRARY AND CPUID_INCLUDE_DIR)
else ()
if (OS_FREEBSD)
# need in /usr/local/include/libcpuid/libcpuid_types.h
# Freebsd: /usr/local/include/libcpuid/libcpuid_types.h:61:29: error: conflicting declaration 'typedef long long int int64_t'
add_definitions(-DHAVE_STDINT_H)
# TODO: make virtual target cpuid:cpuid with COMPILE_DEFINITIONS property
endif ()
elseif (NOT MISSING_INTERNAL_CPUID_LIBRARY)
set (CPUID_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libcpuid/include)
set (USE_INTERNAL_CPUID_LIBRARY 1)
set (CPUID_LIBRARY cpuid)

View File

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

View File

@ -14,11 +14,12 @@ cmake_push_check_state ()
#define __SSE4_1__ 1
set (TEST_FLAG "-msse4.1")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0")
check_cxx_source_compiles("
#include <smmintrin.h>
int main() {
_mm_insert_epi8(__m128i(), 0, 0);
auto a = _mm_insert_epi8(__m128i(), 0, 0);
(void)a;
return 0;
}
" HAVE_SSE41)
@ -31,11 +32,12 @@ endif ()
#define __SSE4_2__ 1
set (TEST_FLAG "-msse4.2")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0")
check_cxx_source_compiles("
#include <nmmintrin.h>
int main() {
_mm_crc32_u64(0, 0);
auto a = _mm_crc32_u64(0, 0);
(void)a;
return 0;
}
" HAVE_SSE42)
@ -49,10 +51,11 @@ endif ()
set (TEST_FLAG "-mpopcnt")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0")
check_cxx_source_compiles("
int main() {
__builtin_popcountll(0);
auto a = __builtin_popcountll(0);
(void)a;
return 0;
}
" HAVE_POPCNT)

View File

@ -142,6 +142,7 @@ if (USE_INTERNAL_POCO_LIBRARY)
set (save_CMAKE_C_FLAGS ${CMAKE_C_FLAGS})
set (_save ${ENABLE_TESTS})
set (ENABLE_TESTS 0)
set (POCO_ENABLE_TESTS 0)
set (CMAKE_DISABLE_FIND_PACKAGE_ZLIB 1)
if (MSVC)
set (ENABLE_DATA_ODBC 0 CACHE INTERNAL "") # TODO (build fail)

View File

@ -155,7 +155,7 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELW
PROPERTIES COMPILE_FLAGS -g0)
endif ()
if (NOT ARCH_ARM)
if (NOT ARCH_ARM AND CPUID_LIBRARY)
set (LINK_LIBRARIES_ONLY_ON_X86_64 ${CPUID_LIBRARY})
endif()

View File

@ -26,7 +26,7 @@ elseif (EXISTS ${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE})
set (COPY_HEADERS_COMPILER "${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE}")
endif ()
if (COPY_HEADERS_COMPILER AND NOT APPLE)
if (COPY_HEADERS_COMPILER AND OS_LINUX)
add_custom_target (copy-headers ALL env CLANG=${COPY_HEADERS_COMPILER} BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} ${ClickHouse_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS ${COPY_HEADERS_DEPENDS} WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES ${ClickHouse_SOURCE_DIR}/copy_headers.sh)
if (USE_INTERNAL_LLVM_LIBRARY)

View File

@ -2,9 +2,9 @@
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
#if USE_POCO_SQLODBC
#include <Poco/SQL/ODBC/ODBCException.h>
#include <Poco/SQL/ODBC/SessionImpl.h>
#include <Poco/SQL/ODBC/Utility.h>
#include <Poco/SQL/ODBC/ODBCException.h> // Y_IGNORE
#include <Poco/SQL/ODBC/SessionImpl.h> // Y_IGNORE
#include <Poco/SQL/ODBC/Utility.h> // Y_IGNORE
#define POCO_SQL_ODBC_CLASS Poco::SQL::ODBC
#endif
#if USE_POCO_DATAODBC

View File

@ -0,0 +1,41 @@
#include "TimeoutSetter.h"
#include <common/logger_useful.h>
namespace DB
{
TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_,
const Poco::Timespan & send_timeout_,
const Poco::Timespan & receive_timeout_,
bool limit_max_timeout)
: socket(socket_), send_timeout(send_timeout_), receive_timeout(receive_timeout_)
{
old_send_timeout = socket.getSendTimeout();
old_receive_timeout = socket.getReceiveTimeout();
if (!limit_max_timeout || old_send_timeout > send_timeout)
socket.setSendTimeout(send_timeout);
if (!limit_max_timeout || old_receive_timeout > receive_timeout)
socket.setReceiveTimeout(receive_timeout);
}
TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & timeout_, bool limit_max_timeout)
: TimeoutSetter(socket_, timeout_, timeout_, limit_max_timeout)
{
}
TimeoutSetter::~TimeoutSetter()
{
try
{
socket.setSendTimeout(old_send_timeout);
socket.setReceiveTimeout(old_receive_timeout);
}
catch (std::exception & e)
{
// Sometimes catched on macos
LOG_ERROR(&Logger::get("Client"), std::string{"TimeoutSetter: Can't reset timeouts: "} + e.what());
}
}
}

View File

@ -1,47 +1,30 @@
#pragma once
#include <Poco/Timespan.h>
#include <Poco/Net/StreamSocket.h>
#include <Poco/Timespan.h>
namespace DB
{
/// Temporarily overrides socket send/recieve timeouts and reset them back into destructor
/// Temporarily overrides socket send/receive timeouts and reset them back into destructor
/// If "limit_max_timeout" is true, timeouts could be only decreased (maxed by previous value).
struct TimeoutSetter
{
TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & send_timeout_, const Poco::Timespan & recieve_timeout_,
bool limit_max_timeout = false)
: socket(socket_), send_timeout(send_timeout_), recieve_timeout(recieve_timeout_)
{
old_send_timeout = socket.getSendTimeout();
old_receive_timeout = socket.getReceiveTimeout();
TimeoutSetter(Poco::Net::StreamSocket & socket_,
const Poco::Timespan & send_timeout_,
const Poco::Timespan & receive_timeout_,
bool limit_max_timeout = false);
if (!limit_max_timeout || old_send_timeout > send_timeout)
socket.setSendTimeout(send_timeout);
TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & timeout_, bool limit_max_timeout = false);
if (!limit_max_timeout || old_receive_timeout > recieve_timeout)
socket.setReceiveTimeout(recieve_timeout);
}
TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & timeout_, bool limit_max_timeout = false)
: TimeoutSetter(socket_, timeout_, timeout_, limit_max_timeout) {}
~TimeoutSetter()
{
socket.setSendTimeout(old_send_timeout);
socket.setReceiveTimeout(old_receive_timeout);
}
~TimeoutSetter();
Poco::Net::StreamSocket & socket;
Poco::Timespan send_timeout;
Poco::Timespan recieve_timeout;
Poco::Timespan receive_timeout;
Poco::Timespan old_send_timeout;
Poco::Timespan old_receive_timeout;
};
}

View File

@ -1027,7 +1027,7 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data)
read(err);
if (read_xid != auth_xid)
throw Exception("Unexpected event recieved in reply to auth request: " + toString(read_xid),
throw Exception("Unexpected event received in reply to auth request: " + toString(read_xid),
ZMARSHALLINGERROR);
int32_t actual_length = in->count() - count_before_event;

View File

@ -17,7 +17,7 @@ namespace ErrorCodes
* (lazy_sources contains not pointers themself, but their delayed constructors)
*
* Firtly, CascadeWriteBuffer redirects data to first buffer of the sequence
* If current WriteBuffer cannot recieve data anymore, it throws special exception CURRENT_WRITE_BUFFER_IS_EXHAUSTED in nextImpl() body,
* If current WriteBuffer cannot receive data anymore, it throws special exception CURRENT_WRITE_BUFFER_IS_EXHAUSTED in nextImpl() body,
* CascadeWriteBuffer prepare next buffer and continuously redirects data to it.
* If there are no buffers anymore CascadeWriteBuffer throws an exception.
*

View File

@ -424,7 +424,7 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task)
}
if (!task.query || !(task.query_on_cluster = dynamic_cast<ASTQueryWithOnCluster *>(task.query.get())))
throw Exception("Recieved unknown DDL query", ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
throw Exception("Received unknown DDL query", ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
task.cluster_name = task.query_on_cluster->cluster;
task.cluster = context.tryGetCluster(task.cluster_name);

View File

@ -1,2 +1,10 @@
if (CLICKHOUSE_SPLIT_BINARY)
set (TEST_USE_BINARIES CLICKHOUSE_TESTS_SERVER_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/clickhouse-server CLICKHOUSE_TESTS_CLIENT_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/clickhouse-client)
else()
set (TEST_USE_BINARIES CLICKHOUSE_TESTS_SERVER_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/clickhouse CLICKHOUSE_TESTS_CLIENT_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/clickhouse)
endif()
add_test(NAME integration WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND env CLICKHOUSE_TESTS_SERVER_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/server/clickhouse-server CLICKHOUSE_TESTS_CLIENT_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/server/clickhouse-client "CLICKHOUSE_TESTS_BASE_CONFIG_DIR=\${CLICKHOUSE_TESTS_BASE_CONFIG_DIR:=${ClickHouse_SOURCE_DIR}/dbms/programs/server/}" ${PYTEST_STARTER} pytest ${PYTEST_OPT})
# will mount only one binary to docker container - build with .so cant work
if (MAKE_STATIC_LIBRARIES)
add_test (NAME integration WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND env ${TEST_USE_BINARIES} "CLICKHOUSE_TESTS_BASE_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/dbms/programs/server/" ${PYTEST_STARTER} pytest ${PYTEST_OPT})
endif()

View File

@ -30,6 +30,16 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME):
f.write("=".join([var, value]) + "\n")
return full_path
def subprocess_check_call(args):
# Uncomment for debugging
# print('run:', ' ' . join(args))
subprocess.check_call(args)
def subprocess_call(args):
# Uncomment for debugging
# print('run:', ' ' . join(args))
subprocess.call(args)
class ClickHouseCluster:
"""ClickHouse cluster with several instances and (possibly) ZooKeeper.
@ -45,8 +55,8 @@ class ClickHouseCluster:
self.name = name if name is not None else ''
self.base_configs_dir = base_configs_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', '/etc/clickhouse-server/')
self.server_bin_path = server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse')
self.client_bin_path = client_bin_path or os.environ.get('CLICKHOUSE_TESTS_CLIENT_BIN_PATH', '/usr/bin/clickhouse-client')
self.server_bin_path = p.realpath(server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse'))
self.client_bin_path = p.realpath(client_bin_path or os.environ.get('CLICKHOUSE_TESTS_CLIENT_BIN_PATH', '/usr/bin/clickhouse-client'))
self.zookeeper_config_path = p.join(self.base_dir, zookeeper_config_path) if zookeeper_config_path else p.join(HELPERS_DIR, 'zookeeper_config.xml')
self.project_name = pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name
@ -179,8 +189,8 @@ class ClickHouseCluster:
# Just in case kill unstopped containers from previous launch
try:
if not subprocess.call(['docker-compose', 'kill']):
subprocess.call(['docker-compose', 'down', '--volumes'])
if not subprocess_call(['docker-compose', 'kill']):
subprocess_call(['docker-compose', 'down', '--volumes'])
except:
pass
@ -194,23 +204,20 @@ class ClickHouseCluster:
self.docker_client = docker.from_env(version=self.docker_api_version)
if self.with_zookeeper and self.base_zookeeper_cmd:
subprocess.check_call(self.base_zookeeper_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
subprocess_check_call(self.base_zookeeper_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
for command in self.pre_zookeeper_commands:
self.run_kazoo_commands_with_retries(command, repeats=5)
self.wait_zookeeper_to_start(120)
if self.with_mysql and self.base_mysql_cmd:
subprocess.check_call(self.base_mysql_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
subprocess_check_call(self.base_mysql_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
self.wait_mysql_to_start(120)
if self.with_kafka and self.base_kafka_cmd:
subprocess.check_call(self.base_kafka_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
subprocess_check_call(self.base_kafka_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
self.kafka_docker_id = self.get_instance_docker_id('kafka1')
# Uncomment for debugging
#print ' '.join(self.base_cmd + ['up', '--no-recreate'])
subprocess.check_call(self.base_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
subprocess_check_call(self.base_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
start_deadline = time.time() + 20.0 # seconds
for instance in self.instances.itervalues():
@ -226,8 +233,8 @@ class ClickHouseCluster:
def shutdown(self, kill=True):
if kill:
subprocess.check_call(self.base_cmd + ['kill'])
subprocess.check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans'])
subprocess_check_call(self.base_cmd + ['kill'])
subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans'])
self.is_up = False
self.docker_client = None
@ -468,8 +475,12 @@ class ClickHouseInstance:
shutil.copy(p.join(self.base_configs_dir, 'config.xml'), configs_dir)
shutil.copy(p.join(self.base_configs_dir, 'users.xml'), configs_dir)
# used by all utils with any config
conf_d_dir = p.abspath(p.join(configs_dir, 'conf.d'))
# used by server with main config.xml
config_d_dir = p.abspath(p.join(configs_dir, 'config.d'))
users_d_dir = p.abspath(p.join(configs_dir, 'users.d'))
os.mkdir(conf_d_dir)
os.mkdir(config_d_dir)
os.mkdir(users_d_dir)
@ -483,7 +494,7 @@ class ClickHouseInstance:
# Put ZooKeeper config
if self.with_zookeeper:
shutil.copy(self.zookeeper_config_path, config_d_dir)
shutil.copy(self.zookeeper_config_path, conf_d_dir)
# Copy config dir
if self.custom_config_dir:

View File

@ -58,4 +58,4 @@
</shard_0_0>
</remote_servers>
</yandex>
</yandex>

View File

@ -8,4 +8,4 @@
<stderr>/var/log/clickhouse-server/copier/stderr.log</stderr>
<stdout>/var/log/clickhouse-server/copier/stdout.log</stdout>
</logger>
</yandex>
</yandex>

2
debian/.pbuilderrc vendored
View File

@ -179,7 +179,7 @@ fi
# For killall in pbuilder-hooks:
EXTRAPACKAGES+=" psmisc "
[[ $CCACHE_PREFIX == 'distcc' ]] && EXTRAPACKAGES+=" $CCACHE_PREFIX "
[[ $CCACHE_PREFIX == 'distcc' ]] && EXTRAPACKAGES+=" $CCACHE_PREFIX " && USENETWORK=yes && export DISTCC_DIR=/var/cache/pbuilder/distcc
export DEB_BUILD_OPTIONS=parallel=`nproc`

View File

@ -7,9 +7,9 @@ echo "CCACHEDIR=$CCACHEDIR CCACHE_DIR=$CCACHE_DIR SET_CCACHEDIR=$SET_CCACHEDIR"
[ -z "$CCACHE_DIR" ] && export CCACHE_DIR=${CCACHEDIR:=${SET_CCACHEDIR=/var/cache/pbuilder/ccache}}
if [ -n "$CCACHE_DIR" ]; then
mkdir -p $CCACHE_DIR || true
chown -R $BUILDUSERID:$BUILDUSERID $CCACHE_DIR || true
chmod -R a+rwx $CCACHE_DIR || true
mkdir -p $CCACHE_DIR $DISTCC_DIR || true
chown -R $BUILDUSERID:$BUILDUSERID $CCACHE_DIR $DISTCC_DIR || true
chmod -R a+rwx $CCACHE_DIR $DISTCC_DIR || true
fi
df -h

View File

@ -2,4 +2,4 @@
# Try stop parallel build after timeout
killall make gcc gcc-7 g++-7 gcc-8 g++-8 clang-5.0 clang++-5.0 clang-6.0 clang++-6.0 clang-7 clang++-7 || true
killall make gcc gcc-7 g++-7 gcc-8 g++-8 clang clang-5.0 clang++-5.0 clang-6.0 clang++-6.0 clang-7 clang++-7 || true

12
debian/rules vendored
View File

@ -14,8 +14,10 @@ DESTDIR = $(CURDIR)/debian/tmp
DEB_HOST_MULTIARCH ?= $(shell dpkg-architecture -qDEB_HOST_MULTIARCH)
#TODO: why it not working? (maybe works in debhelper 10+)
ifndef THREADS_COUNT
ifeq ($(CCACHE_PREFIX),distcc)
THREADS_COUNT=$(shell distcc -j)
endif
ifeq ($(THREADS_COUNT),)
THREADS_COUNT=$(shell nproc || grep -c ^processor /proc/cpuinfo || sysctl -n hw.ncpu || echo 4)
endif
DEB_BUILD_OPTIONS+=parallel=$(THREADS_COUNT)
@ -73,6 +75,10 @@ else
THREADS_COUNT = 1
endif
ifneq ($(THREADS_COUNT),)
THREADS_COUNT:=-j$(THREADS_COUNT)
endif
%:
dh $@ $(DH_FLAGS) --buildsystem=cmake --builddirectory=$(BUILDDIR)
@ -81,7 +87,7 @@ override_dh_auto_configure:
override_dh_auto_build:
# Fix for ninja. Do not add -O.
cd $(BUILDDIR) && $(MAKE) -j$(THREADS_COUNT) $(MAKE_TARGET)
cd $(BUILDDIR) && $(MAKE) $(THREADS_COUNT) $(MAKE_TARGET)
#cd $(BUILDDIR) && cmake --build . -- -j$(THREADS_COUNT) # cmake return true on error
override_dh_auto_test:

View File

@ -12,11 +12,18 @@ elseif ()
endif()
if (ENABLE_JEMALLOC)
if (USE_INTERNAL_JEMALLOC_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/jemalloc/src/jemalloc.c")
message (WARNING "submodule contrib/jemalloc is missing. to fix try run: \n git submodule update --init --recursive")
set (USE_INTERNAL_JEMALLOC_LIBRARY 0)
set (MISSING_INTERNAL_JEMALLOC_LIBRARY 1)
endif ()
if (NOT USE_INTERNAL_JEMALLOC_LIBRARY)
find_package (JeMalloc)
endif ()
if (NOT JEMALLOC_LIBRARIES)
if (NOT JEMALLOC_LIBRARIES AND NOT MISSING_INTERNAL_JEMALLOC_LIBRARY)
set (JEMALLOC_LIBRARIES "jemalloc")
set (USE_INTERNAL_JEMALLOC_LIBRARY 1)
endif ()