Merge remote-tracking branch 'upstream/master' into query-poor-mans-profiler

This commit is contained in:
Nikita Lapkov 2019-05-24 13:28:42 +00:00
commit 30c31bc85a
41 changed files with 385 additions and 177 deletions

View File

@ -5,7 +5,7 @@ endmacro()
macro(add_headers_and_sources prefix common_path) macro(add_headers_and_sources prefix common_path)
add_glob(${prefix}_headers RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h) add_glob(${prefix}_headers RELATIVE ${CMAKE_CURRENT_SOURCE_DIR} ${common_path}/*.h)
add_glob(${prefix}_sources ${common_path}/*.cpp ${common_path}/*.h) add_glob(${prefix}_sources ${common_path}/*.cpp ${common_path}/*.c ${common_path}/*.h)
endmacro() endmacro()
macro(add_headers_only prefix common_path) macro(add_headers_only prefix common_path)

View File

@ -26,8 +26,8 @@ elseif (EXISTS ${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE})
set (COPY_HEADERS_COMPILER "${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE}") set (COPY_HEADERS_COMPILER "${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE}")
endif () endif ()
if (COPY_HEADERS_COMPILER AND OS_LINUX) if (COPY_HEADERS_COMPILER)
add_custom_target (copy-headers [ -f ${TMP_HEADERS_DIR}/dbms/src/Interpreters/SpecializedAggregator.h ] || env CLANG=${COPY_HEADERS_COMPILER} BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS ${COPY_HEADERS_DEPENDS} WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES copy_headers.sh) add_custom_target (copy-headers [ -f ${TMP_HEADERS_DIR}/dbms/src/Interpreters/SpecializedAggregator.h ] || env CLANG=${COPY_HEADERS_COMPILER} BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} CMAKE_CXX_COMPILER_VERSION=${CMAKE_CXX_COMPILER_VERSION} ${CMAKE_CURRENT_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS ${COPY_HEADERS_DEPENDS} WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES copy_headers.sh)
if (USE_INTERNAL_LLVM_LIBRARY) if (USE_INTERNAL_LLVM_LIBRARY)
set (CLANG_HEADERS_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm/clang/lib/Headers") set (CLANG_HEADERS_DIR "${ClickHouse_SOURCE_DIR}/contrib/llvm/clang/lib/Headers")

View File

@ -38,26 +38,28 @@ for header in $START_HEADERS; do
START_HEADERS_INCLUDE+="-include $header " START_HEADERS_INCLUDE+="-include $header "
done done
# Опция -mcx16 для того, чтобы выбиралось больше заголовочных файлов (с запасом).
# The latter options are the same that are added while building packages.
# TODO: Does not work on macos:
GCC_ROOT=`$CLANG -v 2>&1 | grep "Selected GCC installation"| sed -n -e 's/^.*: //p'` GCC_ROOT=`$CLANG -v 2>&1 | grep "Selected GCC installation"| sed -n -e 's/^.*: //p'`
for src_file in $(echo | $CLANG -M -xc++ -std=c++1z -Wall -Werror -msse4 -mcx16 -mpopcnt -O3 -g -fPIC -fstack-protector -D_FORTIFY_SOURCE=2 \ # TODO: Does not work on macos?
GCC_ROOT=${GCC_ROOT:=/usr/lib/clang/${CMAKE_CXX_COMPILER_VERSION}}
# Опция -mcx16 для того, чтобы выбиралось больше заголовочных файлов (с запасом).
# The latter options are the same that are added while building packages.
for src_file in $(echo | $CLANG -M -xc++ -std=c++1z -Wall -Werror -msse2 -msse4 -mcx16 -mpopcnt -O3 -g -fPIC -fstack-protector -D_FORTIFY_SOURCE=2 \
-I $GCC_ROOT/include \ -I $GCC_ROOT/include \
-I $GCC_ROOT/include-fixed \ -I $GCC_ROOT/include-fixed \
$(cat "$BUILD_PATH/include_directories.txt") \ $(cat "$BUILD_PATH/include_directories.txt") \
$START_HEADERS_INCLUDE \ $START_HEADERS_INCLUDE \
- | - |
tr -d '\\' | tr -d '\\' |
sed --posix -E -e 's/^-\.o://'); sed -E -e 's/^-\.o://');
do do
dst_file=$src_file; dst_file=$src_file;
[ -n $BUILD_PATH ] && dst_file=$(echo $dst_file | sed --posix -E -e "s!^$BUILD_PATH!!") [ -n $BUILD_PATH ] && dst_file=$(echo $dst_file | sed -E -e "s!^$BUILD_PATH!!")
[ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed --posix -E -e "s!^$DESTDIR!!") [ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed -E -e "s!^$DESTDIR!!")
dst_file=$(echo $dst_file | sed --posix -E -e 's/build\///') # for simplicity reasons, will put generated headers near the rest. dst_file=$(echo $dst_file | sed -E -e 's/build\///') # for simplicity reasons, will put generated headers near the rest.
mkdir -p "$DST/$(echo $dst_file | sed --posix -E -e 's/\/[^/]*$/\//')"; mkdir -p "$DST/$(echo $dst_file | sed -E -e 's/\/[^/]*$/\//')";
cp "$src_file" "$DST/$dst_file"; cp "$src_file" "$DST/$dst_file";
done done
@ -68,9 +70,9 @@ done
for src_file in $(ls -1 $($CLANG -v -xc++ - <<<'' 2>&1 | grep '^ /' | grep 'include' | grep -E '/lib/clang/|/include/clang/')/*.h | grep -vE 'arm|altivec|Intrin'); for src_file in $(ls -1 $($CLANG -v -xc++ - <<<'' 2>&1 | grep '^ /' | grep 'include' | grep -E '/lib/clang/|/include/clang/')/*.h | grep -vE 'arm|altivec|Intrin');
do do
dst_file=$src_file; dst_file=$src_file;
[ -n $BUILD_PATH ] && dst_file=$(echo $dst_file | sed --posix -E -e "s!^$BUILD_PATH!!") [ -n $BUILD_PATH ] && dst_file=$(echo $dst_file | sed -E -e "s!^$BUILD_PATH!!")
[ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed --posix -E -e "s!^$DESTDIR!!") [ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed -E -e "s!^$DESTDIR!!")
mkdir -p "$DST/$(echo $dst_file | sed --posix -E -e 's/\/[^/]*$/\//')"; mkdir -p "$DST/$(echo $dst_file | sed -E -e 's/\/[^/]*$/\//')";
cp "$src_file" "$DST/$dst_file"; cp "$src_file" "$DST/$dst_file";
done done
@ -79,9 +81,9 @@ if [ -d "$SOURCE_PATH/contrib/boost/libs/smart_ptr/include/boost/smart_ptr/detai
for src_file in $(ls -1 $SOURCE_PATH/contrib/boost/libs/smart_ptr/include/boost/smart_ptr/detail/*); for src_file in $(ls -1 $SOURCE_PATH/contrib/boost/libs/smart_ptr/include/boost/smart_ptr/detail/*);
do do
dst_file=$src_file; dst_file=$src_file;
[ -n $BUILD_PATH ] && dst_file=$(echo $dst_file | sed --posix -E -e "s!^$BUILD_PATH!!") [ -n $BUILD_PATH ] && dst_file=$(echo $dst_file | sed -E -e "s!^$BUILD_PATH!!")
[ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed --posix -E -e "s!^$DESTDIR!!") [ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed -E -e "s!^$DESTDIR!!")
mkdir -p "$DST/$(echo $dst_file | sed --posix -E -e 's/\/[^/]*$/\//')"; mkdir -p "$DST/$(echo $dst_file | sed -E -e 's/\/[^/]*$/\//')";
cp "$src_file" "$DST/$dst_file"; cp "$src_file" "$DST/$dst_file";
done done
fi fi
@ -90,9 +92,9 @@ if [ -d "$SOURCE_PATH/contrib/boost/boost/smart_ptr/detail" ]; then
for src_file in $(ls -1 $SOURCE_PATH/contrib/boost/boost/smart_ptr/detail/*); for src_file in $(ls -1 $SOURCE_PATH/contrib/boost/boost/smart_ptr/detail/*);
do do
dst_file=$src_file; dst_file=$src_file;
[ -n $BUILD_PATH ] && dst_file=$(echo $dst_file | sed --posix -E -e "s!^$BUILD_PATH!!") [ -n $BUILD_PATH ] && dst_file=$(echo $dst_file | sed -E -e "s!^$BUILD_PATH!!")
[ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed --posix -E -e "s!^$DESTDIR!!") [ -n $DESTDIR ] && dst_file=$(echo $dst_file | sed -E -e "s!^$DESTDIR!!")
mkdir -p "$DST/$(echo $dst_file | sed --posix -E -e 's/\/[^/]*$/\//')"; mkdir -p "$DST/$(echo $dst_file | sed -E -e 's/\/[^/]*$/\//')";
cp "$src_file" "$DST/$dst_file"; cp "$src_file" "$DST/$dst_file";
done done
fi fi

View File

@ -159,17 +159,9 @@ UInt64 PerformanceTest::calculateMaxExecTime() const
void PerformanceTest::prepare() const void PerformanceTest::prepare() const
{ {
for (const auto & query : test_info.create_queries) for (const auto & query : test_info.create_and_fill_queries)
{ {
LOG_INFO(log, "Executing create query \"" << query << '\"'); LOG_INFO(log, "Executing create or fill query \"" << query << '\"');
connection.sendQuery(query, "", QueryProcessingStage::Complete, &test_info.settings, nullptr, false);
waitQuery(connection);
LOG_INFO(log, "Query finished");
}
for (const auto & query : test_info.fill_queries)
{
LOG_INFO(log, "Executing fill query \"" << query << '\"');
connection.sendQuery(query, "", QueryProcessingStage::Complete, &test_info.settings, nullptr, false); connection.sendQuery(query, "", QueryProcessingStage::Complete, &test_info.settings, nullptr, false);
waitQuery(connection); waitQuery(connection);
LOG_INFO(log, "Query finished"); LOG_INFO(log, "Query finished");

View File

@ -30,11 +30,6 @@ public:
std::vector<TestStats> execute(); std::vector<TestStats> execute();
void finish() const; void finish() const;
const PerformanceTestInfo & getTestInfo() const
{
return test_info;
}
bool checkSIGINT() const bool checkSIGINT() const
{ {
return got_SIGINT; return got_SIGINT;

View File

@ -60,10 +60,10 @@ PerformanceTestInfo::PerformanceTestInfo(
applySettings(config); applySettings(config);
extractQueries(config); extractQueries(config);
extractAuxiliaryQueries(config);
processSubstitutions(config); processSubstitutions(config);
getExecutionType(config); getExecutionType(config);
getStopConditions(config); getStopConditions(config);
extractAuxiliaryQueries(config);
} }
void PerformanceTestInfo::applySettings(XMLConfigurationPtr config) void PerformanceTestInfo::applySettings(XMLConfigurationPtr config)
@ -153,13 +153,29 @@ void PerformanceTestInfo::processSubstitutions(XMLConfigurationPtr config)
ConfigurationPtr substitutions_view(config->createView("substitutions")); ConfigurationPtr substitutions_view(config->createView("substitutions"));
constructSubstitutions(substitutions_view, substitutions); constructSubstitutions(substitutions_view, substitutions);
auto queries_pre_format = queries; auto create_and_fill_queries_preformat = create_and_fill_queries;
create_and_fill_queries.clear();
for (const auto & query : create_and_fill_queries_preformat)
{
auto formatted = formatQueries(query, substitutions);
create_and_fill_queries.insert(create_and_fill_queries.end(), formatted.begin(), formatted.end());
}
auto queries_preformat = queries;
queries.clear(); queries.clear();
for (const auto & query : queries_pre_format) for (const auto & query : queries_preformat)
{ {
auto formatted = formatQueries(query, substitutions); auto formatted = formatQueries(query, substitutions);
queries.insert(queries.end(), formatted.begin(), formatted.end()); queries.insert(queries.end(), formatted.begin(), formatted.end());
} }
auto drop_queries_preformat = drop_queries;
drop_queries.clear();
for (const auto & query : drop_queries_preformat)
{
auto formatted = formatQueries(query, substitutions);
drop_queries.insert(drop_queries.end(), formatted.begin(), formatted.end());
}
} }
} }
@ -203,13 +219,20 @@ void PerformanceTestInfo::getStopConditions(XMLConfigurationPtr config)
void PerformanceTestInfo::extractAuxiliaryQueries(XMLConfigurationPtr config) void PerformanceTestInfo::extractAuxiliaryQueries(XMLConfigurationPtr config)
{ {
if (config->has("create_query")) if (config->has("create_query"))
create_queries = getMultipleValuesFromConfig(*config, "", "create_query"); {
create_and_fill_queries = getMultipleValuesFromConfig(*config, "", "create_query");
}
if (config->has("fill_query")) if (config->has("fill_query"))
fill_queries = getMultipleValuesFromConfig(*config, "", "fill_query"); {
auto fill_queries = getMultipleValuesFromConfig(*config, "", "fill_query");
create_and_fill_queries.insert(create_and_fill_queries.end(), fill_queries.begin(), fill_queries.end());
}
if (config->has("drop_query")) if (config->has("drop_query"))
{
drop_queries = getMultipleValuesFromConfig(*config, "", "drop_query"); drop_queries = getMultipleValuesFromConfig(*config, "", "drop_query");
} }
}
} }

View File

@ -42,8 +42,7 @@ public:
std::vector<TestStopConditions> stop_conditions_by_run; std::vector<TestStopConditions> stop_conditions_by_run;
Strings create_queries; Strings create_and_fill_queries;
Strings fill_queries;
Strings drop_queries; Strings drop_queries;
private: private:
@ -52,7 +51,6 @@ private:
void processSubstitutions(XMLConfigurationPtr config); void processSubstitutions(XMLConfigurationPtr config);
void getExecutionType(XMLConfigurationPtr config); void getExecutionType(XMLConfigurationPtr config);
void getStopConditions(XMLConfigurationPtr config); void getStopConditions(XMLConfigurationPtr config);
void getMetrics(XMLConfigurationPtr config);
void extractAuxiliaryQueries(XMLConfigurationPtr config); void extractAuxiliaryQueries(XMLConfigurationPtr config);
}; };

View File

@ -202,8 +202,7 @@ private:
LOG_INFO(log, "Preconditions for test '" << info.test_name << "' are fullfilled"); LOG_INFO(log, "Preconditions for test '" << info.test_name << "' are fullfilled");
LOG_INFO( LOG_INFO(
log, log,
"Preparing for run, have " << info.create_queries.size() << " create queries and " << info.fill_queries.size() "Preparing for run, have " << info.create_and_fill_queries.size() << " create and fill queries");
<< " fill queries");
current.prepare(); current.prepare();
LOG_INFO(log, "Prepared"); LOG_INFO(log, "Prepared");
LOG_INFO(log, "Running test '" << info.test_name << "'"); LOG_INFO(log, "Running test '" << info.test_name << "'");

View File

@ -110,8 +110,8 @@ namespace
void registerAggregateFunctionMLMethod(AggregateFunctionFactory & factory) void registerAggregateFunctionMLMethod(AggregateFunctionFactory & factory)
{ {
factory.registerFunction("LinearRegression", createAggregateFunctionMLMethod<FuncLinearRegression>); factory.registerFunction("linearRegression", createAggregateFunctionMLMethod<FuncLinearRegression>);
factory.registerFunction("LogisticRegression", createAggregateFunctionMLMethod<FuncLogisticRegression>); factory.registerFunction("logisticRegression", createAggregateFunctionMLMethod<FuncLogisticRegression>);
} }
LinearModelData::LinearModelData( LinearModelData::LinearModelData(

View File

@ -321,10 +321,10 @@ private:
struct NameLinearRegression struct NameLinearRegression
{ {
static constexpr auto name = "LinearRegression"; static constexpr auto name = "linearRegression";
}; };
struct NameLogisticRegression struct NameLogisticRegression
{ {
static constexpr auto name = "LogisticRegression"; static constexpr auto name = "logisticRegression";
}; };
} }

View File

@ -1,8 +1,9 @@
#include <AggregateFunctions/AggregateFunctionLeastSqr.h> #include <AggregateFunctions/AggregateFunctionSimpleLinearRegression.h>
#include <AggregateFunctions/AggregateFunctionFactory.h> #include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/FactoryHelpers.h> #include <AggregateFunctions/FactoryHelpers.h>
#include <Core/TypeListNumber.h>
namespace DB namespace DB
{ {
@ -10,7 +11,7 @@ namespace DB
namespace namespace
{ {
AggregateFunctionPtr createAggregateFunctionLeastSqr( AggregateFunctionPtr createAggregateFunctionSimpleLinearRegression(
const String & name, const String & name,
const DataTypes & arguments, const DataTypes & arguments,
const Array & params const Array & params
@ -20,16 +21,11 @@ AggregateFunctionPtr createAggregateFunctionLeastSqr(
assertBinary(name, arguments); assertBinary(name, arguments);
const IDataType * x_arg = arguments.front().get(); const IDataType * x_arg = arguments.front().get();
WhichDataType which_x = x_arg;
WhichDataType which_x {
x_arg
};
const IDataType * y_arg = arguments.back().get(); const IDataType * y_arg = arguments.back().get();
WhichDataType which_y = y_arg;
WhichDataType which_y {
y_arg
};
#define FOR_LEASTSQR_TYPES_2(M, T) \ #define FOR_LEASTSQR_TYPES_2(M, T) \
M(T, UInt8) \ M(T, UInt8) \
@ -55,7 +51,7 @@ AggregateFunctionPtr createAggregateFunctionLeastSqr(
FOR_LEASTSQR_TYPES_2(M, Float64) FOR_LEASTSQR_TYPES_2(M, Float64)
#define DISPATCH(T1, T2) \ #define DISPATCH(T1, T2) \
if (which_x.idx == TypeIndex::T1 && which_y.idx == TypeIndex::T2) \ if (which_x.idx == TypeIndex::T1 && which_y.idx == TypeIndex::T2) \
return std::make_shared<AggregateFunctionLeastSqr<T1, T2>>( \ return std::make_shared<AggregateFunctionSimpleLinearRegression<T1, T2>>( \
arguments, \ arguments, \
params \ params \
); );
@ -77,9 +73,9 @@ AggregateFunctionPtr createAggregateFunctionLeastSqr(
} }
void registerAggregateFunctionLeastSqr(AggregateFunctionFactory & factory) void registerAggregateFunctionSimpleLinearRegression(AggregateFunctionFactory & factory)
{ {
factory.registerFunction("leastSqr", createAggregateFunctionLeastSqr); factory.registerFunction("simpleLinearRegression", createAggregateFunctionSimpleLinearRegression);
} }
} }

View File

@ -19,7 +19,7 @@ namespace ErrorCodes
} }
template <typename X, typename Y, typename Ret> template <typename X, typename Y, typename Ret>
struct AggregateFunctionLeastSqrData final struct AggregateFunctionSimpleLinearRegressionData final
{ {
size_t count = 0; size_t count = 0;
Ret sum_x = 0; Ret sum_x = 0;
@ -36,7 +36,7 @@ struct AggregateFunctionLeastSqrData final
sum_xy += x * y; sum_xy += x * y;
} }
void merge(const AggregateFunctionLeastSqrData & other) void merge(const AggregateFunctionSimpleLinearRegressionData & other)
{ {
count += other.count; count += other.count;
sum_x += other.sum_x; sum_x += other.sum_x;
@ -85,19 +85,19 @@ struct AggregateFunctionLeastSqrData final
/// Calculates simple linear regression parameters. /// Calculates simple linear regression parameters.
/// Result is a tuple (k, b) for y = k * x + b equation, solved by least squares approximation. /// Result is a tuple (k, b) for y = k * x + b equation, solved by least squares approximation.
template <typename X, typename Y, typename Ret = Float64> template <typename X, typename Y, typename Ret = Float64>
class AggregateFunctionLeastSqr final : public IAggregateFunctionDataHelper< class AggregateFunctionSimpleLinearRegression final : public IAggregateFunctionDataHelper<
AggregateFunctionLeastSqrData<X, Y, Ret>, AggregateFunctionSimpleLinearRegressionData<X, Y, Ret>,
AggregateFunctionLeastSqr<X, Y, Ret> AggregateFunctionSimpleLinearRegression<X, Y, Ret>
> >
{ {
public: public:
AggregateFunctionLeastSqr( AggregateFunctionSimpleLinearRegression(
const DataTypes & arguments, const DataTypes & arguments,
const Array & params const Array & params
): ):
IAggregateFunctionDataHelper< IAggregateFunctionDataHelper<
AggregateFunctionLeastSqrData<X, Y, Ret>, AggregateFunctionSimpleLinearRegressionData<X, Y, Ret>,
AggregateFunctionLeastSqr<X, Y, Ret> AggregateFunctionSimpleLinearRegression<X, Y, Ret>
> {arguments, params} > {arguments, params}
{ {
// notice: arguments has been checked before // notice: arguments has been checked before
@ -105,7 +105,7 @@ public:
String getName() const override String getName() const override
{ {
return "leastSqr"; return "simpleLinearRegression";
} }
const char * getHeaderFilePath() const override const char * getHeaderFilePath() const override
@ -120,12 +120,8 @@ public:
Arena * Arena *
) const override ) const override
{ {
auto col_x { auto col_x = static_cast<const ColumnVector<X> *>(columns[0]);
static_cast<const ColumnVector<X> *>(columns[0]) auto col_y = static_cast<const ColumnVector<Y> *>(columns[1]);
};
auto col_y {
static_cast<const ColumnVector<Y> *>(columns[1])
};
X x = col_x->getData()[row_num]; X x = col_x->getData()[row_num];
Y y = col_y->getData()[row_num]; Y y = col_y->getData()[row_num];
@ -159,12 +155,14 @@ public:
DataTypePtr getReturnType() const override DataTypePtr getReturnType() const override
{ {
DataTypes types { DataTypes types
{
std::make_shared<DataTypeNumber<Ret>>(), std::make_shared<DataTypeNumber<Ret>>(),
std::make_shared<DataTypeNumber<Ret>>(), std::make_shared<DataTypeNumber<Ret>>(),
}; };
Strings names { Strings names
{
"k", "k",
"b", "b",
}; };

View File

@ -30,7 +30,7 @@ void registerAggregateFunctionsBitmap(AggregateFunctionFactory &);
void registerAggregateFunctionsMaxIntersections(AggregateFunctionFactory &); void registerAggregateFunctionsMaxIntersections(AggregateFunctionFactory &);
void registerAggregateFunctionMLMethod(AggregateFunctionFactory &); void registerAggregateFunctionMLMethod(AggregateFunctionFactory &);
void registerAggregateFunctionEntropy(AggregateFunctionFactory &); void registerAggregateFunctionEntropy(AggregateFunctionFactory &);
void registerAggregateFunctionLeastSqr(AggregateFunctionFactory &); void registerAggregateFunctionSimpleLinearRegression(AggregateFunctionFactory &);
void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorArray(AggregateFunctionCombinatorFactory &); void registerAggregateFunctionCombinatorArray(AggregateFunctionCombinatorFactory &);
@ -73,7 +73,7 @@ void registerAggregateFunctions()
registerAggregateFunctionTimeSeriesGroupSum(factory); registerAggregateFunctionTimeSeriesGroupSum(factory);
registerAggregateFunctionMLMethod(factory); registerAggregateFunctionMLMethod(factory);
registerAggregateFunctionEntropy(factory); registerAggregateFunctionEntropy(factory);
registerAggregateFunctionLeastSqr(factory); registerAggregateFunctionSimpleLinearRegression(factory);
} }
{ {

View File

@ -81,9 +81,18 @@ StringRef ColumnNullable::getDataAt(size_t /*n*/) const
throw Exception{"Method getDataAt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED}; throw Exception{"Method getDataAt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED};
} }
void ColumnNullable::insertData(const char * /*pos*/, size_t /*length*/) void ColumnNullable::insertData(const char * pos, size_t length)
{ {
throw Exception{"Method insertData is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED}; if (pos == nullptr)
{
getNestedColumn().insertDefault();
getNullMapData().push_back(1);
}
else
{
getNestedColumn().insertData(pos, length);
getNullMapData().push_back(0);
}
} }
StringRef ColumnNullable::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const StringRef ColumnNullable::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const

View File

@ -51,6 +51,8 @@ public:
bool getBool(size_t n) const override { return isNullAt(n) ? 0 : nested_column->getBool(n); } bool getBool(size_t n) const override { return isNullAt(n) ? 0 : nested_column->getBool(n); }
UInt64 get64(size_t n) const override { return nested_column->get64(n); } UInt64 get64(size_t n) const override { return nested_column->get64(n); }
StringRef getDataAt(size_t n) const override; StringRef getDataAt(size_t n) const override;
/// Will insert null value if pos=nullptr
void insertData(const char * pos, size_t length) override; void insertData(const char * pos, size_t length) override;
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override;
const char * deserializeAndInsertFromArena(const char * pos) override; const char * deserializeAndInsertFromArena(const char * pos) override;

View File

@ -141,6 +141,7 @@ public:
/// Appends data located in specified memory chunk if it is possible (throws an exception if it cannot be implemented). /// Appends data located in specified memory chunk if it is possible (throws an exception if it cannot be implemented).
/// Is used to optimize some computations (in aggregation, for example). /// Is used to optimize some computations (in aggregation, for example).
/// Parameter length could be ignored if column values have fixed size. /// Parameter length could be ignored if column values have fixed size.
/// All data will be inserted as single element
virtual void insertData(const char * pos, size_t length) = 0; virtual void insertData(const char * pos, size_t length) = 0;
/// Appends "default value". /// Appends "default value".

View File

@ -16,15 +16,15 @@ struct BlockIO
BlockIO(const BlockIO &) = default; BlockIO(const BlockIO &) = default;
~BlockIO() = default; ~BlockIO() = default;
BlockOutputStreamPtr out;
BlockInputStreamPtr in;
/** process_list_entry should be destroyed after in and after out, /** process_list_entry should be destroyed after in and after out,
* since in and out contain pointer to objects inside process_list_entry (query-level MemoryTracker for example), * since in and out contain pointer to objects inside process_list_entry (query-level MemoryTracker for example),
* which could be used before destroying of in and out. * which could be used before destroying of in and out.
*/ */
std::shared_ptr<ProcessListEntry> process_list_entry; std::shared_ptr<ProcessListEntry> process_list_entry;
BlockOutputStreamPtr out;
BlockInputStreamPtr in;
/// Callbacks for query logging could be set here. /// Callbacks for query logging could be set here.
std::function<void(IBlockInputStream *, IBlockOutputStream *)> finish_callback; std::function<void(IBlockInputStream *, IBlockOutputStream *)> finish_callback;
std::function<void()> exception_callback; std::function<void()> exception_callback;

View File

@ -94,6 +94,14 @@ def colored(text, args, color=None, on_color=None, attrs=None):
else: else:
return text return text
def print_err(*args):
sys.stderr.write(' '.join(map(str,args)) + '\n')
def report_failure(name, msg):
print(msg)
# If stderr is not the same as stdout, duplicate the test name there.
if os.fstat(2) != os.fstat(1):
print_err(name, ":", msg)
SERVER_DIED = False SERVER_DIED = False
exit_code = 0 exit_code = 0
@ -140,7 +148,9 @@ def run_tests_array(all_tests_with_params):
elif not args.zookeeper and 'zookeeper' in name: elif not args.zookeeper and 'zookeeper' in name:
print(MSG_SKIPPED + " - no zookeeper") print(MSG_SKIPPED + " - no zookeeper")
skipped_total += 1 skipped_total += 1
elif not args.shard and 'shard' in name: elif not args.shard and ('shard' in name
or 'distributed' in name
or 'global' in name):
print(MSG_SKIPPED + " - no shard") print(MSG_SKIPPED + " - no shard")
skipped_total += 1 skipped_total += 1
elif not args.no_long and 'long' in name: elif not args.no_long and 'long' in name:
@ -171,7 +181,7 @@ def run_tests_array(all_tests_with_params):
raise raise
failures += 1 failures += 1
print("{0} - Timeout!".format(MSG_FAIL)) report_failure(name, "{0} - Timeout!".format(MSG_FAIL))
else: else:
counter = 1 counter = 1
while proc.returncode != 0 and need_retry(stderr): while proc.returncode != 0 and need_retry(stderr):
@ -184,10 +194,10 @@ def run_tests_array(all_tests_with_params):
if proc.returncode != 0: if proc.returncode != 0:
failures += 1 failures += 1
failures_chain += 1 failures_chain += 1
print("{0} - return code {1}".format(MSG_FAIL, proc.returncode)) report_failure(name, "{0} - return code {1}".format(MSG_FAIL, proc.returncode))
if stderr: if stderr:
print(stderr.encode('utf-8')) print_err(stderr.encode('utf-8'))
if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr: if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr:
SERVER_DIED = True SERVER_DIED = True
@ -195,20 +205,20 @@ def run_tests_array(all_tests_with_params):
elif stderr: elif stderr:
failures += 1 failures += 1
failures_chain += 1 failures_chain += 1
print("{0} - having stderror:\n{1}".format(MSG_FAIL, stderr.encode('utf-8'))) report_failure(name, "{0} - having stderror:\n{1}".format(MSG_FAIL, stderr.encode('utf-8')))
elif 'Exception' in stdout: elif 'Exception' in stdout:
failures += 1 failures += 1
failures_chain += 1 failures_chain += 1
print("{0} - having exception:\n{1}".format(MSG_FAIL, stdout.encode('utf-8'))) report_failure(name, "{0} - having exception:\n{1}".format(MSG_FAIL, stdout.encode('utf-8')))
elif not os.path.isfile(reference_file): elif not os.path.isfile(reference_file):
print("{0} - no reference file".format(MSG_UNKNOWN)) report_failure(name, "{0} - no reference file".format(MSG_UNKNOWN))
else: else:
result_is_different = subprocess.call(['diff', '-q', reference_file, stdout_file], stdout = PIPE) result_is_different = subprocess.call(['diff', '-q', reference_file, stdout_file], stdout = PIPE)
if result_is_different: if result_is_different:
diff = Popen(['diff', '--unified', reference_file, stdout_file], stdout = PIPE).communicate()[0] diff = Popen(['diff', '--unified', reference_file, stdout_file], stdout = PIPE).communicate()[0]
failures += 1 failures += 1
print("{0} - result differs with reference:\n{1}".format(MSG_FAIL, diff)) report_failure(name, "{0} - result differs with reference:\n{1}".format(MSG_FAIL, diff))
else: else:
passed_total += 1 passed_total += 1
failures_chain = 0 failures_chain = 0
@ -224,7 +234,7 @@ def run_tests_array(all_tests_with_params):
import traceback import traceback
exc_type, exc_value, tb = sys.exc_info() exc_type, exc_value, tb = sys.exc_info()
failures += 1 failures += 1
print("{0} - Test internal error: {1}\n{2}\n{3}".format(MSG_FAIL, exc_type.__name__, exc_value, "\n".join(traceback.format_tb(tb, 10)))) print_err("{0} - Test internal error: {1}\n{2}\n{3}".format(MSG_FAIL, exc_type.__name__, exc_value, "\n".join(traceback.format_tb(tb, 10))))
if failures_chain >= 20: if failures_chain >= 20:
break break
@ -232,7 +242,7 @@ def run_tests_array(all_tests_with_params):
failures_total = failures_total + failures failures_total = failures_total + failures
if failures_total > 0: if failures_total > 0:
print(colored("\nHaving {failures_total} errors! {passed_total} tests passed. {skipped_total} tests skipped.".format(passed_total = passed_total, skipped_total = skipped_total, failures_total = failures_total), args, "red", attrs=["bold"])) print_err(colored("\nHaving {failures_total} errors! {passed_total} tests passed. {skipped_total} tests skipped.".format(passed_total = passed_total, skipped_total = skipped_total, failures_total = failures_total), args, "red", attrs=["bold"]))
exit_code = 1 exit_code = 1
else: else:
print(colored("\n{passed_total} tests passed. {skipped_total} tests skipped.".format(passed_total = passed_total, skipped_total = skipped_total), args, "green", attrs=["bold"])) print(colored("\n{passed_total} tests passed. {skipped_total} tests skipped.".format(passed_total = passed_total, skipped_total = skipped_total), args, "green", attrs=["bold"]))
@ -388,11 +398,11 @@ def main(args):
processlist = get_processlist(args.client_with_database) processlist = get_processlist(args.client_with_database)
if processlist: if processlist:
server_pid = get_server_pid(os.getenv("CLICKHOUSE_PORT_TCP", '9000')) server_pid = get_server_pid(os.getenv("CLICKHOUSE_PORT_TCP", '9000'))
print(colored("\nFound hung queries in processlist:", args, "red", attrs=["bold"])) print_err(colored("\nFound hung queries in processlist:", args, "red", attrs=["bold"]))
print(processlist) print_err(processlist)
if server_pid: if server_pid:
print("\nStacktraces of all threads:") print_err("\nStacktraces of all threads:")
print(get_stacktraces(server_pid)) print_err(get_stacktraces(server_pid))
exit_code = 1 exit_code = 1
else: else:
print(colored("\nNo queries hung.", args, "green", attrs=["bold"])) print(colored("\nNo queries hung.", args, "green", attrs=["bold"]))
@ -455,6 +465,9 @@ if __name__ == '__main__':
args.queries = '/usr/share/clickhouse-test/queries' args.queries = '/usr/share/clickhouse-test/queries'
if args.tmp is None: if args.tmp is None:
args.tmp = '/tmp/clickhouse-test' args.tmp = '/tmp/clickhouse-test'
if args.queries is None:
print_err("Failed to detect path to the queries directory. Please specify it with '--queries' option.")
exit(1)
if args.tmp is None: if args.tmp is None:
args.tmp = args.queries args.tmp = args.queries
if args.client is None: if args.client is None:

View File

@ -1,7 +1,8 @@
FROM ubuntu:18.04 FROM ubuntu:18.04
# yandex/clickhouse-integration-tests-runner
RUN apt-get update \
RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes --force-yes \ && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
ca-certificates \ ca-certificates \
bash \ bash \
btrfs-progs \ btrfs-progs \
@ -21,8 +22,11 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes -
libicu-dev \ libicu-dev \
bsdutils \ bsdutils \
curl \ curl \
llvm-6.0 \ && rm -rf \
llvm-6.0-dev /var/lib/apt/lists/* \
/var/cache/debconf \
/tmp/* \
&& apt-get clean
ENV TZ=Europe/Moscow ENV TZ=Europe/Moscow
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
@ -32,12 +36,6 @@ RUN pip install pytest docker-compose==1.22.0 docker dicttoxml kazoo PyMySQL psy
ENV DOCKER_CHANNEL stable ENV DOCKER_CHANNEL stable
ENV DOCKER_VERSION 17.09.1-ce ENV DOCKER_VERSION 17.09.1-ce
ENV TSAN_OPTIONS 'halt_on_error=1 history_size=7'
ENV UBSAN_OPTIONS 'print_stacktrace=1'
ENV ASAN_SYMBOLIZER_PATH /usr/lib/llvm-6.0/bin/llvm-symbolizer
ENV UBSAN_SYMBOLIZER_PATH /usr/lib/llvm-6.0/bin/llvm-symbolizer
ENV LLVM_SYMBOLIZER_PATH /usr/lib/llvm-6.0/bin/llvm-symbolizer
RUN set -eux; \ RUN set -eux; \
\ \
# this "case" statement is generated via "update.sh" # this "case" statement is generated via "update.sh"

View File

@ -0,0 +1,48 @@
<test>
<name>test_read_formats</name>
<type>loop</type>
<create_query>CREATE TABLE IF NOT EXISTS table_{format} ENGINE = File({format}) AS test.hits</create_query>
<fill_query>INSERT INTO table_{format} SELECT * FROM test.hits LIMIT 100000</fill_query>
<stop_conditions>
<all_of>
<iterations>5</iterations>
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
</all_of>
<any_of>
<iterations>100</iterations>
<total_time_ms>60000</total_time_ms>
</any_of>
</stop_conditions>
<main_metric>
<min_time/>
</main_metric>
<substitutions>
<substitution>
<name>format</name>
<values>
<value>TabSeparated</value>
<value>TabSeparatedWithNames</value>
<value>TabSeparatedWithNamesAndTypes</value>
<value>CSV</value>
<value>CSVWithNames</value>
<value>Values</value>
<value>JSONEachRow</value>
<value>TSKV</value>
</values>
</substitution>
</substitutions>
<preconditions>
<table_exists>test.hits</table_exists>
</preconditions>
<query>SELECT * FROM table_{format}</query>
<drop_query>DROP TABLE IF EXISTS table_{format}</drop_query>
</test>

View File

@ -0,0 +1,53 @@
<test>
<name>test_select_format</name>
<type>loop</type>
<stop_conditions>
<all_of>
<iterations>5</iterations>
<min_time_not_changing_for_ms>10000</min_time_not_changing_for_ms>
</all_of>
<any_of>
<iterations>100</iterations>
<total_time_ms>60000</total_time_ms>
</any_of>
</stop_conditions>
<main_metric>
<min_time/>
</main_metric>
<substitutions>
<substitution>
<name>format</name>
<values>
<value>TabSeparated</value>
<value>TabSeparatedRaw</value>
<value>TabSeparatedWithNames</value>
<value>TabSeparatedWithNamesAndTypes</value>
<value>CSV</value>
<value>CSVWithNames</value>
<value>Values</value>
<value>Vertical</value>
<value>JSON</value>
<value>JSONCompact</value>
<value>JSONEachRow</value>
<value>TSKV</value>
<value>Pretty</value>
<value>PrettyCompact</value>
<value>PrettyCompactMonoBlock</value>
<value>PrettyNoEscapes</value>
<value>PrettySpace</value>
<value>XML</value>
</values>
</substitution>
</substitutions>
<preconditions>
<table_exists>test.hits</table_exists>
</preconditions>
<query>SELECT * FROM test.hits LIMIT 1000000 FORMAT {format}</query>
</test>

View File

@ -1,9 +1,9 @@
select arrayReduce('leastSqr', [1, 2, 3, 4], [100, 110, 120, 130]); select arrayReduce('simpleLinearRegression', [1, 2, 3, 4], [100, 110, 120, 130]);
select arrayReduce('leastSqr', [1, 2, 3, 4], [100, 110, 120, 131]); select arrayReduce('simpleLinearRegression', [1, 2, 3, 4], [100, 110, 120, 131]);
select arrayReduce('leastSqr', [-1, -2, -3, -4], [-100, -110, -120, -130]); select arrayReduce('simpleLinearRegression', [-1, -2, -3, -4], [-100, -110, -120, -130]);
select arrayReduce('leastSqr', [5, 5.1], [6, 6.1]); select arrayReduce('simpleLinearRegression', [5, 5.1], [6, 6.1]);
select arrayReduce('leastSqr', [0], [0]); select arrayReduce('simpleLinearRegression', [0], [0]);
select arrayReduce('leastSqr', [3, 4], [3, 3]); select arrayReduce('simpleLinearRegression', [3, 4], [3, 3]);
select arrayReduce('leastSqr', [3, 3], [3, 4]); select arrayReduce('simpleLinearRegression', [3, 3], [3, 4]);
select arrayReduce('leastSqr', emptyArrayUInt8(), emptyArrayUInt8()); select arrayReduce('simpleLinearRegression', emptyArrayUInt8(), emptyArrayUInt8());

View File

@ -162,7 +162,7 @@ INSERT INTO test.adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15',
OPTIMIZE TABLE test.adaptive_granularity_alter FINAL; OPTIMIZE TABLE test.adaptive_granularity_alter FINAL;
SELECT k, v2 FROM test.adaptive_granularity_alter WHERE k >= 100 OR k = 42; SELECT k, v2 FROM test.adaptive_granularity_alter WHERE k >= 100 OR k = 42 ORDER BY k;
SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter' and database='test' and active=1; SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter' and database='test' and active=1;
@ -356,7 +356,7 @@ INSERT INTO test.adaptive_granularity_alter (p, k, v1, v2) VALUES ('2018-05-15',
OPTIMIZE TABLE test.adaptive_granularity_alter FINAL; OPTIMIZE TABLE test.adaptive_granularity_alter FINAL;
SELECT k, v2 FROM test.adaptive_granularity_alter WHERE k >= 100 OR k = 42; SELECT k, v2 FROM test.adaptive_granularity_alter WHERE k >= 100 OR k = 42 ORDER BY k;
SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter' and database='test' and active=1; SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter' and database='test' and active=1;

View File

@ -199,14 +199,14 @@ SELECT sleep(0.7) Format Null;
OPTIMIZE TABLE test.adaptive_granularity_alter1 FINAL; OPTIMIZE TABLE test.adaptive_granularity_alter1 FINAL;
SELECT 'Parts optimized'; SELECT 'Parts optimized';
SELECT k, v2 FROM test.adaptive_granularity_alter1 WHERE k >= 100 OR k = 42; SELECT k, v2 FROM test.adaptive_granularity_alter1 WHERE k >= 100 OR k = 42 ORDER BY k;
SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1; SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1;
SYSTEM SYNC REPLICA test.adaptive_granularity_alter2; SYSTEM SYNC REPLICA test.adaptive_granularity_alter2;
SELECT 'Replica synced'; SELECT 'Replica synced';
SELECT k, v2 FROM test.adaptive_granularity_alter2 WHERE k >= 100 OR k = 42; SELECT k, v2 FROM test.adaptive_granularity_alter2 WHERE k >= 100 OR k = 42 ORDER BY k;
SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1; SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1;

File diff suppressed because one or more lines are too long

View File

@ -11,7 +11,7 @@ CREATE TABLE IF NOT EXISTS test.defaults
insert into test.defaults values (-3.273, -1.452, 4.267, 20.0, 40.0), (0.121, -0.615, 4.290, 20.0, 40.0); insert into test.defaults values (-3.273, -1.452, 4.267, 20.0, 40.0), (0.121, -0.615, 4.290, 20.0, 40.0);
DROP TABLE IF EXISTS test.model; DROP TABLE IF EXISTS test.model;
create table test.model engine = Memory as select LinearRegressionState(0.1, 0.0, 2, 'SGD')(target, param1, param2) as state from test.defaults; create table test.model engine = Memory as select linearRegressionState(0.1, 0.0, 2, 'SGD')(target, param1, param2) as state from test.defaults;
select ans < -61.374 and ans > -61.375 from select ans < -61.374 and ans > -61.375 from
(with (select state from remote('127.0.0.1', test.model)) as model select evalMLMethod(model, predict1, predict2) as ans from remote('127.0.0.1', test.defaults)); (with (select state from remote('127.0.0.1', test.model)) as model select evalMLMethod(model, predict1, predict2) as ans from remote('127.0.0.1', test.defaults));

View File

@ -10,7 +10,7 @@ CREATE TABLE IF NOT EXISTS test.defaults
) ENGINE = Memory; ) ENGINE = Memory;
insert into test.defaults values (1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2) insert into test.defaults values (1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2),(1,2,1,-1,-2),(-1,-2,-1,1,2)
DROP TABLE IF EXISTS test.model; DROP TABLE IF EXISTS test.model;
create table test.model engine = Memory as select LogisticRegressionState(0.1, 0.0, 1.0, 'SGD')(target, param1, param2) as state from test.defaults; create table test.model engine = Memory as select logisticRegressionState(0.1, 0.0, 1.0, 'SGD')(target, param1, param2) as state from test.defaults;
select ans < 1.1 and ans > 0.9 from select ans < 1.1 and ans > 0.9 from
(with (select state from test.model) as model select evalMLMethod(model, predict1, predict2) as ans from test.defaults limit 2); (with (select state from test.model) as model select evalMLMethod(model, predict1, predict2) as ans from test.defaults limit 2);

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

9
debian/.pbuilderrc vendored
View File

@ -3,10 +3,13 @@
# #
# ubuntu: # ubuntu:
# prepare old (trusty or earlier) host system: # prepare old (trusty or earlier) host system:
# sudo ln -s gutsy /usr/share/debootstrap/scripts/eoan
# sudo ln -s gutsy /usr/share/debootstrap/scripts/disco # sudo ln -s gutsy /usr/share/debootstrap/scripts/disco
# sudo ln -s gutsy /usr/share/debootstrap/scripts/cosmic # sudo ln -s gutsy /usr/share/debootstrap/scripts/cosmic
# sudo ln -s gutsy /usr/share/debootstrap/scripts/artful # sudo ln -s gutsy /usr/share/debootstrap/scripts/artful
# sudo ln -s gutsy /usr/share/debootstrap/scripts/bionic # sudo ln -s gutsy /usr/share/debootstrap/scripts/bionic
# sudo ln -s sid /usr/share/debootstrap/scripts/buster
# build ubuntu: # build ubuntu:
# sudo DIST=trusty pbuilder create --configfile debian/.pbuilderrc && DIST=trusty pdebuild --configfile debian/.pbuilderrc # sudo DIST=trusty pbuilder create --configfile debian/.pbuilderrc && DIST=trusty pdebuild --configfile debian/.pbuilderrc
# sudo DIST=xenial pbuilder create --configfile debian/.pbuilderrc && DIST=xenial pdebuild --configfile debian/.pbuilderrc # sudo DIST=xenial pbuilder create --configfile debian/.pbuilderrc && DIST=xenial pdebuild --configfile debian/.pbuilderrc
@ -56,7 +59,7 @@ DEBIAN_SUITES=($UNSTABLE_CODENAME $TESTING_CODENAME $STABLE_CODENAME $STABLE_BAC
"experimental" "unstable" "testing" "stable") "experimental" "unstable" "testing" "stable")
# List of Ubuntu suites. Update these when needed. # List of Ubuntu suites. Update these when needed.
UBUNTU_SUITES=("disco" "cosmic" "bionic" "artful" "zesty" "xenial" "trusty" "devel") UBUNTU_SUITES=("eoan" "disco" "cosmic" "bionic" "artful" "zesty" "xenial" "trusty" "devel")
# Set a default distribution if none is used. Note that you can set your own default (i.e. ${DIST:="unstable"}). # Set a default distribution if none is used. Note that you can set your own default (i.e. ${DIST:="unstable"}).
HOST_DIST=`lsb_release --short --codename` HOST_DIST=`lsb_release --short --codename`
@ -167,12 +170,12 @@ esac
if [ "$ARCH" != arm64 ]; then if [ "$ARCH" != arm64 ]; then
case "$DIST" in case "$DIST" in
# TODO: fix llvm-8 and use for "disco" # TODO: fix llvm-8 and use for "disco" and "eoan"
"experimental") "experimental")
EXTRAPACKAGES+=" liblld-8-dev libclang-8-dev llvm-8-dev liblld-8 " EXTRAPACKAGES+=" liblld-8-dev libclang-8-dev llvm-8-dev liblld-8 "
export CMAKE_FLAGS="-DLLVM_VERSION=8 $CMAKE_FLAGS" export CMAKE_FLAGS="-DLLVM_VERSION=8 $CMAKE_FLAGS"
;; ;;
"disco" | "cosmic" | "testing" | "unstable") "eoan" | "disco" | "cosmic" | "testing" | "unstable")
EXTRAPACKAGES+=" liblld-7-dev libclang-7-dev llvm-7-dev liblld-7 " EXTRAPACKAGES+=" liblld-7-dev libclang-7-dev llvm-7-dev liblld-7 "
export CMAKE_FLAGS="-DLLVM_VERSION=7 $CMAKE_FLAGS" export CMAKE_FLAGS="-DLLVM_VERSION=7 $CMAKE_FLAGS"
;; ;;

1
debian/control vendored
View File

@ -11,6 +11,7 @@ Build-Depends: debhelper (>= 9),
libicu-dev, libicu-dev,
libreadline-dev, libreadline-dev,
gperf, gperf,
python,
tzdata tzdata
Standards-Version: 3.9.8 Standards-Version: 3.9.8

View File

@ -8,10 +8,10 @@ PACKAGE_INSTALL=${PACKAGE_INSTALL=1}
TEST_PORT_RANDOM=${TEST_PORT_RANDOM=1} TEST_PORT_RANDOM=${TEST_PORT_RANDOM=1}
if [ "${PACKAGE_INSTALL}" ]; then if [ "${PACKAGE_INSTALL}" ]; then
for PKG in $(ls /tmp/buildd/*.deb | sed -e's,.*/,,;s,_.*,,' ); do #for PKG in $(ls /tmp/buildd/*.deb | sed -e's,.*/,,;s,_.*,,' ); do
apt-get install -y --force-yes "$PKG" ||: # apt-get install -y --force-yes "$PKG" ||:
apt-get remove -y "$PKG" ||: # apt-get remove -y "$PKG" ||:
done #done
dpkg --auto-deconfigure -i /tmp/buildd/*.deb ||: dpkg --auto-deconfigure -i /tmp/buildd/*.deb ||:
apt install -y -f --allow-downgrades ||: apt install -y -f --allow-downgrades ||:
@ -45,6 +45,8 @@ export CLICKHOUSE_PORT_TCP_SECURE=${CLICKHOUSE_PORT_TCP_SECURE:=9440}
export CLICKHOUSE_PORT_HTTPS=${CLICKHOUSE_PORT_HTTPS:=8443} export CLICKHOUSE_PORT_HTTPS=${CLICKHOUSE_PORT_HTTPS:=8443}
if [ "${TEST_CONNECT}" ]; then if [ "${TEST_CONNECT}" ]; then
sed -i 's/ssl_conf = ssl_sect/#ssl_conf = ssl_sect/g' /etc/ssl/openssl.cnf
cat /etc/ssl/openssl.cnf
[ "${TEST_PORT_RANDOM}" ] && echo "<yandex><http_port>${CLICKHOUSE_PORT_HTTP}</http_port><tcp_port>${CLICKHOUSE_PORT_TCP}</tcp_port><interserver_http_port>${CLICKHOUSE_PORT_INTERSERVER}</interserver_http_port></yandex>" > /etc/clickhouse-server/config.d/port.xml [ "${TEST_PORT_RANDOM}" ] && echo "<yandex><http_port>${CLICKHOUSE_PORT_HTTP}</http_port><tcp_port>${CLICKHOUSE_PORT_TCP}</tcp_port><interserver_http_port>${CLICKHOUSE_PORT_INTERSERVER}</interserver_http_port></yandex>" > /etc/clickhouse-server/config.d/port.xml
if [ "${TEST_SSL}" ]; then if [ "${TEST_SSL}" ]; then

View File

@ -1,12 +1,19 @@
FROM ubuntu:18.04 FROM ubuntu:18.04
# yandex/clickhouse-integration-test
RUN apt-get update && apt-get -y install tzdata python llvm-6.0 llvm-6.0-dev RUN apt-get update \
&& env DEBIAN_FRONTEND=noninteractive apt-get -y install tzdata python llvm-6.0 llvm-6.0-dev libreadline-dev libicu-dev bsdutils \
&& rm -rf \
/var/lib/apt/lists/* \
/var/cache/debconf \
/tmp/* \
&& apt-get clean
ENV TZ=Europe/Moscow ENV TZ=Europe/Moscow
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
CMD echo "TSAN_OPTIONS='halt_on_error=1 history_size=7'" >> /etc/environment; RUN echo "TSAN_OPTIONS='halt_on_error=1 history_size=7'" >> /etc/environment; \
CMD echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \
CMD echo "ASAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment; echo "ASAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment; \
CMD echo "UBSAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment; echo "UBSAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment; \
CMD echo "LLVM_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment; echo "LLVM_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment;

View File

@ -21,7 +21,12 @@ Tests should use (create, drop, etc) only tables in `test` database that is assu
If you want to use distributed queries in functional tests, you can leverage `remote` table function with `127.0.0.{1..2}` addresses for the server to query itself; or you can use predefined test clusters in server configuration file like `test_shard_localhost`. If you want to use distributed queries in functional tests, you can leverage `remote` table function with `127.0.0.{1..2}` addresses for the server to query itself; or you can use predefined test clusters in server configuration file like `test_shard_localhost`.
Some tests are marked with `zookeeper`, `shard` or `long` in their names. `zookeeper` is for tests that are using ZooKeeper; `shard` is for tests that requires server to listen `127.0.0.*`; `long` is for tests that run slightly longer that one second. Some tests are marked with `zookeeper`, `shard` or `long` in their names.
`zookeeper` is for tests that are using ZooKeeper. `shard` is for tests that
requires server to listen `127.0.0.*`; `distributed` or `global` have the same
meaning. `long` is for tests that run slightly longer that one second. You can
disable these groups of tests using `--no-zookeeper`, `--no-shard` and
`--no-long` options, respectively.
## Known bugs ## Known bugs

View File

@ -193,7 +193,7 @@ SELECT Carrier, count(*) FROM ontime WHERE DepDelay>10 AND Year = 2007 GROUP BY
Q5. The percentage of delays by carrier for 2007 Q5. The percentage of delays by carrier for 2007
``` sql ``` sql
SELECT Carrier, c, c2, c*1000/c2 as c3 SELECT Carrier, c, c2, c*100/c2 as c3
FROM FROM
( (
SELECT SELECT
@ -219,13 +219,13 @@ ORDER BY c3 DESC;
Better version of the same query: Better version of the same query:
``` sql ``` sql
SELECT Carrier, avg(DepDelay > 10) * 1000 AS c3 FROM ontime WHERE Year = 2007 GROUP BY Carrier ORDER BY Carrier SELECT Carrier, avg(DepDelay > 10) * 100 AS c3 FROM ontime WHERE Year = 2007 GROUP BY Carrier ORDER BY Carrier
``` ```
Q6. The previous request for a broader range of years, 2000-2008 Q6. The previous request for a broader range of years, 2000-2008
``` sql ``` sql
SELECT Carrier, c, c2, c*1000/c2 as c3 SELECT Carrier, c, c2, c*100/c2 as c3
FROM FROM
( (
SELECT SELECT
@ -251,7 +251,7 @@ ORDER BY c3 DESC;
Better version of the same query: Better version of the same query:
``` sql ``` sql
SELECT Carrier, avg(DepDelay > 10) * 1000 AS c3 FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY Carrier ORDER BY Carrier SELECT Carrier, avg(DepDelay > 10) * 100 AS c3 FROM ontime WHERE Year >= 2000 AND Year <= 2008 GROUP BY Carrier ORDER BY Carrier
``` ```
Q7. Percentage of flights delayed for more than 10 minutes, by year Q7. Percentage of flights delayed for more than 10 minutes, by year
@ -262,7 +262,7 @@ FROM
( (
select select
Year, Year,
count(*)*1000 as c1 count(*)*100 as c1
from ontime from ontime
WHERE DepDelay>10 WHERE DepDelay>10
GROUP BY Year GROUP BY Year

View File

@ -622,5 +622,19 @@ When sequential consistency is enabled, ClickHouse allows the client to execute
- [insert_quorum](#settings-insert_quorum) - [insert_quorum](#settings-insert_quorum)
- [insert_quorum_timeout](#settings-insert_quorum_timeout) - [insert_quorum_timeout](#settings-insert_quorum_timeout)
## allow_experimental_cross_to_join_conversion {#settings-allow_experimental_cross_to_join_conversion}
Enables or disables:
1. Rewriting of queries with multiple [JOIN clauses](../../query_language/select.md#select-join) from the syntax with commas to the `JOIN ON/USING` syntax. If the setting value is 0, ClickHouse doesn't process queries with the syntax with commas, and throws an exception.
2. Converting of `CROSS JOIN` into `INNER JOIN` if conditions of join allow it.
Possible values:
- 0 — Disabled.
- 1 — Enabled.
Default value: 1.
[Original article](https://clickhouse.yandex/docs/en/operations/settings/settings/) <!--hide--> [Original article](https://clickhouse.yandex/docs/en/operations/settings/settings/) <!--hide-->

View File

@ -62,7 +62,29 @@ Returns 1, if the set of bytes is valid UTF-8 encoded, otherwise 0.
## toValidUTF8 ## toValidUTF8
Returns string where all invalid UTF-8 characters are replaced by replacement character `<60>` (U+FFFD). If there are many invalid characters subsequently, it is collapsed to one replacement character. Replaces invalid UTF-8 characters by the `<60>` (U+FFFD) character. All running in a row invalid characters are collapsed into the one replacement character.
```
toValidUTF8( input_string )
```
Parameters:
- input_string — Any set of bytes represented as the [String](../../data_types/string.md) data type object.
Returned value: Valid UTF-8 string.
### Example
```sql
SELECT toValidUTF8('\x61\xF0\x80\x80\x80b')
```
```text
┌─toValidUTF8('a<><61><EFBFBD><EFBFBD>b')─┐
│ a<>b │
└───────────────────────┘
```
## reverse ## reverse

View File

@ -438,7 +438,7 @@ FROM <left_subquery>
The table names can be specified instead of `<left_subquery>` and `<right_subquery>`. This is equivalent to the `SELECT * FROM table` subquery, except in a special case when the table has the [Join](../operations/table_engines/join.md) engine an array prepared for joining. The table names can be specified instead of `<left_subquery>` and `<right_subquery>`. This is equivalent to the `SELECT * FROM table` subquery, except in a special case when the table has the [Join](../operations/table_engines/join.md) engine an array prepared for joining.
**Supported types of `JOIN`** #### Supported Types of `JOIN`
- `INNER JOIN` (or `JOIN`) - `INNER JOIN` (or `JOIN`)
- `LEFT JOIN` (or `LEFT OUTER JOIN`) - `LEFT JOIN` (or `LEFT OUTER JOIN`)
@ -448,14 +448,34 @@ The table names can be specified instead of `<left_subquery>` and `<right_subque
See the standard [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) description. See the standard [SQL JOIN](https://en.wikipedia.org/wiki/Join_(SQL)) description.
**ANY or ALL strictness** #### Multiple JOIN
Performing queries, ClickHouse rewrites multiple joins into the combination of two-table joins and processes them sequentially. If there are four tables for join ClickHouse joins the first and the second, then joins the result with the third table, and at the last step, it joins the fourth one.
If a query contains `WHERE` clause, ClickHouse tries to push down filters from this clause into the intermediate join. If it cannot apply the filter to each intermediate join, ClickHouse applies the filters after all joins are completed.
We recommend the `JOIN ON` or `JOIN USING` syntax for creating a query. For example:
```
SELECT * FROM t1 JOIN t2 ON t1.a = t2.a JOIN t3 ON t1.a = t3.a
```
Also, you can use comma separated list of tables for join. Works only with the [allow_experimental_cross_to_join_conversion = 1](../operations/settings/settings.md#settings-allow_experimental_cross_to_join_conversion) setting.
For example, `SELECT * FROM t1, t2, t3 WHERE t1.a = t2.a AND t1.a = t3.a`
Don't mix these syntaxes.
ClickHouse doesn't support the syntax with commas directly, so we don't recommend to use it. The algorithm tries to rewrite the query in terms of `CROSS` and `INNER` `JOIN` clauses and then proceeds the query processing. When rewriting the query, ClickHouse tries to optimize performance and memory consumption. By default, ClickHouse treats comma as an `INNER JOIN` clause and converts it to `CROSS JOIN` when the algorithm cannot guaranty that `INNER JOIN` returns required data.
#### ANY or ALL Strictness
If `ALL` is specified and the right table has several matching rows, the data will be multiplied by the number of these rows. This is the normal `JOIN` behavior for standard SQL. If `ALL` is specified and the right table has several matching rows, the data will be multiplied by the number of these rows. This is the normal `JOIN` behavior for standard SQL.
If `ANY` is specified and the right table has several matching rows, only the first one found is joined. If the right table has only one matching row, the results of `ANY` and `ALL` are the same. If `ANY` is specified and the right table has several matching rows, only the first one found is joined. If the right table has only one matching row, the results of `ANY` and `ALL` are the same.
To set the default strictness value, use the session configuration parameter [join_default_strictness](../operations/settings/settings.md#settings-join_default_strictness). To set the default strictness value, use the session configuration parameter [join_default_strictness](../operations/settings/settings.md#settings-join_default_strictness).
**GLOBAL JOIN** #### GLOBAL JOIN
When using a normal `JOIN`, the query is sent to remote servers. Subqueries are run on each of them in order to make the right table, and the join is performed with this table. In other words, the right table is formed on each server separately. When using a normal `JOIN`, the query is sent to remote servers. Subqueries are run on each of them in order to make the right table, and the join is performed with this table. In other words, the right table is formed on each server separately.
@ -463,7 +483,7 @@ When using `GLOBAL ... JOIN`, first the requestor server runs a subquery to calc
Be careful when using `GLOBAL`. For more information, see the section [Distributed subqueries](#select-distributed-subqueries). Be careful when using `GLOBAL`. For more information, see the section [Distributed subqueries](#select-distributed-subqueries).
**Usage Recommendations** #### Usage Recommendations
All columns that are not needed for the `JOIN` are deleted from the subquery. All columns that are not needed for the `JOIN` are deleted from the subquery.

View File

@ -42,7 +42,30 @@
Возвращает 1, если набор байт является корректным в кодировке UTF-8, 0 иначе. Возвращает 1, если набор байт является корректным в кодировке UTF-8, 0 иначе.
## toValidUTF8 ## toValidUTF8
Возвращает строку, где все некорректные UTF-8 символы заменены на заменяющий символ `<60>` (U+FFFD). Если много некорректных символов идут подряд, схлопывается до одного заменяющего символа.
Заменяет некорректные символы UTF-8 на символ `<60>` (U+FFFD). Все идущие подряд некорректные символы схлопываются в один заменяющий символ.
```
toValidUTF8( input_string )
```
Параметры:
- input_string — произвольный набор байтов, представленный как объект типа [String](../../data_types/string.md).
Возвращаемое значение: Корректная строка UTF-8.
### Пример
```sql
SELECT toValidUTF8('\x61\xF0\x80\x80\x80b')
```
```text
┌─toValidUTF8('a<><61><EFBFBD><EFBFBD>b')─┐
│ a<>b │
└───────────────────────┘
```
## reverse ## reverse
Разворачивает строку (как последовательность байт). Разворачивает строку (как последовательность байт).

View File

@ -7,37 +7,19 @@ if(COMPILER_CLANG)
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-builtin-requires-header") set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-builtin-requires-header")
endif() endif()
add_headers_and_sources(glibc_compatibility .)
add_headers_and_sources(glibc_compatibility musl)
list(APPEND glibc_compatibility_sources musl/syscall.s musl/longjmp.s)
set(GLIBC_COMPATIBILITY_SOURCES list(REMOVE_ITEM glibc_compatibility_sources musl/getentropy.c)
glibc-compatibility.c
musl/pipe2.c
musl/fallocate.c
musl/longjmp.s
musl/vasprintf.c
musl/lgamma.c
musl/posix_spawn.c
musl/futimens.c
musl/syscall.s
musl/syscall_ret.c
musl/sched_cpucount.c
musl/glob.c
musl/exp2f.c
musl/pwritev.c
musl/getrandom.c
musl/fcntl.c
musl/timespec_get.c
musl/sched_getcpu.c
)
if(HAVE_SYS_RANDOM_H) if(HAVE_SYS_RANDOM_H)
list(APPEND GLIBC_COMPATIBILITY_SOURCES musl/getentropy.c) list(APPEND glibc_compatibility_sources musl/getentropy.c)
endif() endif()
if(MAKE_STATIC_LIBRARIES) if(MAKE_STATIC_LIBRARIES)
set(GLIBC_COMPATIBILITY_SOURCES ${GLIBC_COMPATIBILITY_SOURCES} list(APPEND glibc_compatibility_sources libcxxabi/cxa_thread_atexit.cpp)
libcxxabi/cxa_thread_atexit.cpp)
endif() endif()
add_library(glibc-compatibility ${GLIBC_COMPATIBILITY_SOURCES}) add_library(glibc-compatibility ${glibc_compatibility_sources})
target_include_directories(glibc-compatibility PRIVATE libcxxabi) target_include_directories(glibc-compatibility PRIVATE libcxxabi)

View File

@ -3,6 +3,8 @@
# If you have "no space left" error, you can change the location of temporary files with BUILDPLACE environment variable. # If you have "no space left" error, you can change the location of temporary files with BUILDPLACE environment variable.
# Advanced usage: # Advanced usage:
# Test gcc-9:
# env DIST=disco EXTRAPACKAGES="gcc-9 g++-9" DEB_CC=gcc-9 DEB_CXX=g++-9 CMAKE_FLAGS=" -DNO_WERROR=1 " ./release
# Test gcc-8: # Test gcc-8:
# env DIST=bionic EXTRAPACKAGES="gcc-8 g++-8" DEB_CC=gcc-8 DEB_CXX=g++-8 CMAKE_FLAGS=" -DNO_WERROR=1 " ./release # env DIST=bionic EXTRAPACKAGES="gcc-8 g++-8" DEB_CC=gcc-8 DEB_CXX=g++-8 CMAKE_FLAGS=" -DNO_WERROR=1 " ./release
# Clang6 build: # Clang6 build: