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

This commit is contained in:
BayoNet 2018-10-31 14:02:06 +03:00
commit aa8495dc55
63 changed files with 564 additions and 211 deletions

6
.gitignore vendored
View File

@ -9,7 +9,10 @@
# auto generated files # auto generated files
*.logrt *.logrt
dbms/src/Storages/System/StorageSystemContributors.generated.cpp
/build /build
/build_*
/docs/build /docs/build
/docs/edit /docs/edit
/docs/tools/venv/ /docs/tools/venv/
@ -243,3 +246,6 @@ website/presentations
website/package-lock.json website/package-lock.json
.DS_Store .DS_Store
*/.DS_Store */.DS_Store
# Ignore files for locally disabled tests
/dbms/tests/queries/**/*.disabled

View File

@ -1,3 +1,17 @@
## ClickHouse release 18.14.11, 2018-10-29
### Исправления ошибок:
* Исправлена ошибка `Block structure mismatch in UNION stream: different number of columns` в запросах с LIMIT. [#2156](https://github.com/yandex/ClickHouse/issues/2156)
* Исправлены ошибки при слиянии данных в таблицах, содержащих массивы внутри Nested структур. [#3397](https://github.com/yandex/ClickHouse/pull/3397)
* Исправлен неправильный результат запросов при выключенной настройке `merge_tree_uniform_read_distribution` (включена по умолчанию). [#3429](https://github.com/yandex/ClickHouse/pull/3429)
* Исправлена ошибка при вставке в Distributed таблицу в формате Native. [#3411](https://github.com/yandex/ClickHouse/issues/3411)
## ClickHouse release 18.14.10, 2018-10-23
* Настройка `compile_expressions` (JIT компиляция выражений) выключена по умолчанию. [#3410](https://github.com/yandex/ClickHouse/pull/3410)
* Настройка `enable_optimize_predicate_expression` выключена по умолчанию.
## ClickHouse release 18.14.9, 2018-10-16 ## ClickHouse release 18.14.9, 2018-10-16
### Новые возможности: ### Новые возможности:

View File

@ -86,4 +86,4 @@ if (ENABLE_ODBC)
endif () endif ()
endif () endif ()
message (STATUS "Using odbc: ${ODBC_INCLUDE_DIRECTORIES} : ${ODBC_LIBRARIES}") message (STATUS "Using odbc=${ODBC_FOUND}: ${ODBC_INCLUDE_DIRECTORIES} : ${ODBC_LIBRARIES}")

View File

@ -116,10 +116,10 @@ endif ()
if (Poco_MongoDB_LIBRARY) if (Poco_MongoDB_LIBRARY)
set (USE_POCO_MONGODB 1) set (USE_POCO_MONGODB 1)
endif () endif ()
if (Poco_DataODBC_LIBRARY) if (Poco_DataODBC_LIBRARY AND ODBC_FOUND)
set (USE_POCO_DATAODBC 1) set (USE_POCO_DATAODBC 1)
endif () endif ()
if (Poco_SQLODBC_LIBRARY) if (Poco_SQLODBC_LIBRARY AND ODBC_FOUND)
set (USE_POCO_SQLODBC 1) set (USE_POCO_SQLODBC 1)
endif () endif ()

2
contrib/ssl vendored

@ -1 +1 @@
Subproject commit de02224a42c69e3d8c9112c82018816f821878d0 Subproject commit 919f6f1331d500bfdd26f8bbbf88e92c0119879b

View File

@ -6,18 +6,12 @@ include(${CMAKE_CURRENT_SOURCE_DIR}/cmake/find_vectorclass.cmake)
set (CONFIG_VERSION ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config_version.h) set (CONFIG_VERSION ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config_version.h)
set (CONFIG_COMMON ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config.h) set (CONFIG_COMMON ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config.h)
set (CONFIG_BUILD ${CMAKE_CURRENT_BINARY_DIR}/src/Common/config_build.cpp)
include (cmake/version.cmake) include (cmake/version.cmake)
message (STATUS "Will build ${VERSION_FULL}") message (STATUS "Will build ${VERSION_FULL}")
configure_file (${CMAKE_CURRENT_SOURCE_DIR}/src/Common/config.h.in ${CONFIG_COMMON}) configure_file (${CMAKE_CURRENT_SOURCE_DIR}/src/Common/config.h.in ${CONFIG_COMMON})
configure_file (${CMAKE_CURRENT_SOURCE_DIR}/src/Common/config_version.h.in ${CONFIG_VERSION}) configure_file (${CMAKE_CURRENT_SOURCE_DIR}/src/Common/config_version.h.in ${CONFIG_VERSION})
get_property (BUILD_COMPILE_DEFINITIONS DIRECTORY ${ClickHouse_SOURCE_DIR} PROPERTY COMPILE_DEFINITIONS)
get_property (BUILD_INCLUDE_DIRECTORIES DIRECTORY ${ClickHouse_SOURCE_DIR} PROPERTY INCLUDE_DIRECTORIES)
string (TIMESTAMP BUILD_DATE "%Y-%m-%d" UTC)
configure_file (${CMAKE_CURRENT_SOURCE_DIR}/src/Common/config_build.cpp.in ${CONFIG_BUILD})
if (NOT MSVC) if (NOT MSVC)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra")
endif () endif ()

View File

@ -63,6 +63,8 @@ namespace ErrorCodes
extern const int TOO_BIG_AST; extern const int TOO_BIG_AST;
extern const int UNEXPECTED_AST_STRUCTURE; extern const int UNEXPECTED_AST_STRUCTURE;
extern const int SYNTAX_ERROR;
extern const int UNKNOWN_TABLE; extern const int UNKNOWN_TABLE;
extern const int UNKNOWN_FUNCTION; extern const int UNKNOWN_FUNCTION;
extern const int UNKNOWN_IDENTIFIER; extern const int UNKNOWN_IDENTIFIER;
@ -109,6 +111,8 @@ static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int excepti
exception_code == ErrorCodes::TOO_BIG_AST || exception_code == ErrorCodes::TOO_BIG_AST ||
exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE) exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE)
return HTTPResponse::HTTP_BAD_REQUEST; return HTTPResponse::HTTP_BAD_REQUEST;
else if (exception_code == ErrorCodes::SYNTAX_ERROR)
return HTTPResponse::HTTP_BAD_REQUEST;
else if (exception_code == ErrorCodes::UNKNOWN_TABLE || else if (exception_code == ErrorCodes::UNKNOWN_TABLE ||
exception_code == ErrorCodes::UNKNOWN_FUNCTION || exception_code == ErrorCodes::UNKNOWN_FUNCTION ||
exception_code == ErrorCodes::UNKNOWN_IDENTIFIER || exception_code == ErrorCodes::UNKNOWN_IDENTIFIER ||

View File

@ -80,7 +80,13 @@ void Connection::connect()
if (timeouts.tcp_keep_alive_timeout.totalSeconds()) if (timeouts.tcp_keep_alive_timeout.totalSeconds())
{ {
socket->setKeepAlive(true); socket->setKeepAlive(true);
socket->setOption(IPPROTO_TCP, TCP_KEEPIDLE, timeouts.tcp_keep_alive_timeout); socket->setOption(IPPROTO_TCP,
#if defined(TCP_KEEPALIVE)
TCP_KEEPALIVE
#else
TCP_KEEPIDLE // __APPLE__
#endif
, timeouts.tcp_keep_alive_timeout);
} }
in = std::make_shared<ReadBufferFromPocoSocket>(*socket); in = std::make_shared<ReadBufferFromPocoSocket>(*socket);

View File

@ -462,6 +462,8 @@ XMLDocumentPtr ConfigProcessor::processConfig(
std::string include_from_path; std::string include_from_path;
if (node) if (node)
{ {
/// if we include_from env or zk.
doIncludesRecursive(config, nullptr, node, zk_node_cache, contributing_zk_paths);
include_from_path = node->innerText(); include_from_path = node->innerText();
} }
else else

View File

@ -1,4 +0,0 @@
#pragma once
#include <vector>
extern const char * auto_config_build[];

View File

@ -33,9 +33,6 @@ RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, const
if (Protocol::Server::Data == packet.type) if (Protocol::Server::Data == packet.type)
{ {
header = packet.block; header = packet.block;
if (!header)
throw Exception("Logical error: empty block received as table structure", ErrorCodes::LOGICAL_ERROR);
break; break;
} }
else if (Protocol::Server::Exception == packet.type) else if (Protocol::Server::Exception == packet.type)
@ -58,7 +55,8 @@ RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, const
void RemoteBlockOutputStream::write(const Block & block) void RemoteBlockOutputStream::write(const Block & block)
{ {
assertBlocksHaveEqualStructure(block, header, "RemoteBlockOutputStream"); if (header)
assertBlocksHaveEqualStructure(block, header, "RemoteBlockOutputStream");
try try
{ {

View File

@ -2,6 +2,7 @@
#include <Interpreters/evaluateConstantExpression.h> #include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/convertFieldToType.h> #include <Interpreters/convertFieldToType.h>
#include <DataTypes/DataTypeArray.h>
#include <Parsers/TokenIterator.h> #include <Parsers/TokenIterator.h>
#include <Parsers/ExpressionListParsers.h> #include <Parsers/ExpressionListParsers.h>
#include <Formats/ValuesRowInputStream.h> #include <Formats/ValuesRowInputStream.h>
@ -29,6 +30,20 @@ namespace ErrorCodes
} }
bool is_array_type_compatible(const DataTypeArray & type, const Field & value)
{
if (type.getNestedType()->isNullable())
return true;
const Array & array = DB::get<const Array &>(value);
size_t size = array.size();
for (size_t i = 0; i < size; ++i)
if (array[i].isNull())
return false;
return true;
}
ValuesRowInputStream::ValuesRowInputStream(ReadBuffer & istr_, const Block & header_, const Context & context_, const FormatSettings & format_settings) ValuesRowInputStream::ValuesRowInputStream(ReadBuffer & istr_, const Block & header_, const Context & context_, const FormatSettings & format_settings)
: istr(istr_), header(header_), context(std::make_unique<Context>(context_)), format_settings(format_settings) : istr(istr_), header(header_), context(std::make_unique<Context>(context_)), format_settings(format_settings)
{ {
@ -116,14 +131,15 @@ bool ValuesRowInputStream::read(MutableColumns & columns)
std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(ast, *context); std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(ast, *context);
Field value = convertFieldToType(value_raw.first, type, value_raw.second.get()); Field value = convertFieldToType(value_raw.first, type, value_raw.second.get());
if (value.isNull()) const auto * array_type = typeid_cast<const DataTypeArray *>(&type);
/// Check that we are indeed allowed to insert a NULL.
if ((value.isNull() && !type.isNullable()) || (array_type && !is_array_type_compatible(*array_type, value)))
{ {
/// Check that we are indeed allowed to insert a NULL. throw Exception{"Expression returns value " + applyVisitor(FieldVisitorToString(), value)
if (!type.isNullable()) + ", that is out of range of type " + type.getName()
throw Exception{"Expression returns value " + applyVisitor(FieldVisitorToString(), value) + ", at: " + String(prev_istr_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - prev_istr_position)),
+ ", that is out of range of type " + type.getName() ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE};
+ ", at: " + String(prev_istr_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - prev_istr_position)),
ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE};
} }
columns[i]->insert(value); columns[i]->insert(value);

View File

@ -784,6 +784,9 @@ public:
} }
else if constexpr (to_decimal) else if constexpr (to_decimal)
{ {
if (!arguments[1].column)
throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
UInt64 scale = extractToDecimalScale(arguments[1]); UInt64 scale = extractToDecimalScale(arguments[1]);
if constexpr (std::is_same_v<Name, NameToDecimal32>) if constexpr (std::is_same_v<Name, NameToDecimal32>)

View File

@ -288,7 +288,10 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo
const ColumnWithTypeAndName & column = block.getByPosition(args[arg_num]); const ColumnWithTypeAndName & column = block.getByPosition(args[arg_num]);
if (arguments_to_remain_constants.end() != std::find(arguments_to_remain_constants.begin(), arguments_to_remain_constants.end(), arg_num)) if (arguments_to_remain_constants.end() != std::find(arguments_to_remain_constants.begin(), arguments_to_remain_constants.end(), arg_num))
temporary_block.insert(column); if (column.column->empty())
temporary_block.insert({column.column->cloneResized(1), column.type, column.name});
else
temporary_block.insert(column);
else else
{ {
have_converted_columns = true; have_converted_columns = true;
@ -311,7 +314,15 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo
executeWithoutLowCardinalityColumns(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows()); executeWithoutLowCardinalityColumns(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows());
block.getByPosition(result).column = ColumnConst::create(temporary_block.getByPosition(arguments_size).column, input_rows_count); ColumnPtr result_column;
/// extremely rare case, when we have function with completely const arguments
/// but some of them produced by non isDeterministic function
if (temporary_block.getByPosition(arguments_size).column->size() > 1)
result_column = temporary_block.getByPosition(arguments_size).column->cloneResized(1);
else
result_column = temporary_block.getByPosition(arguments_size).column;
block.getByPosition(result).column = ColumnConst::create(result_column, input_rows_count);
return true; return true;
} }

View File

@ -0,0 +1,64 @@
#pragma once
#include <Parsers/ASTQueryWithTableAndOutput.h>
#include <Parsers/ASTRenameQuery.h>
namespace DB
{
/// Visits AST nodes, add default database to DDLs if not set.
class AddDefaultDatabaseVisitor
{
public:
AddDefaultDatabaseVisitor(const String & default_database_)
: default_database(default_database_)
{}
void visit(ASTPtr & ast) const
{
visitChildren(ast);
if (!tryVisit<ASTQueryWithTableAndOutput>(ast) &&
!tryVisit<ASTRenameQuery>(ast))
{}
}
private:
const String default_database;
void visit(ASTQueryWithTableAndOutput * node, ASTPtr &) const
{
if (node->database.empty())
node->database = default_database;
}
void visit(ASTRenameQuery * node, ASTPtr &) const
{
for (ASTRenameQuery::Element & elem : node->elements)
{
if (elem.from.database.empty())
elem.from.database = default_database;
if (elem.to.database.empty())
elem.to.database = default_database;
}
}
void visitChildren(ASTPtr & ast) const
{
for (auto & child : ast->children)
visit(child);
}
template <typename T>
bool tryVisit(ASTPtr & ast) const
{
if (T * t = dynamic_cast<T *>(ast.get()))
{
visit(t, ast);
return true;
}
return false;
}
};
}

View File

@ -12,6 +12,7 @@
#include <DataStreams/IProfilingBlockInputStream.h> #include <DataStreams/IProfilingBlockInputStream.h>
#include <Interpreters/executeQuery.h> #include <Interpreters/executeQuery.h>
#include <Interpreters/Cluster.h> #include <Interpreters/Cluster.h>
#include <Interpreters/AddDefaultDatabaseVisitor.h>
#include <Common/DNSResolver.h> #include <Common/DNSResolver.h>
#include <Common/Macros.h> #include <Common/Macros.h>
#include <Common/getFQDNOrHostName.h> #include <Common/getFQDNOrHostName.h>
@ -39,6 +40,7 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_ELEMENT_IN_CONFIG; extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER; extern const int INVALID_CONFIG_PARAMETER;
extern const int UNKNOWN_FORMAT_VERSION; extern const int UNKNOWN_FORMAT_VERSION;
@ -1135,7 +1137,7 @@ private:
}; };
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & context, const NameSet & query_databases) BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & context, NameSet && query_databases)
{ {
/// Remove FORMAT <fmt> and INTO OUTFILE <file> if exists /// Remove FORMAT <fmt> and INTO OUTFILE <file> if exists
ASTPtr query_ptr = query_ptr_->clone(); ASTPtr query_ptr = query_ptr_->clone();
@ -1163,30 +1165,56 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont
ClusterPtr cluster = context.getCluster(query->cluster); ClusterPtr cluster = context.getCluster(query->cluster);
DDLWorker & ddl_worker = context.getDDLWorker(); DDLWorker & ddl_worker = context.getDDLWorker();
DDLLogEntry entry;
entry.query = queryToString(query_ptr);
entry.initiator = ddl_worker.getCommonHostID();
/// Check database access rights, assume that all servers have the same users config /// Check database access rights, assume that all servers have the same users config
NameSet databases_to_check_access_rights; NameSet databases_to_access;
const String & current_database = context.getCurrentDatabase();
Cluster::AddressesWithFailover shards = cluster->getShardsAddresses(); Cluster::AddressesWithFailover shards = cluster->getShardsAddresses();
std::vector<HostID> hosts;
bool use_shard_default_db = false;
bool use_local_default_db = false;
for (const auto & shard : shards) for (const auto & shard : shards)
{ {
for (const auto & addr : shard) for (const auto & addr : shard)
{ {
entry.hosts.emplace_back(addr); hosts.emplace_back(addr);
/// Expand empty database name to shards' default database name /// Expand empty database name to shards' default (o current) database name
for (const String & database : query_databases) for (const String & database : query_databases)
databases_to_check_access_rights.emplace(database.empty() ? addr.default_database : database); {
if (database.empty())
{
bool has_shard_default_db = !addr.default_database.empty();
use_shard_default_db |= has_shard_default_db;
use_local_default_db |= !has_shard_default_db;
databases_to_access.emplace(has_shard_default_db ? addr.default_database : current_database );
}
else
databases_to_access.emplace(database);
}
} }
} }
for (const String & database : databases_to_check_access_rights) if (use_shard_default_db && use_local_default_db)
context.checkDatabaseAccessRights(database.empty() ? context.getCurrentDatabase() : database); throw Exception("Mixed local default DB and shard default DB in DDL query", ErrorCodes::NOT_IMPLEMENTED);
if (databases_to_access.empty())
throw Exception("No databases to access in distributed DDL query", ErrorCodes::LOGICAL_ERROR);
for (const String & database : databases_to_access)
context.checkDatabaseAccessRights(database);
if (use_local_default_db)
{
AddDefaultDatabaseVisitor visitor(current_database);
visitor.visit(query_ptr);
}
DDLLogEntry entry;
entry.hosts = std::move(hosts);
entry.query = queryToString(query_ptr);
entry.initiator = ddl_worker.getCommonHostID();
String node_path = ddl_worker.enqueueQuery(entry); String node_path = ddl_worker.enqueueQuery(entry);
BlockIO io; BlockIO io;

View File

@ -20,7 +20,7 @@ struct DDLTask;
/// Pushes distributed DDL query to the queue /// Pushes distributed DDL query to the queue
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context, const NameSet & query_databases); BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context, NameSet && query_databases);
class DDLWorker class DDLWorker

View File

@ -206,8 +206,14 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings)
if (auto * prepared_function = dynamic_cast<PreparedFunctionImpl *>(function.get())) if (auto * prepared_function = dynamic_cast<PreparedFunctionImpl *>(function.get()))
prepared_function->createLowCardinalityResultCache(settings.max_threads); prepared_function->createLowCardinalityResultCache(settings.max_threads);
bool compile_expressions = false;
#if USE_EMBEDDED_COMPILER
compile_expressions = settings.compile_expressions;
#endif
/// If all arguments are constants, and function is suitable to be executed in 'prepare' stage - execute function. /// If all arguments are constants, and function is suitable to be executed in 'prepare' stage - execute function.
if (all_const && function_base->isSuitableForConstantFolding()) /// But if we compile expressions compiled version of this function maybe placed in cache,
/// so we don't want to unfold non deterministic functions
if (all_const && function_base->isSuitableForConstantFolding() && (!compile_expressions || function_base->isDeterministic()))
{ {
function->execute(sample_block, arguments, result_position, sample_block.rows()); function->execute(sample_block, arguments, result_position, sample_block.rows());
@ -671,7 +677,8 @@ void ExpressionActions::addImpl(ExpressionAction action, Names & new_names)
new_names.push_back(action.result_name); new_names.push_back(action.result_name);
new_names.insert(new_names.end(), action.array_joined_columns.begin(), action.array_joined_columns.end()); new_names.insert(new_names.end(), action.array_joined_columns.begin(), action.array_joined_columns.end());
if (action.type == ExpressionAction::APPLY_FUNCTION) /// Compiled functions are custom functions and them don't need building
if (action.type == ExpressionAction::APPLY_FUNCTION && !action.is_function_compiled)
{ {
if (sample_block.has(action.result_name)) if (sample_block.has(action.result_name))
throw Exception("Column '" + action.result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); throw Exception("Column '" + action.result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN);

View File

@ -275,55 +275,14 @@ bool ExpressionAnalyzer::isRemoteStorage() const
} }
static std::vector<ASTTableExpression> getTableExpressions(const ASTPtr & query)
{
ASTSelectQuery * select_query = typeid_cast<ASTSelectQuery *>(query.get());
std::vector<ASTTableExpression> table_expressions;
if (select_query && select_query->tables)
{
for (const auto & element : select_query->tables->children)
{
ASTTablesInSelectQueryElement & select_element = static_cast<ASTTablesInSelectQueryElement &>(*element);
if (select_element.table_expression)
table_expressions.emplace_back(static_cast<ASTTableExpression &>(*select_element.table_expression));
}
}
return table_expressions;
}
void ExpressionAnalyzer::translateQualifiedNames() void ExpressionAnalyzer::translateQualifiedNames()
{ {
if (!select_query || !select_query->tables || select_query->tables->children.empty()) if (!select_query || !select_query->tables || select_query->tables->children.empty())
return; return;
std::vector<DatabaseAndTableWithAlias> tables; std::vector<DatabaseAndTableWithAlias> tables = getDatabaseAndTableWithAliases(select_query, context.getCurrentDatabase());
std::vector<ASTTableExpression> tables_expression = getTableExpressions(query);
LogAST log; LogAST log;
for (const auto & table_expression : tables_expression)
{
auto table = getTableNameWithAliasFromTableExpression(table_expression, context.getCurrentDatabase());
{ /// debug print
size_t depth = 0;
DumpASTNode dump(table_expression, log.stream(), depth, "getTableNames");
if (table_expression.database_and_table_name)
DumpASTNode(*table_expression.database_and_table_name, log.stream(), depth);
if (table_expression.table_function)
DumpASTNode(*table_expression.table_function, log.stream(), depth);
if (table_expression.subquery)
DumpASTNode(*table_expression.subquery, log.stream(), depth);
dump.print("getTableNameWithAlias", table.database + '.' + table.table + ' ' + table.alias);
}
tables.emplace_back(table);
}
TranslateQualifiedNamesVisitor visitor(source_columns, tables, log.stream()); TranslateQualifiedNamesVisitor visitor(source_columns, tables, log.stream());
visitor.visit(query); visitor.visit(query);
} }
@ -602,13 +561,13 @@ void ExpressionAnalyzer::normalizeTree()
TableNamesAndColumnNames table_names_and_column_names; TableNamesAndColumnNames table_names_and_column_names;
if (select_query && select_query->tables && !select_query->tables->children.empty()) if (select_query && select_query->tables && !select_query->tables->children.empty())
{ {
std::vector<ASTTableExpression> tables_expression = getTableExpressions(query); std::vector<const ASTTableExpression *> tables_expression = getSelectTablesExpression(select_query);
bool first = true; bool first = true;
for (const auto & table_expression : tables_expression) for (const auto * table_expression : tables_expression)
{ {
const auto table_name = getTableNameWithAliasFromTableExpression(table_expression, context.getCurrentDatabase()); const auto table_name = getTableNameWithAliasFromTableExpression(*table_expression, context.getCurrentDatabase());
NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(table_expression, context); NamesAndTypesList names_and_types = getNamesAndTypeListFromTableExpression(*table_expression, context);
if (!first) if (!first)
{ {

View File

@ -27,7 +27,7 @@ BlockIO InterpreterAlterQuery::execute()
auto & alter = typeid_cast<ASTAlterQuery &>(*query_ptr); auto & alter = typeid_cast<ASTAlterQuery &>(*query_ptr);
if (!alter.cluster.empty()) if (!alter.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, context, {alter.table}); return executeDDLQueryOnCluster(query_ptr, context, {alter.database});
const String & table_name = alter.table; const String & table_name = alter.table;
String database_name = alter.database.empty() ? context.getCurrentDatabase() : alter.database; String database_name = alter.database.empty() ? context.getCurrentDatabase() : alter.database;

View File

@ -487,7 +487,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
if (!create.to_table.empty()) if (!create.to_table.empty())
databases.emplace(create.to_database); databases.emplace(create.to_database);
return executeDDLQueryOnCluster(query_ptr, context, databases); return executeDDLQueryOnCluster(query_ptr, context, std::move(databases));
} }
String path = context.getPath(); String path = context.getPath();

View File

@ -47,7 +47,7 @@ BlockIO InterpreterRenameQuery::execute()
databases.emplace(elem.to.database); databases.emplace(elem.to.database);
} }
return executeDDLQueryOnCluster(query_ptr, context, databases); return executeDDLQueryOnCluster(query_ptr, context, std::move(databases));
} }
String path = context.getPath(); String path = context.getPath();

View File

@ -44,11 +44,8 @@ bool PredicateExpressionsOptimizer::optimizeImpl(
/// split predicate with `and` /// split predicate with `and`
PredicateExpressions outer_predicate_expressions = splitConjunctionPredicate(outer_expression); PredicateExpressions outer_predicate_expressions = splitConjunctionPredicate(outer_expression);
std::vector<ASTTableExpression *> tables_expression = getSelectTablesExpression(ast_select); std::vector<DatabaseAndTableWithAlias> database_and_table_with_aliases =
std::vector<DatabaseAndTableWithAlias> database_and_table_with_aliases; getDatabaseAndTableWithAliases(ast_select, context.getCurrentDatabase());
for (const auto & table_expression : tables_expression)
database_and_table_with_aliases.emplace_back(
getTableNameWithAliasFromTableExpression(*table_expression, context.getCurrentDatabase()));
bool is_rewrite_subquery = false; bool is_rewrite_subquery = false;
for (const auto & outer_predicate : outer_predicate_expressions) for (const auto & outer_predicate : outer_predicate_expressions)
@ -336,7 +333,7 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que
if (!select_query->tables || select_query->tables->children.empty()) if (!select_query->tables || select_query->tables->children.empty())
return {}; return {};
std::vector<ASTTableExpression *> tables_expression = getSelectTablesExpression(select_query); std::vector<const ASTTableExpression *> tables_expression = getSelectTablesExpression(select_query);
if (const auto qualified_asterisk = typeid_cast<ASTQualifiedAsterisk *>(asterisk.get())) if (const auto qualified_asterisk = typeid_cast<ASTQualifiedAsterisk *>(asterisk.get()))
{ {
@ -406,25 +403,6 @@ ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_que
return projection_columns; return projection_columns;
} }
std::vector<ASTTableExpression *> PredicateExpressionsOptimizer::getSelectTablesExpression(ASTSelectQuery * select_query)
{
if (!select_query->tables)
return {};
std::vector<ASTTableExpression *> tables_expression;
const ASTTablesInSelectQuery & tables_in_select_query = static_cast<const ASTTablesInSelectQuery &>(*select_query->tables);
for (const auto & child : tables_in_select_query.children)
{
ASTTablesInSelectQueryElement * tables_element = static_cast<ASTTablesInSelectQueryElement *>(child.get());
if (tables_element->table_expression)
tables_expression.emplace_back(static_cast<ASTTableExpression *>(tables_element->table_expression.get()));
}
return tables_expression;
}
void PredicateExpressionsOptimizer::cleanExpressionAlias(ASTPtr & expression) void PredicateExpressionsOptimizer::cleanExpressionAlias(ASTPtr & expression)
{ {
const auto my_alias = expression->tryGetAlias(); const auto my_alias = expression->tryGetAlias();

View File

@ -105,8 +105,6 @@ private:
ASTs getSelectQueryProjectionColumns(ASTPtr & ast); ASTs getSelectQueryProjectionColumns(ASTPtr & ast);
std::vector<ASTTableExpression *> getSelectTablesExpression(ASTSelectQuery * select_query);
ASTs evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk); ASTs evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk);
void cleanExpressionAlias(ASTPtr & expression); void cleanExpressionAlias(ASTPtr & expression);

View File

@ -75,7 +75,7 @@ struct Settings
M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.") \ M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.") \
\ \
M(SettingBool, compile, false, "Whether query compilation is enabled.") \ M(SettingBool, compile, false, "Whether query compilation is enabled.") \
M(SettingBool, compile_expressions, false, "Compile some scalar functions and operators to native code.") \ M(SettingBool, compile_expressions, true, "Compile some scalar functions and operators to native code.") \
M(SettingUInt64, min_count_to_compile, 3, "The number of structurally identical queries before they are compiled.") \ M(SettingUInt64, min_count_to_compile, 3, "The number of structurally identical queries before they are compiled.") \
M(SettingUInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.") \ M(SettingUInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.") \
M(SettingUInt64, group_by_two_level_threshold_bytes, 100000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.") \ M(SettingUInt64, group_by_two_level_threshold_bytes, 100000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.") \

View File

@ -5,6 +5,7 @@
#include <Parsers/IAST.h> #include <Parsers/IAST.h>
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTSelectQuery.h>
namespace DB namespace DB
{ {
@ -164,4 +165,35 @@ void DatabaseAndTableWithAlias::makeQualifiedName(const ASTPtr & ast) const
} }
} }
std::vector<const ASTTableExpression *> getSelectTablesExpression(const ASTSelectQuery * select_query)
{
if (!select_query->tables)
return {};
std::vector<const ASTTableExpression *> tables_expression;
for (const auto & child : select_query->tables->children)
{
ASTTablesInSelectQueryElement * tables_element = static_cast<ASTTablesInSelectQueryElement *>(child.get());
if (tables_element->table_expression)
tables_expression.emplace_back(static_cast<const ASTTableExpression *>(tables_element->table_expression.get()));
}
return tables_expression;
}
std::vector<DatabaseAndTableWithAlias> getDatabaseAndTableWithAliases(const ASTSelectQuery * select_query, const String & current_database)
{
std::vector<const ASTTableExpression *> tables_expression = getSelectTablesExpression(select_query);
std::vector<DatabaseAndTableWithAlias> database_and_table_with_aliases;
database_and_table_with_aliases.reserve(tables_expression.size());
for (const auto & table_expression : tables_expression)
database_and_table_with_aliases.emplace_back(getTableNameWithAliasFromTableExpression(*table_expression, current_database));
return database_and_table_with_aliases;
}
} }

View File

@ -9,6 +9,7 @@ namespace DB
class IAST; class IAST;
using ASTPtr = std::shared_ptr<IAST>; using ASTPtr = std::shared_ptr<IAST>;
class ASTSelectQuery;
class ASTIdentifier; class ASTIdentifier;
struct ASTTableExpression; struct ASTTableExpression;
@ -36,4 +37,7 @@ size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifie
std::pair<String, String> getDatabaseAndTableNameFromIdentifier(const ASTIdentifier & identifier); std::pair<String, String> getDatabaseAndTableNameFromIdentifier(const ASTIdentifier & identifier);
std::vector<const ASTTableExpression *> getSelectTablesExpression(const ASTSelectQuery * select_query);
std::vector<DatabaseAndTableWithAlias> getDatabaseAndTableWithAliases(const ASTSelectQuery * select_query, const String & current_database);
} }

View File

@ -187,18 +187,6 @@ ASTPtr ASTAlterQuery::clone() const
return res; return res;
} }
ASTPtr ASTAlterQuery::getRewrittenASTWithoutOnCluster(const std::string & new_database) const
{
auto query_ptr = clone();
auto & query = static_cast<ASTAlterQuery &>(*query_ptr);
query.cluster.clear();
if (query.database.empty())
query.database = new_database;
return query_ptr;
}
void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{ {
frame.need_parens = false; frame.need_parens = false;

View File

@ -122,7 +122,10 @@ public:
ASTPtr clone() const override; ASTPtr clone() const override;
ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override; ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override
{
return removeOnCluster<ASTAlterQuery>(clone(), new_database);
}
protected: protected:
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;

View File

@ -112,14 +112,7 @@ public:
ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override
{ {
auto query_ptr = clone(); return removeOnCluster<ASTCreateQuery>(clone(), new_database);
ASTCreateQuery & query = static_cast<ASTCreateQuery &>(*query_ptr);
query.cluster.clear();
if (query.database.empty())
query.database = new_database;
return query_ptr;
} }
protected: protected:

View File

@ -29,18 +29,6 @@ ASTPtr ASTDropQuery::clone() const
return res; return res;
} }
ASTPtr ASTDropQuery::getRewrittenASTWithoutOnCluster(const std::string & new_database) const
{
auto query_ptr = clone();
auto & query = static_cast<ASTDropQuery &>(*query_ptr);
query.cluster.clear();
if (query.database.empty())
query.database = new_database;
return query_ptr;
}
void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{ {
settings.ostr << (settings.hilite ? hilite_keyword : ""); settings.ostr << (settings.hilite ? hilite_keyword : "");

View File

@ -26,7 +26,10 @@ public:
String getID() const override; String getID() const override;
ASTPtr clone() const override; ASTPtr clone() const override;
ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override; ASTPtr getRewrittenASTWithoutOnCluster(const std::string & new_database) const override
{
return removeOnCluster<ASTDropQuery>(clone(), new_database);
}
protected: protected:
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override;

View File

@ -8,16 +8,6 @@ String ASTKillQueryQuery::getID() const
return "KillQueryQuery_" + (where_expression ? where_expression->getID() : "") + "_" + String(sync ? "SYNC" : "ASYNC"); return "KillQueryQuery_" + (where_expression ? where_expression->getID() : "") + "_" + String(sync ? "SYNC" : "ASYNC");
} }
ASTPtr ASTKillQueryQuery::getRewrittenASTWithoutOnCluster(const std::string & /*new_database*/) const
{
auto query_ptr = clone();
ASTKillQueryQuery & query = static_cast<ASTKillQueryQuery &>(*query_ptr);
query.cluster.clear();
return query_ptr;
}
void ASTKillQueryQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const void ASTKillQueryQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{ {
settings.ostr << (settings.hilite ? hilite_keyword : "") << "KILL QUERY "; settings.ostr << (settings.hilite ? hilite_keyword : "") << "KILL QUERY ";

View File

@ -24,7 +24,10 @@ public:
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const std::string &new_database) const override; ASTPtr getRewrittenASTWithoutOnCluster(const std::string &) const override
{
return removeOnCluster<ASTKillQueryQuery>(clone());
}
}; };
} }

View File

@ -3,19 +3,6 @@
namespace DB namespace DB
{ {
ASTPtr ASTOptimizeQuery::getRewrittenASTWithoutOnCluster(const std::string & new_database) const
{
auto query_ptr = clone();
ASTOptimizeQuery & query = static_cast<ASTOptimizeQuery &>(*query_ptr);
query.cluster.clear();
if (query.database.empty())
query.database = new_database;
return query_ptr;
}
void ASTOptimizeQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const void ASTOptimizeQuery::formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{ {
settings.ostr << (settings.hilite ? hilite_keyword : "") << "OPTIMIZE TABLE " << (settings.hilite ? hilite_none : "") settings.ostr << (settings.hilite ? hilite_keyword : "") << "OPTIMIZE TABLE " << (settings.hilite ? hilite_none : "")

View File

@ -40,7 +40,10 @@ public:
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
ASTPtr getRewrittenASTWithoutOnCluster(const std::string &new_database) const override; ASTPtr getRewrittenASTWithoutOnCluster(const std::string &new_database) const override
{
return removeOnCluster<ASTOptimizeQuery>(clone(), new_database);
}
}; };
} }

View File

@ -28,6 +28,27 @@ public:
static bool parse(Pos & pos, std::string & cluster_str, Expected & expected); static bool parse(Pos & pos, std::string & cluster_str, Expected & expected);
virtual ~ASTQueryWithOnCluster() = default; virtual ~ASTQueryWithOnCluster() = default;
protected:
template <typename T>
static ASTPtr removeOnCluster(ASTPtr query_ptr, const std::string & new_database)
{
T & query = static_cast<T &>(*query_ptr);
query.cluster.clear();
if (query.database.empty())
query.database = new_database;
return query_ptr;
}
template <typename T>
static ASTPtr removeOnCluster(ASTPtr query_ptr)
{
T & query = static_cast<T &>(*query_ptr);
query.cluster.clear();
return query_ptr;
}
}; };
} }

View File

@ -1,5 +1,16 @@
if (NOT EXISTS ${CMAKE_CURRENT_SOURCE_DIR}/StorageSystemContributors.generated.cpp)
execute_process(COMMAND ${CMAKE_CURRENT_SOURCE_DIR}/StorageSystemContributors.sh)
endif()
set (CONFIG_BUILD ${CMAKE_CURRENT_BINARY_DIR}/StorageSystemBuildOptions.generated.cpp)
get_property (BUILD_COMPILE_DEFINITIONS DIRECTORY ${ClickHouse_SOURCE_DIR} PROPERTY COMPILE_DEFINITIONS)
get_property (BUILD_INCLUDE_DIRECTORIES DIRECTORY ${ClickHouse_SOURCE_DIR} PROPERTY INCLUDE_DIRECTORIES)
string (TIMESTAMP BUILD_DATE "%Y-%m-%d" UTC)
configure_file (StorageSystemBuildOptions.generated.cpp.in ${CONFIG_BUILD})
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
add_headers_and_sources(storages_system .) add_headers_and_sources(storages_system .)
list (APPEND storages_system_sources ${CONFIG_BUILD})
add_library(clickhouse_storages_system ${LINK_MODE} ${storages_system_headers} ${storages_system_sources}) add_library(clickhouse_storages_system ${LINK_MODE} ${storages_system_headers} ${storages_system_sources})
target_link_libraries(clickhouse_storages_system dbms) target_link_libraries(clickhouse_storages_system dbms)
target_include_directories(clickhouse_storages_system PRIVATE ${CMAKE_CURRENT_SOURCE_DIR})

