Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
Alexey Milovidov 2018-12-20 19:11:55 +03:00
commit cbbf0bdb58
39 changed files with 481 additions and 61 deletions

View File

@ -31,7 +31,7 @@
* Исправлена работа некоторых случаев `VIEW` и подзапросов без указания базы данных. [Winter Zhang](https://github.com/yandex/ClickHouse/pull/3521)
* Исправлен race condition при одновременном чтении из `MATERIALIZED VIEW` и удалением `MATERIALIZED VIEW` из-за отсутствия блокировки внутренней таблицы `MATERIALIZED VIEW`. [#3404](https://github.com/yandex/ClickHouse/pull/3404) [#3694](https://github.com/yandex/ClickHouse/pull/3694)
* Исправлена ошибка `Lock handler cannot be nullptr.` [#3689](https://github.com/yandex/ClickHouse/pull/3689)
* Исправления выполнения запросов при включенной настройке `compile_expressions`ыключена по-умолчанию) - убрана свёртка недетерминированных константных выражений, как например, функции `now`. [#3457](https://github.com/yandex/ClickHouse/pull/3457)
* Исправления выполнения запросов при включенной настройке `compile_expressions` (включена по-умолчанию) - убрана свёртка недетерминированных константных выражений, как например, функции `now`. [#3457](https://github.com/yandex/ClickHouse/pull/3457)
* Исправлено падение при указании неконстантного аргумента scale в функциях `toDecimal32/64/128`.
* Исправлена ошибка при попытке вставки в формате `Values` массива с `NULL` элементами в столбец типа `Array` без `Nullable` (в случае `input_format_values_interpret_expressions` = 1). [#3487](https://github.com/yandex/ClickHouse/pull/3487) [#3503](https://github.com/yandex/ClickHouse/pull/3503)
* Исправлено непрерывное логгирование ошибок в `DDLWorker`, если ZooKeeper недоступен. [8f50c620](https://github.com/yandex/ClickHouse/commit/8f50c620334988b28018213ec0092fe6423847e2)

View File

@ -1,4 +1,11 @@
option (ENABLE_BASE64 "Enable base64" ON)
if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64/lib/lib.c")
set (MISSING_INTERNAL_BASE64_LIBRARY 1)
message (WARNING "submodule contrib/base64 is missing. to fix try run: \n git submodule update --init --recursive")
endif ()
if (NOT MISSING_INTERNAL_BASE64_LIBRARY)
option (ENABLE_BASE64 "Enable base64" ON)
endif ()
if (ENABLE_BASE64)
if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/base64")
@ -9,4 +16,3 @@ if (ENABLE_BASE64)
set (USE_BASE64 1)
endif()
endif ()

View File

@ -16,7 +16,7 @@ endif ()
if (HDFS3_LIBRARY AND HDFS3_INCLUDE_DIR)
set(USE_HDFS 1)
elseif (LIBGSASL_LIBRARY)
elseif (LIBGSASL_LIBRARY AND LIBXML2_LIBRARY)
set(HDFS3_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libhdfs3/include")
set(HDFS3_LIBRARY hdfs3)
set(USE_HDFS 1)

View File

@ -2,9 +2,12 @@ if (NOT APPLE AND NOT ARCH_32)
option (USE_INTERNAL_LIBGSASL_LIBRARY "Set to FALSE to use system libgsasl library instead of bundled" ${NOT_UNBUNDLED})
endif ()
if (USE_INTERNAL_LIBGSASL_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src/gsasl.h")
message (WARNING "submodule contrib/libgsasl is missing. to fix try run: \n git submodule update --init --recursive")
set (USE_INTERNAL_LIBGSASL_LIBRARY 0)
if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src/gsasl.h")
if (USE_INTERNAL_LIBGSASL_LIBRARY)
message (WARNING "submodule contrib/libgsasl is missing. to fix try run: \n git submodule update --init --recursive")
set (USE_INTERNAL_LIBGSASL_LIBRARY 0)
endif ()
set (MISSING_INTERNAL_LIBGSASL_LIBRARY 1)
endif ()
if (NOT USE_INTERNAL_LIBGSASL_LIBRARY)
@ -13,7 +16,7 @@ if (NOT USE_INTERNAL_LIBGSASL_LIBRARY)
endif ()
if (LIBGSASL_LIBRARY AND LIBGSASL_INCLUDE_DIR)
elseif (NOT APPLE AND NOT ARCH_32)
elseif (NOT MISSING_INTERNAL_LIBGSASL_LIBRARY AND NOT APPLE AND NOT ARCH_32)
set (LIBGSASL_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/src ${ClickHouse_SOURCE_DIR}/contrib/libgsasl/linux_x86_64/include)
set (USE_INTERNAL_LIBGSASL_LIBRARY 1)
set (LIBGSASL_LIBRARY libgsasl)

View File

@ -1,8 +1,11 @@
option (USE_INTERNAL_LIBXML2_LIBRARY "Set to FALSE to use system libxml2 library instead of bundled" ${NOT_UNBUNDLED})
if (USE_INTERNAL_LIBXML2_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libxml2/libxml.h")
message (WARNING "submodule contrib/libxml2 is missing. to fix try run: \n git submodule update --init --recursive")
set (USE_INTERNAL_LIBXML2_LIBRARY 0)
if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libxml2/libxml.h")
if (USE_INTERNAL_LIBXML2_LIBRARY)
message (WARNING "submodule contrib/libxml2 is missing. to fix try run: \n git submodule update --init --recursive")
set (USE_INTERNAL_LIBXML2_LIBRARY 0)
endif ()
set (MISSING_INTERNAL_LIBXML2_LIBRARY 1)
endif ()
if (NOT USE_INTERNAL_LIBXML2_LIBRARY)
@ -11,7 +14,7 @@ if (NOT USE_INTERNAL_LIBXML2_LIBRARY)
endif ()
if (LIBXML2_LIBRARY AND LIBXML2_INCLUDE_DIR)
else ()
elseif (NOT MISSING_INTERNAL_LIBXML2_LIBRARY)
set (LIBXML2_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libxml2/include ${ClickHouse_SOURCE_DIR}/contrib/libxml2-cmake/linux_x86_64/include)
set (USE_INTERNAL_LIBXML2_LIBRARY 1)
set (LIBXML2_LIBRARY libxml2)

View File

@ -2,7 +2,7 @@
if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-stringop-overflow")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-implicit-fallthrough -Wno-class-memaccess -std=c++1z")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-but-set-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-maybe-uninitialized -Wno-format -Wno-misleading-indentation -Wno-implicit-fallthrough -Wno-class-memaccess -Wno-sign-compare -std=c++1z")
elseif (CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Wno-unused-function -Wno-unused-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-format -Wno-parentheses-equality")
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-old-style-cast -Wno-unused-function -Wno-unused-variable -Wno-unused-result -Wno-deprecated-declarations -Wno-non-virtual-dtor -Wno-format -std=c++1z")

View File

@ -61,7 +61,7 @@ int mainEntryClickHouseCompressor(int argc, char ** argv)
("block-size,b", boost::program_options::value<unsigned>()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size")
("hc", "use LZ4HC instead of LZ4")
("zstd", "use ZSTD instead of LZ4")
("level", "compression level")
("level", boost::program_options::value<int>(), "compression level")
("none", "use no compression instead of LZ4")
("stat", "print block statistics of compressed data")
;
@ -94,7 +94,9 @@ int mainEntryClickHouseCompressor(int argc, char ** argv)
else if (use_none)
method = DB::CompressionMethod::NONE;
DB::CompressionSettings settings(method, options.count("level") > 0 ? options["level"].as<int>() : DB::CompressionSettings::getDefaultLevel(method));
DB::CompressionSettings settings(method, options.count("level")
? options["level"].as<int>()
: DB::CompressionSettings::getDefaultLevel(method));
DB::ReadBufferFromFileDescriptor rb(STDIN_FILENO);
DB::WriteBufferFromFileDescriptor wb(STDOUT_FILENO);

View File

@ -25,6 +25,7 @@ namespace Poco { class Logger; }
namespace DB
{
struct ColumnsDescription;
/// State of query processing.
struct QueryState

View File

@ -0,0 +1,36 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionBoundingRatio.h>
#include <AggregateFunctions/FactoryHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
namespace
{
AggregateFunctionPtr createAggregateFunctionRate(const std::string & name, const DataTypes & argument_types, const Array & parameters)
{
assertNoParameters(name, parameters);
assertBinary(name, argument_types);
if (argument_types.size() < 2)
throw Exception("Aggregate function " + name + " requires at least two arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return std::make_shared<AggregateFunctionBoundingRatio>(argument_types);
}
}
void registerAggregateFunctionRate(AggregateFunctionFactory & factory)
{
factory.registerFunction("boundingRatio", createAggregateFunctionRate, AggregateFunctionFactory::CaseInsensitive);
}
}

View File

@ -0,0 +1,162 @@
#pragma once
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <Common/FieldVisitors.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <AggregateFunctions/Helpers.h>
#include <AggregateFunctions/IAggregateFunction.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
/** Tracks the leftmost and rightmost (x, y) data points.
*/
struct AggregateFunctionBoundingRatioData
{
struct Point
{
Float64 x;
Float64 y;
};
bool empty = true;
Point left;
Point right;
void add(Float64 x, Float64 y)
{
Point point{x, y};
if (empty)
{
left = point;
right = point;
empty = false;
}
else if (point.x < left.x)
{
left = point;
}
else if (point.x > right.x)
{
right = point;
}
}
void merge(const AggregateFunctionBoundingRatioData & other)
{
if (empty)
{
*this = other;
}
else
{
if (other.left.x < left.x)
left = other.left;
if (other.right.x > right.x)
right = other.right;
}
}
void serialize(WriteBuffer & buf) const
{
writeBinary(empty, buf);
if (!empty)
{
writePODBinary(left, buf);
writePODBinary(right, buf);
}
}
void deserialize(ReadBuffer & buf)
{
readBinary(empty, buf);
if (!empty)
{
readPODBinary(left, buf);
readPODBinary(right, buf);
}
}
};
class AggregateFunctionBoundingRatio final : public IAggregateFunctionDataHelper<AggregateFunctionBoundingRatioData, AggregateFunctionBoundingRatio>
{
private:
/** Calculates the slope of a line between leftmost and rightmost data points.
* (y2 - y1) / (x2 - x1)
*/
Float64 getBoundingRatio(const AggregateFunctionBoundingRatioData & data) const
{
if (data.empty)
return std::numeric_limits<Float64>::quiet_NaN();
return (data.right.y - data.left.y) / (data.right.x - data.left.x);
}
public:
String getName() const override
{
return "boundingRatio";
}
AggregateFunctionBoundingRatio(const DataTypes & arguments)
{
const auto x_arg = arguments.at(0).get();
const auto y_arg = arguments.at(0).get();
if (!x_arg->isValueRepresentedByNumber() || !y_arg->isValueRepresentedByNumber())
throw Exception("Illegal types of arguments of aggregate function " + getName() + ", must have number representation.",
ErrorCodes::BAD_ARGUMENTS);
}
DataTypePtr getReturnType() const override
{
return std::make_shared<DataTypeFloat64>();
}
void add(AggregateDataPtr place, const IColumn ** columns, const size_t row_num, Arena *) const override
{
/// TODO Inefficient.
const auto x = applyVisitor(FieldVisitorConvertToNumber<Float64>(), (*columns[0])[row_num]);
const auto y = applyVisitor(FieldVisitorConvertToNumber<Float64>(), (*columns[1])[row_num]);
data(place).add(x, y);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override
{
data(place).merge(data(rhs));
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
data(place).serialize(buf);
}
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena *) const override
{
data(place).deserialize(buf);
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
static_cast<ColumnFloat64 &>(to).getData().push_back(getBoundingRatio(data(place)));
}
const char * getHeaderFilePath() const override
{
return __FILE__;
}
};
}

View File

@ -17,12 +17,13 @@ namespace ErrorCodes
extern const int PARAMETER_OUT_OF_BOUND;
}
namespace
{
AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name, const DataTypes & arguments, const Array & params)
AggregateFunctionPtr createAggregateFunctionHistogram(const std::string &, const DataTypes &, const Array &)
{
if (params.size() != 1)
/* if (params.size() != 1)
throw Exception("Function " + name + " requires single parameter: bins count", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (params[0].getType() != Field::Types::UInt64)
@ -43,7 +44,9 @@ AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name,
if (!res)
throw Exception("Illegal type " + arguments[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
return res;*/
return nullptr;
}
}

View File

@ -15,6 +15,7 @@ void registerAggregateFunctionGroupArrayInsertAt(AggregateFunctionFactory &);
void registerAggregateFunctionsQuantile(AggregateFunctionFactory &);
void registerAggregateFunctionsSequenceMatch(AggregateFunctionFactory &);
void registerAggregateFunctionWindowFunnel(AggregateFunctionFactory &);
void registerAggregateFunctionRate(AggregateFunctionFactory &);
void registerAggregateFunctionsMinMaxAny(AggregateFunctionFactory &);
void registerAggregateFunctionsStatisticsStable(AggregateFunctionFactory &);
void registerAggregateFunctionsStatisticsSimple(AggregateFunctionFactory &);
@ -50,6 +51,7 @@ void registerAggregateFunctions()
registerAggregateFunctionsQuantile(factory);
registerAggregateFunctionsSequenceMatch(factory);
registerAggregateFunctionWindowFunnel(factory);
registerAggregateFunctionRate(factory);
registerAggregateFunctionsMinMaxAny(factory);
registerAggregateFunctionsStatisticsStable(factory);
registerAggregateFunctionsStatisticsSimple(factory);

View File

@ -24,7 +24,7 @@ namespace DB
{
/// For cutting prerpocessed path to this base
std::string main_config_path;
static std::string main_config_path;
/// Extracts from a string the first encountered number consisting of at least two digits.
static std::string numberFromHost(const std::string & s)

View File

@ -402,6 +402,7 @@ namespace ErrorCodes
extern const int SYSTEM_ERROR = 425;
extern const int NULL_POINTER_DEREFERENCE = 426;
extern const int CANNOT_COMPILE_REGEXP = 427;
extern const int UNKNOWN_LOG_LEVEL = 428;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -27,7 +27,6 @@ bool callOnBasicType(TypeIndex number, F && f)
case TypeIndex::UInt16: return f(TypePair<T, UInt16>());
case TypeIndex::UInt32: return f(TypePair<T, UInt32>());
case TypeIndex::UInt64: return f(TypePair<T, UInt64>());
//case TypeIndex::UInt128>: return f(TypePair<T, UInt128>());
case TypeIndex::Int8: return f(TypePair<T, Int8>());
case TypeIndex::Int16: return f(TypePair<T, Int16>());
@ -35,6 +34,9 @@ bool callOnBasicType(TypeIndex number, F && f)
case TypeIndex::Int64: return f(TypePair<T, Int64>());
case TypeIndex::Int128: return f(TypePair<T, Int128>());
case TypeIndex::Enum8: return f(TypePair<T, Int8>());
case TypeIndex::Enum16: return f(TypePair<T, Int16>());
default:
break;
}
@ -89,13 +91,16 @@ inline bool callOnBasicTypes(TypeIndex type_num1, TypeIndex type_num2, F && f)
case TypeIndex::UInt16: return callOnBasicType<UInt16, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::UInt32: return callOnBasicType<UInt32, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::UInt64: return callOnBasicType<UInt64, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
//case TypeIndex::UInt128: return callOnBasicType<UInt128, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::Int8: return callOnBasicType<Int8, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::Int16: return callOnBasicType<Int16, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::Int32: return callOnBasicType<Int32, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::Int64: return callOnBasicType<Int64, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::Int128: return callOnBasicType<Int128, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::Enum8: return callOnBasicType<Int8, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
case TypeIndex::Enum16: return callOnBasicType<Int16, _int, _float, _decimal, _datetime>(type_num2, std::forward<F>(f));
default:
break;
}

View File

@ -143,15 +143,17 @@ void ExpressionAnalyzer::analyzeAggregation()
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(source_columns, context);
if (select_query && select_query->array_join_expression_list())
{
getRootActions(select_query->array_join_expression_list(), true, temp_actions);
addMultipleArrayJoinAction(temp_actions);
array_join_columns = temp_actions->getSampleBlock().getNamesAndTypesList();
}
if (select_query)
{
bool is_array_join_left;
ASTPtr array_join_expression_list = select_query->array_join_expression_list(is_array_join_left);
if (array_join_expression_list)
{
getRootActions(array_join_expression_list, true, temp_actions);
addMultipleArrayJoinAction(temp_actions, is_array_join_left);
array_join_columns = temp_actions->getSampleBlock().getNamesAndTypesList();
}
const ASTTablesInSelectQueryElement * join = select_query->join();
if (join)
{
@ -512,7 +514,7 @@ void ExpressionAnalyzer::initChain(ExpressionActionsChain & chain, const NamesAn
}
/// "Big" ARRAY JOIN.
void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const
void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool array_join_is_left) const
{
NameSet result_columns;
for (const auto & result_source : syntax->array_join_result_to_source)
@ -525,22 +527,24 @@ void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actio
result_columns.insert(result_source.first);
}
actions->add(ExpressionAction::arrayJoin(result_columns, select_query->array_join_is_left(), context));
actions->add(ExpressionAction::arrayJoin(result_columns, array_join_is_left, context));
}
bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types)
{
assertSelect();
if (!select_query->array_join_expression_list())
bool is_array_join_left;
ASTPtr array_join_expression_list = select_query->array_join_expression_list(is_array_join_left);
if (!array_join_expression_list)
return false;
initChain(chain, source_columns);
ExpressionActionsChain::Step & step = chain.steps.back();
getRootActions(select_query->array_join_expression_list(), only_types, step.actions);
getRootActions(array_join_expression_list, only_types, step.actions);
addMultipleArrayJoinAction(step.actions);
addMultipleArrayJoinAction(step.actions, is_array_join_left);
return true;
}

View File

@ -240,7 +240,7 @@ private:
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
void initGlobalSubqueriesAndExternalTables();
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const;
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const;
void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const;

View File

@ -8,6 +8,7 @@
#include <Parsers/ASTTablesInSelectQuery.h>
#include <DataTypes/NestedUtils.h>
#include <Common/typeid_cast.h>
#include "InDepthNodeVisitor.h"
namespace DB
{

View File

@ -277,7 +277,7 @@ struct Settings
M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.") \
M(SettingBool, log_query_settings, true, "Log query settings into the query_log.") \
M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.") \
M(SettingString, send_logs_level, "none", "Send server text logs with specified minumum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'") \
M(SettingLogsLevel, send_logs_level, "none", "Send server text logs with specified minumum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'") \
M(SettingBool, enable_optimize_predicate_expression, 0, "If it is set to true, optimize predicates to subqueries.") \
\
M(SettingUInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.") \

View File

@ -23,6 +23,7 @@ namespace ErrorCodes
extern const int UNKNOWN_DISTRIBUTED_PRODUCT_MODE;
extern const int UNKNOWN_GLOBAL_SUBQUERIES_METHOD;
extern const int UNKNOWN_JOIN_STRICTNESS;
extern const int UNKNOWN_LOG_LEVEL;
extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
}
@ -674,4 +675,58 @@ void SettingDateTimeInputFormat::write(WriteBuffer & buf) const
writeBinary(toString(), buf);
}
const std::vector<String> SettingLogsLevel::log_levels =
{
"none",
"trace",
"debug",
"information",
"warning",
"error"
};
SettingLogsLevel::SettingLogsLevel(const String & level)
{
set(level);
}
void SettingLogsLevel::set(const String & level)
{
auto it = std::find(log_levels.begin(), log_levels.end(), level);
if (it == log_levels.end())
throw Exception("Log level '" + level + "' not allowed.", ErrorCodes::UNKNOWN_LOG_LEVEL);
value = *it;
changed = true;
}
void SettingLogsLevel::set(const Field & level)
{
set(safeGet<String>(level));
}
void SettingLogsLevel::set(ReadBuffer & buf)
{
String x;
readBinary(x, buf);
set(x);
}
String SettingLogsLevel::toString() const
{
return value;
}
void SettingLogsLevel::write(WriteBuffer & buf) const
{
writeBinary(toString(), buf);
}
}

View File

@ -404,4 +404,25 @@ struct SettingDateTimeInputFormat
void write(WriteBuffer & buf) const;
};
class SettingLogsLevel
{
public:
String value;
bool changed = false;
static const std::vector<String> log_levels;
SettingLogsLevel(const String & level);
operator String() const { return value; }
void set(const String & level);
void set(const Field & level);
void set(ReadBuffer & buf);
String toString() const;
void write(WriteBuffer & buf) const;
};
}

View File

@ -676,9 +676,13 @@ void optimizeUsing(const ASTSelectQuery * select_query)
void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query,
const Names & source_columns, const NameSet & source_columns_set)
{
if (select_query && select_query->array_join_expression_list())
if (!select_query)
return;
ASTPtr array_join_expression_list = select_query->array_join_expression_list();
if (array_join_expression_list)
{
ASTs & array_join_asts = select_query->array_join_expression_list()->children;
ASTs & array_join_asts = array_join_expression_list->children;
for (const auto & ast : array_join_asts)
{
const String nested_table_name = ast->getColumnName();

View File

@ -1,6 +1,6 @@
#pragma once
#include <Parsers/IAST.h>
#include "IAST.h"
#include <Core/Field.h>
#include <Common/FieldVisitors.h>
@ -18,7 +18,7 @@ public:
ASTEnumElement(const String & name, const Field & value)
: name{name}, value {value} {}
String getID() const override { return "EnumElement"; }
String getID(char) const override { return "EnumElement"; }
ASTPtr clone() const override
{

View File

@ -283,23 +283,21 @@ bool ASTSelectQuery::final() const
}
ASTPtr ASTSelectQuery::array_join_expression_list() const
ASTPtr ASTSelectQuery::array_join_expression_list(bool & is_left) const
{
const ASTArrayJoin * array_join = getFirstArrayJoin(*this);
if (!array_join)
return {};
is_left = (array_join->kind == ASTArrayJoin::Kind::Left);
return array_join->expression_list;
}
bool ASTSelectQuery::array_join_is_left() const
ASTPtr ASTSelectQuery::array_join_expression_list() const
{
const ASTArrayJoin * array_join = getFirstArrayJoin(*this);
if (!array_join)
return {};
return array_join->kind == ASTArrayJoin::Kind::Left;
bool is_left;
return array_join_expression_list(is_left);
}

View File

@ -41,9 +41,9 @@ public:
/// Compatibility with old parser of tables list. TODO remove
ASTPtr sample_size() const;
ASTPtr sample_offset() const;
ASTPtr array_join_expression_list(bool & is_left) const;
ASTPtr array_join_expression_list() const;
const ASTTablesInSelectQueryElement * join() const;
bool array_join_is_left() const;
bool final() const;
void replaceDatabaseAndTable(const String & database_name, const String & table_name);
void addTableFunction(ASTPtr & table_function_ptr);

View File

@ -0,0 +1,21 @@
1
1
1.5
1.5
1.5
0 1.5
1 1.5
2 1.5
3 1.5
4 1.5
5 1.5
6 1.5
7 1.5
8 1.5
9 1.5
0 1.5
1.5
nan
nan
1

View File

@ -0,0 +1,26 @@
drop table if exists rate_test;
create table rate_test (timestamp UInt32, event UInt32) engine=Memory;
insert into rate_test values (0,1000),(1,1001),(2,1002),(3,1003),(4,1004),(5,1005),(6,1006),(7,1007),(8,1008);
select 1.0 = boundingRatio(timestamp, event) from rate_test;
drop table if exists rate_test2;
create table rate_test2 (uid UInt32 default 1,timestamp DateTime, event UInt32) engine=Memory;
insert into rate_test2(timestamp, event) values ('2018-01-01 01:01:01',1001),('2018-01-01 01:01:02',1002),('2018-01-01 01:01:03',1003),('2018-01-01 01:01:04',1004),('2018-01-01 01:01:05',1005),('2018-01-01 01:01:06',1006),('2018-01-01 01:01:07',1007),('2018-01-01 01:01:08',1008);
select 1.0 = boundingRatio(timestamp, event) from rate_test2;
drop table rate_test;
drop table rate_test2;
SELECT boundingRatio(number, number * 1.5) FROM numbers(10);
SELECT boundingRatio(1000 + number, number * 1.5) FROM numbers(10);
SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(10);
SELECT number % 10 AS k, boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(100) GROUP BY k WITH TOTALS ORDER BY k;
SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(2);
SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(1);
SELECT boundingRatio(1000 + number, number * 1.5 - 111) FROM numbers(1) WHERE 0;
SELECT boundingRatio(number, exp(number)) = e() - 1 FROM numbers(2);

View File

@ -1,4 +1,4 @@
select JavaHash('abc');
select JavaHash('874293087');
select HiveHash('abc');
select HiveHash('874293087');
select javaHash('abc');
select javaHash('874293087');
select hiveHash('abc');
select hiveHash('874293087');

View File

@ -0,0 +1,3 @@
0
foo
foo

View File

@ -0,0 +1,11 @@
DROP TABLE IF EXISTS test.alter_update;
CREATE TABLE test.alter_update (d Date, e Enum8('foo'=1, 'bar'=2)) Engine = MergeTree(d, (d), 8192);
INSERT INTO test.alter_update (d, e) VALUES ('2018-01-01', 'foo');
INSERT INTO test.alter_update (d, e) VALUES ('2018-01-02', 'bar');
ALTER TABLE test.alter_update UPDATE e = CAST('foo', 'Enum8(\'foo\' = 1, \'bar\' = 2)') WHERE d='2018-01-02';
SELECT sleep(1); -- TODO: there should be setting for sync ALTER UPDATE someday.
SELECT e FROM test.alter_update ORDER BY d;

View File

@ -169,7 +169,7 @@ When formatting, rows are enclosed in double quotes. A double quote inside a str
clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FORMAT CSV" < data.csv
```
&ast;By default, the delimiter is `,`. See the [format_csv_delimiter](/operations/settings/settings/#format_csv_delimiter) setting for more information.
&ast;By default, the delimiter is `,`. See the [format_csv_delimiter](../operations/settings/settings.md#format_csv_delimiter) setting for more information.
When parsing, all values can be parsed either with or without quotes. Both double and single quotes are supported. Rows can also be arranged without quotes. In this case, they are parsed up to the delimiter character or line feed (CR or LF). In violation of the RFC, when parsing rows without quotes, the leading and trailing spaces and tabs are ignored. For the line feed, Unix (LF), Windows (CR LF) and Mac OS Classic (CR LF) types are all supported.

View File

@ -28,6 +28,8 @@
- [Prometheus](https://prometheus.io/)
- [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter)
- [PromHouse](https://github.com/Percona-Lab/PromHouse)
- [Nagios](https://www.nagios.org/)
- [check_clickhouse](https://github.com/exogroup/check_clickhouse/)
- Logging
- [fluentd](https://www.fluentd.org)
- [loghouse](https://github.com/flant/loghouse) (for [Kubernetes](https://kubernetes.io))

View File

@ -681,8 +681,20 @@ For more information, see the section "[Replication](../../operations/table_engi
**Example**
```xml
<zookeeper incl="zookeeper-servers" optional="true" />
<zookeeper>
<node index="1">
<host>example1</host>
<port>2181</port>
</node>
<node index="2">
<host>example2</host>
<port>2181</port>
</node>
<node index="3">
<host>example3</host>
<port>2181</port>
</node>
</zookeeper>
```
[Original article](https://clickhouse.yandex/docs/en/operations/server_settings/settings/) <!--hide-->

View File

@ -149,7 +149,7 @@ Default value: 0 (off).
Used when performing `SELECT` from a distributed table that points to replicated tables.
## max_threads
## max_threads {#max_threads}
The maximum number of query processing threads

View File

@ -13,7 +13,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1],
name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2],
...
) ENGINE = MergeTree()
) ENGINE = SummingMergeTree()
[PARTITION BY expr]
[ORDER BY expr]
[SAMPLE BY expr]

View File

@ -4,8 +4,8 @@
This query is exactly the same as `CREATE`, but
- instead of the word `CREATE` it uses the word `ATTACH`.
- The query doesn't create data on the disk, but assumes that data is already in the appropriate places, and just adds information about the table to the server.
- Instead of the word `CREATE` it uses the word `ATTACH`.
- The query does not create data on the disk, but assumes that data is already in the appropriate places, and just adds information about the table to the server.
After executing an ATTACH query, the server will know about the existence of the table.
If the table was previously detached (``DETACH``), meaning that its structure is known, you can use shorthand without defining the structure.
@ -16,6 +16,41 @@ ATTACH TABLE [IF NOT EXISTS] [db.]name [ON CLUSTER cluster]
This query is used when starting the server. The server stores table metadata as files with `ATTACH` queries, which it simply runs at launch (with the exception of system tables, which are explicitly created on the server).
## CHECK TABLE
Checks if the data in the table is corrupted.
``` sql
CHECK TABLE [db.]name
```
The `CHECK TABLE` query compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution.
The query response contains the `result` column with a single row. The row has a value of
[Boolean](../data_types/boolean.md) type:
- 0 - The data in the table is corrupted.
- 1 - The data maintains integrity.
The `CHECK TABLE` query is only supported for the following table engines:
- [Log](../operations/table_engines/log.md)
- [TinyLog](../operations/table_engines/tinylog.md)
- StripeLog
These engines do not provide automatic data recovery on failure. Use the `CHECK TABLE` query to track data loss in a timely manner.
To avoid data loss use the [MergeTree](../operations/table_engines/mergetree.md) family tables.
**If the data is corrupted**
If the table is corrupted, you can copy the non-corrupted data to another table. To do this:
1. Create a new table with the same structure as damaged table. To do this execute the query `CREATE TABLE <new_table_name> AS <damaged_table_name>`.
2. Set the [max_threads](../operations/settings/settings.md#max_threads) value to 1 to process the next query in a single thread. To do this run the query `SET max_threads = 1`.
3. Execute the query `INSERT INTO <new_table_name> SELECT * FROM <damaged_table_name>`. This request copies the non-corrupted data from the damaged table to another table. Only the data before the corrupted part will be copied.
4. Restart the `clickhouse-client` to reset the `max_threads` value.
## DESCRIBE TABLE
``` sql
@ -198,8 +233,8 @@ SHOW [TEMPORARY] TABLES [FROM db] [LIKE 'pattern'] [INTO OUTFILE filename] [FORM
Displays a list of tables
- tables from the current database, or from the 'db' database if "FROM db" is specified.
- all tables, or tables whose name matches the pattern, if "LIKE 'pattern'" is specified.
- Tables from the current database, or from the 'db' database if "FROM db" is specified.
- All tables, or tables whose name matches the pattern, if "LIKE 'pattern'" is specified.
This query is identical to: `SELECT name FROM system.tables WHERE database = 'db' [AND name LIKE 'pattern'] [INTO OUTFILE filename] [FORMAT format]`.
@ -207,7 +242,7 @@ See also the section "LIKE operator".
## TRUNCATE
```sql
``` sql
TRUNCATE TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster]
```

View File

@ -34,6 +34,9 @@ inc="-I. \
-I./contrib/poco/Foundation/include \
-I./contrib/boost/libs/*/include \
-I./contrib/boost \
-I/usr/include/llvm-7 \
-I/usr/include/llvm-6.0 \
-I/usr/include/llvm-5.0 \
-I./contrib/llvm/llvm/include \
-I${BUILD_DIR}/contrib/llvm/llvm/include \
-I./contrib/libbtrie/include \