View File

@ -1,7 +1,9 @@
#include <Common/config_build.h> #include "StorageSystemBuildOptions.h"
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
#include <Interpreters/Settings.h> #include <Interpreters/Settings.h>
#include <Storages/System/StorageSystemBuildOptions.h>
extern const char * auto_config_build[];
namespace DB namespace DB
{ {

View File

@ -1,7 +1,5 @@
// .cpp autogenerated by cmake // .cpp autogenerated by cmake
#include <Common/config_build.h>
const char * auto_config_build[] const char * auto_config_build[]
{ {
"VERSION_FULL", "@VERSION_FULL@", "VERSION_FULL", "@VERSION_FULL@",

View File

@ -0,0 +1,36 @@
#if __has_include("StorageSystemContributors.generated.cpp")
#include "StorageSystemContributors.h"
#include <algorithm>
#include <pcg_random.hpp>
#include <DataTypes/DataTypeString.h>
#include <Common/randomSeed.h>
extern const char * auto_contributors[];
namespace DB
{
NamesAndTypesList StorageSystemContributors::getNamesAndTypes()
{
return {
{"name", std::make_shared<DataTypeString>()},
};
}
void StorageSystemContributors::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const
{
std::vector<const char *> contributors;
for (auto it = auto_contributors; *it; ++it)
contributors.emplace_back(*it);
pcg64 rng(randomSeed());
std::shuffle(contributors.begin(), contributors.end(), rng);
for (auto & it : contributors)
res_columns[0]->insert(String(it));
}
}
#endif

View File

@ -0,0 +1,34 @@
#pragma once
#if __has_include("StorageSystemContributors.generated.cpp")
#include <Storages/System/IStorageSystemOneBlock.h>
#include <ext/shared_ptr_helper.h>
namespace DB
{
class Context;
/** System table "contributors" with list of clickhouse contributors
*/
class StorageSystemContributors : public ext::shared_ptr_helper<StorageSystemContributors>,
public IStorageSystemOneBlock<StorageSystemContributors>
{
protected:
void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const override;
using IStorageSystemOneBlock::IStorageSystemOneBlock;
public:
std::string getName() const override
{
return "SystemContributors";
}
static NamesAndTypesList getNamesAndTypes();
};
}
#endif

View File

@ -0,0 +1,19 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
CONTRIBUTORS_FILE=${CONTRIBUTORS_FILE=$CUR_DIR/StorageSystemContributors.generated.cpp}
git shortlog --summary | perl -lnE 's/^\s+\d+\s+(.+)/"$1",/; next unless $1; say $_' > $CONTRIBUTORS_FILE.tmp
# If git history not available - dont make target file
if [ ! -s $CONTRIBUTORS_FILE.tmp ]; then
exit
fi
echo "// autogenerated by $0" > $CONTRIBUTORS_FILE
echo "const char * auto_contributors[] {" >> $CONTRIBUTORS_FILE
cat $CONTRIBUTORS_FILE.tmp >> $CONTRIBUTORS_FILE
echo "nullptr };" >> $CONTRIBUTORS_FILE
rm $CONTRIBUTORS_FILE.tmp

View File

@ -32,6 +32,9 @@
#include <Storages/System/StorageSystemTableFunctions.h> #include <Storages/System/StorageSystemTableFunctions.h>
#include <Storages/System/StorageSystemTables.h> #include <Storages/System/StorageSystemTables.h>
#include <Storages/System/StorageSystemZooKeeper.h> #include <Storages/System/StorageSystemZooKeeper.h>
#if __has_include("StorageSystemContributors.generated.cpp")
# include <Storages/System/StorageSystemContributors.h>
#endif
namespace DB namespace DB
@ -56,6 +59,9 @@ void attachSystemTablesLocal(IDatabase & system_database)
system_database.attachTable("data_type_families", StorageSystemDataTypeFamilies::create("data_type_families")); system_database.attachTable("data_type_families", StorageSystemDataTypeFamilies::create("data_type_families"));
system_database.attachTable("collations", StorageSystemCollations::create("collations")); system_database.attachTable("collations", StorageSystemCollations::create("collations"));
system_database.attachTable("table_engines", StorageSystemTableEngines::create("table_engines")); system_database.attachTable("table_engines", StorageSystemTableEngines::create("table_engines"));
#if __has_include("StorageSystemContributors.generated.cpp")
system_database.attachTable("contributors", StorageSystemContributors::create("contributors"));
#endif
} }
void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper) void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper)

View File

@ -119,16 +119,23 @@ String transformQueryForExternalDatabase(
{ {
if (function->name == "and") if (function->name == "and")
{ {
bool compatible_found = false;
auto new_function_and = std::make_shared<ASTFunction>(); auto new_function_and = std::make_shared<ASTFunction>();
auto new_function_and_arguments = std::make_shared<ASTExpressionList>(); auto new_function_and_arguments = std::make_shared<ASTExpressionList>();
new_function_and->arguments = new_function_and_arguments; new_function_and->arguments = new_function_and_arguments;
new_function_and->children.push_back(new_function_and_arguments); new_function_and->children.push_back(new_function_and_arguments);
for (const auto & elem : function->arguments->children) for (const auto & elem : function->arguments->children)
{
if (isCompatible(*elem)) if (isCompatible(*elem))
{
new_function_and_arguments->children.push_back(elem); new_function_and_arguments->children.push_back(elem);
compatible_found = true;
}
}
select->where_expression = std::move(new_function_and); if (compatible_found)
select->where_expression = std::move(new_function_and);
} }
} }
} }

View File

@ -27,7 +27,7 @@ if (ENABLE_TESTS)
# maybe add --no-long ? # maybe add --no-long ?
# if you want disable some tests: env TEST_OPT0='--skip compile' # if you want disable some tests: env TEST_OPT0='--skip compile'
add_test(NAME with_server COMMAND bash -c "env BUILD_DIR=${ClickHouse_BINARY_DIR} INTERNAL_COMPILER_BIN_ROOT=${INTERNAL_COMPILER_BIN_ROOT} ${CMAKE_CURRENT_SOURCE_DIR}/clickhouse-test-server") add_test(NAME with_server COMMAND bash -c "env BUILD_DIR=${ClickHouse_BINARY_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/clickhouse-test-server")
endif () endif ()
if (ENABLE_TEST_INTEGRATION) if (ENABLE_TEST_INTEGRATION)

View File

@ -242,7 +242,7 @@ def main(args):
stderr_element = et.Element("system-err") stderr_element = et.Element("system-err")
stderr_element.text = et.CDATA(stderr) stderr_element.text = et.CDATA(stderr)
report_testcase.append(stderr_element) report_testcase.append(stderr_element)
print(stderr) print(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

View File

@ -141,10 +141,17 @@ def generate_structure(args):
base_name = 'range_hashed_' + range_hashed_range_type base_name = 'range_hashed_' + range_hashed_range_type
dictionaries.extend([ dictionaries.extend([
[ 'file_' + base_name, 3, False ], [ 'file_' + base_name, 3, False ],
# [ 'clickhouse_' + base_name, 3, True ], [ 'clickhouse_' + base_name, 3, False ],
# [ 'executable_flat' + base_name, 3, True ] # [ 'executable_flat' + base_name, 3, True ]
]) ])
if not args.no_mysql:
for range_hashed_range_type in range_hashed_range_types:
base_name = 'range_hashed_' + range_hashed_range_type
dictionaries.extend([
['mysql_' + base_name, 3, False],
])
files = [ 'key_simple.tsv', 'key_complex_integers.tsv', 'key_complex_mixed.tsv', 'key_range_hashed_{range_hashed_range_type}.tsv' ] files = [ 'key_simple.tsv', 'key_complex_integers.tsv', 'key_complex_mixed.tsv', 'key_range_hashed_{range_hashed_range_type}.tsv' ]
@ -206,6 +213,36 @@ range_hashed_dictGet_values = {
("toDateTime('2015-11-19 23:59:59')", "toDateTime('2015-10-26 00:00:01')", "toDateTime('2018-09-14 00:00:00')")], ("toDateTime('2015-11-19 23:59:59')", "toDateTime('2015-10-26 00:00:01')", "toDateTime('2018-09-14 00:00:00')")],
} }
range_hashed_mysql_column_types = {
'UInt8': 'tinyint unsigned',
'UInt16': 'smallint unsigned',
'UInt32': 'int unsigned',
'UInt64': 'bigint unsigned',
'Int8': 'tinyint',
'Int16': 'smallint',
'Int32': 'int',
'Int64': 'bigint',
# default type (Date) for compatibility with older versions:
'': 'date',
'Date': 'date',
'DateTime': 'datetime',
}
range_hashed_clickhouse_column_types = {
'UInt8': 'UInt8',
'UInt16': 'UInt16',
'UInt32': 'UInt32',
'UInt64': 'UInt64',
'Int8': 'Int8',
'Int16': 'Int16',
'Int32': 'Int32',
'Int64': 'Int64',
# default type (Date) for compatibility with older versions:
'': 'Date',
'Date': 'Date',
'DateTime': 'DateTime',
}
def dump_report(destination, suite, test_case, report): def dump_report(destination, suite, test_case, report):
if destination is not None: if destination is not None:
@ -268,6 +305,41 @@ def generate_data(args):
query = file_source_query % comma_separated(chain(keys, columns(), ['Parent'] if 1 == len(keys) else [])) query = file_source_query % comma_separated(chain(keys, columns(), ['Parent'] if 1 == len(keys) else []))
call([args.client, '--port', args.port, '--query', query], 'generated/' + file) call([args.client, '--port', args.port, '--query', query], 'generated/' + file)
table_name = "test.dictionary_source_" + range_hashed_range_type
col_type = range_hashed_clickhouse_column_types[range_hashed_range_type]
source_tsv_full_path = "{0}/generated/{1}".format(prefix, file)
print 'Creating Clickhouse table for "{0}" range_hashed dictionary...'.format(range_hashed_range_type)
system('cat {source} | {ch} --port={port} -m -n --query "'
'create database if not exists test;'
'drop table if exists {table_name};'
'create table {table_name} ('
'id UInt64, StartDate {col_type}, EndDate {col_type},'
'UInt8_ UInt8, UInt16_ UInt16, UInt32_ UInt32, UInt64_ UInt64,'
'Int8_ Int8, Int16_ Int16, Int32_ Int32, Int64_ Int64,'
'Float32_ Float32, Float64_ Float64,'
'String_ String,'
'Date_ Date, DateTime_ DateTime, UUID_ UUID'
') engine=Log; insert into {table_name} format TabSeparated'
'"'.format(table_name=table_name, col_type=col_type, source=source_tsv_full_path, ch=args.client, port=args.port))
if not args.no_mysql:
print 'Creating MySQL table for "{0}" range_hashed dictionary...'.format(range_hashed_range_type)
col_type = range_hashed_mysql_column_types[range_hashed_range_type]
subprocess.check_call('echo "'
'create database if not exists test;'
'drop table if exists {table_name};'
'create table {table_name} ('
'id tinyint unsigned, StartDate {col_type}, EndDate {col_type}, '
'UInt8_ tinyint unsigned, UInt16_ smallint unsigned, UInt32_ int unsigned, UInt64_ bigint unsigned, '
'Int8_ tinyint, Int16_ smallint, Int32_ int, Int64_ bigint, '
'Float32_ float, Float64_ double, '
'String_ text, Date_ date, DateTime_ datetime, UUID_ varchar(36)'
');'
'load data local infile \'{source}\' into table {table_name};" | mysql $MYSQL_OPTIONS --local-infile=1'
.format(prefix, table_name=table_name, col_type=col_type, source=source_tsv_full_path), shell=True)
# create MySQL table from complete_query # create MySQL table from complete_query
if not args.no_mysql: if not args.no_mysql:
print 'Creating MySQL table' print 'Creating MySQL table'
@ -365,7 +437,7 @@ def generate_dictionaries(args):
<user>default</user> <user>default</user>
<password></password> <password></password>
<db>test</db> <db>test</db>
<table>dictionary_source</table> <table>dictionary_source{key_type}</table>
</clickhouse> </clickhouse>
''' % args.port ''' % args.port
@ -384,7 +456,7 @@ def generate_dictionaries(args):
<user>root</user> <user>root</user>
<password></password> <password></password>
<db>test</db> <db>test</db>
<table>dictionary_source</table> <table>dictionary_source{key_type}</table>
</mysql> </mysql>
''' '''
@ -518,33 +590,34 @@ def generate_dictionaries(args):
</attribute> </attribute>
''' '''
source_clickhouse_deafult = source_clickhouse.format(key_type="")
sources_and_layouts = [ sources_and_layouts = [
# Simple key dictionaries # Simple key dictionaries
[ source_file % (generated_prefix + files[0]), layout_flat], [ source_file % (generated_prefix + files[0]), layout_flat],
[ source_clickhouse, layout_flat ], [ source_clickhouse_deafult, layout_flat ],
[ source_executable % (generated_prefix + files[0]), layout_flat ], [ source_executable % (generated_prefix + files[0]), layout_flat ],
[ source_file % (generated_prefix + files[0]), layout_hashed], [ source_file % (generated_prefix + files[0]), layout_hashed],
[ source_clickhouse, layout_hashed ], [ source_clickhouse_deafult, layout_hashed ],
[ source_executable % (generated_prefix + files[0]), layout_hashed ], [ source_executable % (generated_prefix + files[0]), layout_hashed ],
[ source_clickhouse, layout_cache ], [ source_clickhouse_deafult, layout_cache ],
[ source_executable_cache % (generated_prefix + files[0]), layout_cache ], [ source_executable_cache % (generated_prefix + files[0]), layout_cache ],
# Complex key dictionaries with (UInt8, UInt8) key # Complex key dictionaries with (UInt8, UInt8) key
[ source_file % (generated_prefix + files[1]), layout_complex_key_hashed], [ source_file % (generated_prefix + files[1]), layout_complex_key_hashed],
[ source_clickhouse, layout_complex_key_hashed ], [ source_clickhouse_deafult, layout_complex_key_hashed ],
[ source_executable % (generated_prefix + files[1]), layout_complex_key_hashed ], [ source_executable % (generated_prefix + files[1]), layout_complex_key_hashed ],
[ source_clickhouse, layout_complex_key_cache ], [ source_clickhouse_deafult, layout_complex_key_cache ],
[ source_executable_cache % (generated_prefix + files[1]), layout_complex_key_cache ], [ source_executable_cache % (generated_prefix + files[1]), layout_complex_key_cache ],
# Complex key dictionaries with (String, UInt8) key # Complex key dictionaries with (String, UInt8) key
[ source_file % (generated_prefix + files[2]), layout_complex_key_hashed], [ source_file % (generated_prefix + files[2]), layout_complex_key_hashed],
[ source_clickhouse, layout_complex_key_hashed ], [ source_clickhouse_deafult, layout_complex_key_hashed ],
[ source_executable % (generated_prefix + files[2]), layout_complex_key_hashed ], [ source_executable % (generated_prefix + files[2]), layout_complex_key_hashed ],
[ source_clickhouse, layout_complex_key_cache ], [ source_clickhouse_deafult, layout_complex_key_cache ],
[ source_executable_cache % (generated_prefix + files[2]), layout_complex_key_cache ], [ source_executable_cache % (generated_prefix + files[2]), layout_complex_key_cache ],
] ]
@ -568,14 +641,15 @@ def generate_dictionaries(args):
]) ])
if not args.no_mysql: if not args.no_mysql:
source_mysql_default = source_mysql.format(key_type="")
sources_and_layouts.extend([ sources_and_layouts.extend([
[ source_mysql, layout_flat ], [ source_mysql_default, layout_flat ],
[ source_mysql, layout_hashed ], [ source_mysql_default, layout_hashed ],
[ source_mysql, layout_cache ], [ source_mysql_default, layout_cache ],
[ source_mysql, layout_complex_key_hashed ], [ source_mysql_default, layout_complex_key_hashed ],
[ source_mysql, layout_complex_key_cache ], [ source_mysql_default, layout_complex_key_cache ],
[ source_mysql, layout_complex_key_hashed ], [ source_mysql_default, layout_complex_key_hashed ],
[ source_mysql, layout_complex_key_cache ], [ source_mysql_default, layout_complex_key_cache ],
]) ])
if not args.no_mongo: if not args.no_mongo:
@ -613,16 +687,29 @@ def generate_dictionaries(args):
]) ])
for range_hashed_range_type in range_hashed_range_types: for range_hashed_range_type in range_hashed_range_types:
key_type = "_" + range_hashed_range_type
sources_and_layouts.extend([ sources_and_layouts.extend([
[ source_file % (generated_prefix + (files[3].format(range_hashed_range_type=range_hashed_range_type))), (layout_range_hashed, range_hashed_range_type) ], [ source_file % (generated_prefix + (files[3].format(range_hashed_range_type=range_hashed_range_type))), (layout_range_hashed, range_hashed_range_type) ],
# [ source_clickhouse, layout_range_hashed ], [ source_clickhouse.format(key_type=key_type), (layout_range_hashed, range_hashed_range_type) ],
# [ source_executable, layout_range_hashed ] # [ source_executable, layout_range_hashed ]
]) ])
if not args.no_mysql:
for range_hashed_range_type in range_hashed_range_types:
key_type = "_" + range_hashed_range_type
source_mysql_typed = source_mysql.format(key_type=key_type)
sources_and_layouts.extend([
[source_mysql_typed,
(layout_range_hashed, range_hashed_range_type)],
])
dict_name_filter = args.filter.split('/')[0] if args.filter else None
for (name, key_idx, has_parent), (source, layout) in zip(dictionaries, sources_and_layouts): for (name, key_idx, has_parent), (source, layout) in zip(dictionaries, sources_and_layouts):
if args.filter and not fnmatch.fnmatch(name, dict_name_filter):
continue
filename = os.path.join(args.generated, 'dictionary_%s.xml' % name) filename = os.path.join(args.generated, 'dictionary_%s.xml' % name)
key = keys[key_idx] key = keys[key_idx]
if key_idx == 3: if key_idx == 3:
layout, range_hashed_range_type = layout layout, range_hashed_range_type = layout
# Wrap non-empty type (default) with <type> tag. # Wrap non-empty type (default) with <type> tag.
@ -670,7 +757,7 @@ def run_tests(args):
global SERVER_DIED global SERVER_DIED
print "{0:100}".format('Dictionary: ' + dict + ' Name: ' + name + ": "), print "{0:100}".format('Dictionary: ' + dict + ' Name: ' + name + ": "),
if args.filter and not fnmatch.fnmatch(dict, args.filter) and not fnmatch.fnmatch(name, args.filter): if args.filter and not fnmatch.fnmatch(dict + "/" + name, args.filter):
print " ... skipped due to filter." print " ... skipped due to filter."
return return

View File

@ -0,0 +1,15 @@
<yandex>
<include_from from_env="INCLUDE_FROM_ENV"></include_from>
<profiles>
<default>
<max_query_size incl="mqs" />
</default>
</profiles>
<users>
<default>
<password></password>
<profile>default</profile>
<quota>default</quota>
</default>
</users>
</yandex>

View File

@ -9,6 +9,7 @@ node2 = cluster.add_instance('node2', user_configs=['configs/config_env.xml'], e
node3 = cluster.add_instance('node3', user_configs=['configs/config_zk.xml'], with_zookeeper=True) node3 = cluster.add_instance('node3', user_configs=['configs/config_zk.xml'], with_zookeeper=True)
node4 = cluster.add_instance('node4', user_configs=['configs/config_incl.xml'], main_configs=['configs/max_query_size.xml']) # include value 77777 node4 = cluster.add_instance('node4', user_configs=['configs/config_incl.xml'], main_configs=['configs/max_query_size.xml']) # include value 77777
node5 = cluster.add_instance('node5', user_configs=['configs/config_allow_databases.xml']) node5 = cluster.add_instance('node5', user_configs=['configs/config_allow_databases.xml'])
node6 = cluster.add_instance('node6', user_configs=['configs/config_include_from_env.xml'], env_variables={"INCLUDE_FROM_ENV": "/etc/clickhouse-server/config.d/max_query_size.xml"}, main_configs=['configs/max_query_size.xml'])
@pytest.fixture(scope="module") @pytest.fixture(scope="module")
def start_cluster(): def start_cluster():
@ -27,6 +28,7 @@ def test_config(start_cluster):
assert node2.query("select value from system.settings where name = 'max_query_size'") == "55555\n" assert node2.query("select value from system.settings where name = 'max_query_size'") == "55555\n"
assert node3.query("select value from system.settings where name = 'max_query_size'") == "77777\n" assert node3.query("select value from system.settings where name = 'max_query_size'") == "77777\n"
assert node4.query("select value from system.settings where name = 'max_query_size'") == "99999\n" assert node4.query("select value from system.settings where name = 'max_query_size'") == "99999\n"
assert node6.query("select value from system.settings where name = 'max_query_size'") == "99999\n"
def test_allow_databases(start_cluster): def test_allow_databases(start_cluster):
node5.query("CREATE DATABASE db1") node5.query("CREATE DATABASE db1")

View File

@ -3,4 +3,4 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh . $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="SELECT * FROM system.build_options" | perl -lnE 'print $1 if /(BUILD_DATE|BUILD_TYPE|CXX_COMPILER|CXX_FLAGS|LINK_FLAGS)\s+\S+/'; $CLICKHOUSE_CLIENT --query="SELECT * FROM system.build_options" | perl -lnE 'print $1 if /(BUILD_DATE|BUILD_TYPE|CXX_COMPILER)\s+\S+/ || /(CXX_FLAGS|LINK_FLAGS)/';

View File

@ -18,14 +18,18 @@ $chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(31), 1)"
$chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(1), 2)" $chl "INSERT INTO test.partition_428 (p, k) VALUES(toDate(1), 2)"
for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do
sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt | wc -l # 2 header lines + 3 columns # 2 header lines + 3 columns
(sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt 2>/dev/null || \
cat $ch_dir/data/test/partition_428/$part/columns.txt) | wc -l
done done
$chl "ALTER TABLE test.partition_428 DETACH PARTITION 197001" $chl "ALTER TABLE test.partition_428 DETACH PARTITION 197001"
$chl "ALTER TABLE test.partition_428 ATTACH PARTITION 197001" $chl "ALTER TABLE test.partition_428 ATTACH PARTITION 197001"
for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do for part in `$chl "SELECT name FROM system.parts WHERE database='test' AND table='partition_428'"`; do
sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt | wc -l # 2 header lines + 3 columns # 2 header lines + 3 columns
(sudo -n cat $ch_dir/data/test/partition_428/$part/columns.txt 2>/dev/null || \
cat $ch_dir/data/test/partition_428/$part/columns.txt) | wc -l
done done
$chl "ALTER TABLE test.partition_428 MODIFY COLUMN v1 Int8" $chl "ALTER TABLE test.partition_428 MODIFY COLUMN v1 Int8"

View File

@ -239,4 +239,8 @@ SELECT toUInt64('2147483649') AS x, toDecimal32(x, 0); -- { serverError 407 }
SELECT toUInt64('9223372036854775807') AS x, toDecimal64(x, 0); SELECT toUInt64('9223372036854775807') AS x, toDecimal64(x, 0);
SELECT toUInt64('9223372036854775809') AS x, toDecimal64(x, 0); -- { serverError 407 } SELECT toUInt64('9223372036854775809') AS x, toDecimal64(x, 0); -- { serverError 407 }
SELECT toDecimal32(0, rowNumberInBlock()); -- { serverError 44 }
SELECT toDecimal64(0, rowNumberInBlock()); -- { serverError 44 }
SELECT toDecimal128(0, rowNumberInBlock()); -- { serverError 44 }
DROP TABLE IF EXISTS test.decimal; DROP TABLE IF EXISTS test.decimal;

View File

@ -0,0 +1,20 @@
SET compile_expressions = 1;
SET min_count_to_compile = 1;
DROP TABLE IF EXISTS test.time_table;
CREATE TABLE test.time_table(timecol DateTime, value Int32) ENGINE = MergeTree order by tuple();
INSERT INTO test.time_table VALUES (now() - 5, 5), (now() - 3, 3);
SELECT COUNT() from test.time_table WHERE value < now() - 1 AND value != 0 AND modulo(value, 2) != 0 AND timecol < now() - 1;
SELECT sleep(3);
INSERT INTO test.time_table VALUES (now(), 101);
SELECT sleep(3);
SELECT COUNT() from test.time_table WHERE value < now() - 1 AND value != 0 AND modulo(value, 2) != 0 AND timecol < now() - 1;
DROP TABLE IF EXISTS test.time_table;

View File

@ -12,7 +12,7 @@ $CLICKHOUSE_CLIENT -q "select name from system.table_functions format TSV;" > $S
# This is short run for ordinary tests. # This is short run for ordinary tests.
# if you want long run use: env SQL_FUZZY_RUNS=100000 clickhouse-test sql_fuzzy # if you want long run use: env SQL_FUZZY_RUNS=100000 clickhouse-test sql_fuzzy
for SQL_FUZZY_RUN in $(seq ${SQL_FUZZY_RUNS:=100}); do for SQL_FUZZY_RUN in $(seq ${SQL_FUZZY_RUNS:=10}); do
env SQL_FUZZY_RUN=$SQL_FUZZY_RUN $CURDIR/00746_sql_fuzzy.pl | $CLICKHOUSE_CLIENT -n --ignore-error >/dev/null 2>&1 env SQL_FUZZY_RUN=$SQL_FUZZY_RUN $CURDIR/00746_sql_fuzzy.pl | $CLICKHOUSE_CLIENT -n --ignore-error >/dev/null 2>&1
if [[ `$CLICKHOUSE_CLIENT -q "SELECT 'Still alive'"` != 'Still alive' ]]; then if [[ `$CLICKHOUSE_CLIENT -q "SELECT 'Still alive'"` != 'Still alive' ]]; then
break break

View File

@ -0,0 +1 @@
ok

View File

@ -0,0 +1,2 @@
-- Normally table should contain 250+ contributors. But when fast git clone used (--depth=X) (Travis build) table will contain only <=X contributors
SELECT if ((SELECT count(*) FROM system.contributors) > 1, 'ok', 'fail');

View File

@ -6,3 +6,4 @@ SELECT extractURLParameter('?_', '\0');
SELECT extractURLParameter('ZiqSZeh?', '\0') SELECT extractURLParameter('ZiqSZeh?', '\0')
SELECT globalNotIn(['"wh'], [NULL]); SELECT globalNotIn(['"wh'], [NULL]);
SELECT globalIn([''], [NULL]) SELECT globalIn([''], [NULL])
SELECT ( SELECT toDecimal128([], rowNumberInBlock()) ) , lcm('', [[(CAST(('>A') AS String))]]);

View File

@ -26,4 +26,4 @@ ENV LANG en_US.UTF-8
ENV LANGUAGE en_US:en ENV LANGUAGE en_US:en
ENV LC_ALL en_US.UTF-8 ENV LC_ALL en_US.UTF-8
CMD ["/usr/bin/clickhouse-client"] ENTRYPOINT ["/usr/bin/clickhouse-client"]

View File

@ -10,7 +10,6 @@
#include <readline/history.h> #include <readline/history.h>
#elif USE_LIBEDIT #elif USE_LIBEDIT
#include <editline/readline.h> #include <editline/readline.h>
#include <editline/history.h> // Y_IGNORE
#else #else
#include <string> #include <string>
#include <cstring> #include <cstring>

View File

@ -107,6 +107,8 @@ echo -e "\nCurrent version is $VERSION_STRING"
gen_changelog "$VERSION_STRING" "" "$AUTHOR" "" gen_changelog "$VERSION_STRING" "" "$AUTHOR" ""
$CURDIR/dbms/src/Storages/System/StorageSystemContributors.sh
if [ -z "$USE_PBUILDER" ] ; then if [ -z "$USE_PBUILDER" ] ; then
DEB_CC=${DEB_CC:=`which gcc-7 gcc-8 gcc | head -n1`} DEB_CC=${DEB_CC:=`which gcc-7 gcc-8 gcc | head -n1`}
DEB_CXX=${DEB_CXX:=`which g++-7 g++-8 g++ | head -n1`} DEB_CXX=${DEB_CXX:=`which g++-7 g++-8 g++ | head -n1`}