Merge branch 'master' into brotli

This commit is contained in:
Mike F 2019-02-02 20:14:29 +03:00 committed by GitHub
commit 723d3b3a99
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
64 changed files with 1363 additions and 414 deletions

2
.gitmodules vendored
View File

@ -64,6 +64,8 @@
[submodule "contrib/cppkafka"]
path = contrib/cppkafka
url = https://github.com/mfontanini/cppkafka.git
[submodule "contrib/pdqsort"]
path = contrib/pdqsort
[submodule "contrib/brotli"]
path = contrib/brotli
url = https://github.com/google/brotli.git

View File

@ -254,6 +254,7 @@ include (cmake/find_libgsasl.cmake)
include (cmake/find_libxml2.cmake)
include (cmake/find_brotli.cmake)
include (cmake/find_protobuf.cmake)
include (cmake/find_pdqsort.cmake)
include (cmake/find_hdfs3.cmake)
include (cmake/find_consistent-hashing.cmake)
include (cmake/find_base64.cmake)

2
cmake/find_pdqsort.cmake Normal file
View File

@ -0,0 +1,2 @@
set(PDQSORT_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/pdqsort)
message(STATUS "Using pdqsort: ${PDQSORT_INCLUDE_DIR}")

View File

@ -1,5 +1,11 @@
option(USE_INTERNAL_PROTOBUF_LIBRARY "Set to FALSE to use system protobuf instead of bundled" ${NOT_UNBUNDLED})
if(OS_FREEBSD AND SANITIZE STREQUAL "address")
# ../contrib/protobuf/src/google/protobuf/arena_impl.h:45:10: fatal error: 'sanitizer/asan_interface.h' file not found
set(MISSING_INTERNAL_PROTOBUF_LIBRARY 1)
set(USE_INTERNAL_PROTOBUF_LIBRARY 0)
endif()
if(NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/protobuf/cmake/CMakeLists.txt")
if(USE_INTERNAL_PROTOBUF_LIBRARY)
message(WARNING "submodule contrib/protobuf is missing. to fix try run: \n git submodule update --init --recursive")

1
contrib/pdqsort vendored Submodule

@ -0,0 +1 @@
Subproject commit 08879029ab8dcb80a70142acb709e3df02de5d37

View File

@ -206,6 +206,8 @@ target_link_libraries (clickhouse_common_io
${CMAKE_DL_LIBS}
)
target_include_directories(clickhouse_common_io SYSTEM BEFORE PUBLIC ${PDQSORT_INCLUDE_DIR})
target_include_directories(clickhouse_common_io SYSTEM BEFORE PUBLIC ${RE2_INCLUDE_DIR})
if(CPUID_LIBRARY)
@ -282,6 +284,7 @@ target_link_libraries (dbms PRIVATE ${Poco_Foundation_LIBRARY})
if (USE_ICU)
target_link_libraries (dbms PRIVATE ${ICU_LIBRARIES})
target_include_directories (dbms SYSTEM PRIVATE ${ICU_INCLUDE_DIRS})
endif ()
if (USE_CAPNP)

View File

@ -8,7 +8,7 @@
#include <Common/Exception.h>
#include <IO/ConnectionTimeouts.h>
#include <common/SetTerminalEcho.h>
#include <common/setTerminalEcho.h>
#include <ext/scope_guard.h>
#include <Poco/Util/AbstractConfiguration.h>
@ -56,10 +56,10 @@ struct ConnectionParameters
throw Exception("Specified both --password and --ask-password. Remove one of them", ErrorCodes::BAD_ARGUMENTS);
std::cout << "Password for user " << user << ": ";
SetTerminalEcho(false);
setTerminalEcho(false);
SCOPE_EXIT({
SetTerminalEcho(true);
setTerminalEcho(true);
});
std::getline(std::cin, password);
std::cout << std::endl;

View File

@ -297,7 +297,7 @@ void LocalServer::processQueries()
try
{
executeQuery(read_buf, write_buf, /* allow_into_outfile = */ true, *context, {});
executeQuery(read_buf, write_buf, /* allow_into_outfile = */ true, *context, {}, {});
}
catch (...)
{

View File

@ -128,6 +128,32 @@ UInt64 PerformanceTest::calculateMaxExecTime() const
return result;
}
void PerformanceTest::prepare() const
{
for (const auto & query : test_info.create_queries)
{
LOG_INFO(log, "Executing create query '" << query << "'");
connection.sendQuery(query);
}
for (const auto & query : test_info.fill_queries)
{
LOG_INFO(log, "Executing fill query '" << query << "'");
connection.sendQuery(query);
}
}
void PerformanceTest::finish() const
{
for (const auto & query : test_info.drop_queries)
{
LOG_INFO(log, "Executing drop query '" << query << "'");
connection.sendQuery(query);
}
}
std::vector<TestStats> PerformanceTest::execute()
{
std::vector<TestStats> statistics_by_run;

View File

@ -25,12 +25,15 @@ public:
Context & context_);
bool checkPreconditions() const;
void prepare() const;
std::vector<TestStats> execute();
void finish() const;
const PerformanceTestInfo & getTestInfo() const
{
return test_info;
}
bool checkSIGINT() const
{
return got_SIGINT;

View File

@ -90,6 +90,7 @@ PerformanceTestInfo::PerformanceTestInfo(
getExecutionType(config);
getStopConditions(config);
getMetrics(config);
extractAuxiliaryQueries(config);
}
void PerformanceTestInfo::applySettings(XMLConfigurationPtr config)
@ -269,4 +270,16 @@ void PerformanceTestInfo::getMetrics(XMLConfigurationPtr config)
checkMetricsInput(metrics, exec_type);
}
void PerformanceTestInfo::extractAuxiliaryQueries(XMLConfigurationPtr config)
{
if (config->has("create_query"))
create_queries = getMultipleValuesFromConfig(*config, "", "create_query");
if (config->has("fill_query"))
fill_queries = getMultipleValuesFromConfig(*config, "", "fill_query");
if (config->has("drop_query"))
drop_queries = getMultipleValuesFromConfig(*config, "", "drop_query");
}
}

View File

@ -43,6 +43,10 @@ public:
std::string profiles_file;
std::vector<TestStopConditions> stop_conditions_by_run;
Strings create_queries;
Strings fill_queries;
Strings drop_queries;
private:
void applySettings(XMLConfigurationPtr config);
void extractQueries(XMLConfigurationPtr config);
@ -50,6 +54,7 @@ private:
void getExecutionType(XMLConfigurationPtr config);
void getStopConditions(XMLConfigurationPtr config);
void getMetrics(XMLConfigurationPtr config);
void extractAuxiliaryQueries(XMLConfigurationPtr config);
};
}

View File

@ -202,11 +202,18 @@ private:
current.checkPreconditions();
LOG_INFO(log, "Preconditions for test '" << info.test_name << "' are fullfilled");
LOG_INFO(log, "Preparing for run, have " << info.create_queries.size()
<< " create queries and " << info.fill_queries.size() << " fill queries");
current.prepare();
LOG_INFO(log, "Prepared");
LOG_INFO(log, "Running test '" << info.test_name << "'");
auto result = current.execute();
LOG_INFO(log, "Test '" << info.test_name << "' finished");
LOG_INFO(log, "Running post run queries");
current.finish();
LOG_INFO(log, "Postqueries finished");
if (lite_output)
return {report_builder->buildCompactReport(info, result), current.checkSIGINT()};
else

View File

@ -563,7 +563,8 @@ void HTTPHandler::processQuery(
context.setProgressCallback([&used_output] (const Progress & progress) { used_output.out->onProgress(progress); });
executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context,
[&response] (const String & content_type) { response.setContentType(content_type); });
[&response] (const String & content_type) { response.setContentType(content_type); },
[&response] (const String & current_query_id) { response.add("Query-Id", current_query_id); });
if (used_output.hasDelayed())
{

View File

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

View File

@ -12,6 +12,7 @@
#include <Columns/ColumnsCommon.h>
#include <DataStreams/ColumnGathererStream.h>
#include <ext/bit_cast.h>
#include <pdqsort.h>
#ifdef __SSE2__
#include <emmintrin.h>
@ -90,9 +91,9 @@ void ColumnVector<T>::getPermutation(bool reverse, size_t limit, int nan_directi
else
{
if (reverse)
std::sort(res.begin(), res.end(), greater(*this, nan_direction_hint));
pdqsort(res.begin(), res.end(), greater(*this, nan_direction_hint));
else
std::sort(res.begin(), res.end(), less(*this, nan_direction_hint));
pdqsort(res.begin(), res.end(), less(*this, nan_direction_hint));
}
}

View File

@ -120,17 +120,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery)
if (!done_with_join)
{
for (const auto & name_with_alias : subquery.joined_block_aliases)
{
if (block.has(name_with_alias.first))
{
auto pos = block.getPositionByName(name_with_alias.first);
auto column = block.getByPosition(pos);
block.erase(pos);
column.name = name_with_alias.second;
block.insert(std::move(column));
}
}
subquery.renameColumns(block);
if (subquery.joined_block_actions)
subquery.joined_block_actions->execute(block);

View File

@ -36,6 +36,7 @@ endif ()
if (USE_ICU)
target_link_libraries (clickhouse_functions PRIVATE ${ICU_LIBRARIES})
target_include_directories(clickhouse_functions SYSTEM PRIVATE ${ICU_INCLUDE_DIRS})
endif ()
if (USE_VECTORCLASS)

View File

@ -1,111 +0,0 @@
#include <IO/InterserverWriteBuffer.h>
#include <IO/WriteBufferFromOStream.h>
#include <Poco/Version.h>
#include <Poco/URI.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Net/HTTPResponse.h>
#include <common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_WRITE_TO_OSTREAM;
extern const int RECEIVED_ERROR_FROM_REMOTE_IO_SERVER;
}
InterserverWriteBuffer::InterserverWriteBuffer(const std::string & host_, int port_,
const std::string & endpoint_,
const std::string & path_,
bool compress_,
size_t buffer_size_,
const Poco::Timespan & connection_timeout,
const Poco::Timespan & send_timeout,
const Poco::Timespan & receive_timeout)
: WriteBuffer(nullptr, 0), host(host_), port(port_), path(path_)
{
std::string encoded_path;
Poco::URI::encode(path, "&#", encoded_path);
std::string encoded_endpoint;
Poco::URI::encode(endpoint_, "&#", encoded_endpoint);
std::string compress_str = compress_ ? "true" : "false";
std::string encoded_compress;
Poco::URI::encode(compress_str, "&#", encoded_compress);
std::stringstream uri;
uri << "http://" << host << ":" << port
<< "/?endpoint=" << encoded_endpoint
<< "&compress=" << encoded_compress
<< "&path=" << encoded_path;
std::string uri_str = Poco::URI(uri.str()).getPathAndQuery();
session.setHost(host);
session.setPort(port);
session.setKeepAlive(true);
/// set the timeout
#if POCO_CLICKHOUSE_PATCH || POCO_VERSION >= 0x02000000
session.setTimeout(connection_timeout, send_timeout, receive_timeout);
#else
session.setTimeout(connection_timeout);
static_cast <void> (send_timeout);
static_cast <void> (receive_timeout);
#endif
Poco::Net::HTTPRequest request(Poco::Net::HTTPRequest::HTTP_POST, uri_str, Poco::Net::HTTPRequest::HTTP_1_1);
request.setChunkedTransferEncoding(true);
ostr = &session.sendRequest(request);
impl = std::make_unique<WriteBufferFromOStream>(*ostr, buffer_size_);
set(impl->buffer().begin(), impl->buffer().size());
}
InterserverWriteBuffer::~InterserverWriteBuffer()
{
try
{
finalize();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void InterserverWriteBuffer::nextImpl()
{
if (!offset() || finalized)
return;
/// For correct work with AsynchronousWriteBuffer, which replaces buffers.
impl->set(buffer().begin(), buffer().size());
impl->position() = pos;
impl->next();
}
void InterserverWriteBuffer::finalize()
{
if (finalized)
return;
next();
finalized = true;
}
void InterserverWriteBuffer::cancel()
{
finalized = true;
}
}

View File

@ -1,54 +0,0 @@
#pragma once
#include <IO/WriteBuffer.h>
#include <IO/HashingWriteBuffer.h>
#include <Poco/Net/HTTPClientSession.h>
namespace DB
{
namespace
{
constexpr auto DEFAULT_REMOTE_WRITE_BUFFER_CONNECTION_TIMEOUT = 1;
constexpr auto DEFAULT_REMOTE_WRITE_BUFFER_RECEIVE_TIMEOUT = 1800;
constexpr auto DEFAULT_REMOTE_WRITE_BUFFER_SEND_TIMEOUT = 1800;
}
/** Allows you to write a file to a remote server.
*/
class InterserverWriteBuffer final : public WriteBuffer
{
public:
InterserverWriteBuffer(const std::string & host_, int port_,
const std::string & endpoint_,
const std::string & path_,
bool compress_ = false,
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
const Poco::Timespan & connection_timeout = Poco::Timespan(DEFAULT_REMOTE_WRITE_BUFFER_CONNECTION_TIMEOUT, 0),
const Poco::Timespan & send_timeout = Poco::Timespan(DEFAULT_REMOTE_WRITE_BUFFER_SEND_TIMEOUT, 0),
const Poco::Timespan & receive_timeout = Poco::Timespan(DEFAULT_REMOTE_WRITE_BUFFER_RECEIVE_TIMEOUT, 0));
~InterserverWriteBuffer() override;
void finalize();
void cancel();
private:
void nextImpl() override;
private:
std::string host;
int port;
std::string path;
Poco::Net::HTTPClientSession session;
std::ostream * ostr; /// this is owned by session
std::unique_ptr<WriteBuffer> impl;
/// Sent all the data and renamed the file
bool finalized = false;
};
}

View File

@ -3,6 +3,7 @@
#include <Parsers/IAST.h>
#include <Interpreters/PreparedSets.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/SubqueryForSet.h>
namespace DB
@ -11,32 +12,6 @@ namespace DB
class Context;
class ASTFunction;
class Join;
using JoinPtr = std::shared_ptr<Join>;
/// Information on what to do when executing a subquery in the [GLOBAL] IN/JOIN section.
struct SubqueryForSet
{
/// The source is obtained using the InterpreterSelectQuery subquery.
BlockInputStreamPtr source;
/// If set, build it from result.
SetPtr set;
JoinPtr join;
/// Apply this actions to joined block.
ExpressionActionsPtr joined_block_actions;
/// Rename column from joined block from this list.
NamesWithAliases joined_block_aliases;
/// If set, put the result into the table.
/// This is a temporary table for transferring to remote servers for distributed query processing.
StoragePtr table;
};
/// ID of subquery -> what to do with it.
using SubqueriesForSets = std::unordered_map<String, SubqueryForSet>;
/// The case of an explicit enumeration of values.
SetPtr makeExplicitSet(
const ASTFunction * node, const Block & sample_block, bool create_ordered_set,

View File

@ -0,0 +1,129 @@
#include <Common/typeid_cast.h>
#include <Interpreters/CrossToInnerJoinVisitor.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ParserTablesInSelectQuery.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/// TODO: array join aliases?
struct CheckColumnsVisitorData
{
using TypeToVisit = ASTIdentifier;
const std::vector<DatabaseAndTableWithAlias> & tables;
size_t visited;
size_t found;
size_t allMatch() const { return visited == found; }
void visit(ASTIdentifier & node, ASTPtr &)
{
++visited;
for (const auto & t : tables)
if (IdentifierSemantic::canReferColumnToTable(node, t))
++found;
}
};
static bool extractTableName(const ASTTableExpression & expr, std::vector<DatabaseAndTableWithAlias> & names)
{
/// Subselects are not supported.
if (!expr.database_and_table_name)
return false;
names.emplace_back(DatabaseAndTableWithAlias(expr));
return true;
}
static ASTPtr getCrossJoin(ASTSelectQuery & select, std::vector<DatabaseAndTableWithAlias> & table_names)
{
if (!select.tables)
return {};
auto tables = typeid_cast<const ASTTablesInSelectQuery *>(select.tables.get());
if (!tables)
return {};
size_t num_tables = tables->children.size();
if (num_tables != 2)
return {};
auto left = typeid_cast<const ASTTablesInSelectQueryElement *>(tables->children[0].get());
auto right = typeid_cast<const ASTTablesInSelectQueryElement *>(tables->children[1].get());
if (!left || !right || !right->table_join)
return {};
if (auto join = typeid_cast<const ASTTableJoin *>(right->table_join.get()))
{
if (join->kind == ASTTableJoin::Kind::Cross ||
join->kind == ASTTableJoin::Kind::Comma)
{
if (!join->children.empty())
throw Exception("Logical error: CROSS JOIN has expressions", ErrorCodes::LOGICAL_ERROR);
auto & left_expr = typeid_cast<const ASTTableExpression &>(*left->table_expression);
auto & right_expr = typeid_cast<const ASTTableExpression &>(*right->table_expression);
table_names.reserve(2);
if (extractTableName(left_expr, table_names) &&
extractTableName(right_expr, table_names))
return right->table_join;
}
}
return {};
}
std::vector<ASTPtr *> CrossToInnerJoinMatcher::visit(ASTPtr & ast, Data & data)
{
if (auto * t = typeid_cast<ASTSelectQuery *>(ast.get()))
visit(*t, ast, data);
return {};
}
void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr & ast, Data & data)
{
using CheckColumnsMatcher = OneTypeMatcher<CheckColumnsVisitorData>;
using CheckColumnsVisitor = InDepthNodeVisitor<CheckColumnsMatcher, true>;
std::vector<DatabaseAndTableWithAlias> table_names;
ASTPtr ast_join = getCrossJoin(select, table_names);
if (!ast_join)
return;
/// check Identifier names from where expression
CheckColumnsVisitor::Data columns_data{table_names, 0, 0};
CheckColumnsVisitor(columns_data).visit(select.where_expression);
if (!columns_data.allMatch())
return;
auto & join = typeid_cast<ASTTableJoin &>(*ast_join);
join.kind = ASTTableJoin::Kind::Inner;
join.strictness = ASTTableJoin::Strictness::All; /// TODO: do we need it?
join.on_expression.swap(select.where_expression);
join.children.push_back(join.on_expression);
ast = ast->clone(); /// rewrite AST in right manner
data.done = true;
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
{
class ASTSelectQuery;
/// AST transformer. It replaces cross joins with equivalented inner join if possible.
class CrossToInnerJoinMatcher
{
public:
struct Data
{
bool done = false;
};
static constexpr const char * label = "JoinToSubqueryTransform";
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; }
static std::vector<ASTPtr *> visit(ASTPtr & ast, Data & data);
private:
static void visit(ASTSelectQuery & select, ASTPtr & ast, Data & data);
};
using CrossToInnerJoinVisitor = InDepthNodeVisitor<CrossToInnerJoinMatcher, true>;
}

View File

@ -528,7 +528,7 @@ bool DDLWorker::tryExecuteQuery(const String & query, const DDLTask & task, Exec
{
current_context = std::make_unique<Context>(context);
current_context->setCurrentQueryId(""); // generate random query_id
executeQuery(istr, ostr, false, *current_context, nullptr);
executeQuery(istr, ostr, false, *current_context, {}, {});
}
catch (...)
{

View File

@ -27,7 +27,7 @@ struct DatabaseAndTableWithAlias
DatabaseAndTableWithAlias() = default;
DatabaseAndTableWithAlias(const ASTPtr & identifier_node, const String & current_database = "");
DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = "");
DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database);
DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database = "");
/// "alias." or "table." if alias is empty
String getQualifiedNamePrefix() const;

View File

@ -22,7 +22,6 @@
#include <Columns/IColumn.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
@ -39,7 +38,6 @@
#include <Storages/StorageMemory.h>
#include <Storages/StorageJoin.h>
#include <DataStreams/LazyBlockInputStream.h>
#include <DataStreams/copyData.h>
#include <Dictionaries/IDictionary.h>
@ -568,9 +566,6 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
if (!subquery_for_set.join)
{
JoinPtr join = std::make_shared<Join>(analyzedJoin().key_names_right, settings.join_use_nulls,
settings.size_limits_for_join, join_params.kind, join_params.strictness);
/** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs
* - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1,
* in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`.
@ -587,39 +582,23 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
else if (table_to_join.database_and_table_name)
table = table_to_join.database_and_table_name;
const JoinedColumnsList & columns_from_joined_table = analyzedJoin().columns_from_joined_table;
Names original_columns;
for (const auto & column : analyzedJoin().columns_from_joined_table)
for (const auto & column : columns_from_joined_table)
if (required_columns_from_joined_table.count(column.name_and_type.name))
original_columns.emplace_back(column.original_name);
auto interpreter = interpretSubquery(table, context, subquery_depth, original_columns);
subquery_for_set.source = std::make_shared<LazyBlockInputStream>(
interpreter->getSampleBlock(),
[interpreter]() mutable { return interpreter->execute().in; });
}
/// Alias duplicating columns as qualified.
for (const auto & column : analyzedJoin().columns_from_joined_table)
if (required_columns_from_joined_table.count(column.name_and_type.name))
subquery_for_set.joined_block_aliases.emplace_back(column.original_name, column.name_and_type.name);
auto sample_block = subquery_for_set.source->getHeader();
for (const auto & name_with_alias : subquery_for_set.joined_block_aliases)
{
if (sample_block.has(name_with_alias.first))
{
auto pos = sample_block.getPositionByName(name_with_alias.first);
auto column = sample_block.getByPosition(pos);
sample_block.erase(pos);
column.name = name_with_alias.second;
sample_block.insert(std::move(column));
}
subquery_for_set.makeSource(interpreter, columns_from_joined_table, required_columns_from_joined_table);
}
Block sample_block = subquery_for_set.renamedSampleBlock();
joined_block_actions->execute(sample_block);
/// TODO You do not need to set this up when JOIN is only needed on remote servers.
subquery_for_set.join = join;
subquery_for_set.join = std::make_shared<Join>(analyzedJoin().key_names_right, settings.join_use_nulls,
settings.size_limits_for_join, join_params.kind, join_params.strictness);
subquery_for_set.join->setSampleBlock(sample_block);
subquery_for_set.joined_block_actions = joined_block_actions;
}

View File

@ -380,8 +380,9 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression
if (query_analyzer->appendJoin(chain, dry_run || !res.first_stage))
{
res.has_join = true;
res.before_join = chain.getLastActions();
if (!res.hasJoin())
throw Exception("No expected JOIN", ErrorCodes::LOGICAL_ERROR);
chain.addStep();
}
@ -548,7 +549,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
if (expressions.first_stage)
{
if (expressions.has_join)
if (expressions.hasJoin())
{
const ASTTableJoin & join = static_cast<const ASTTableJoin &>(*query.join()->table_join);
if (join.kind == ASTTableJoin::Kind::Full || join.kind == ASTTableJoin::Kind::Right)

View File

@ -132,7 +132,7 @@ private:
struct AnalysisResult
{
bool has_join = false;
bool hasJoin() const { return before_join.get(); }
bool has_where = false;
bool need_aggregate = false;
bool has_having = false;

View File

@ -185,8 +185,8 @@ BlockIO InterpreterSystemQuery::execute()
case Type::STOP_REPLICATED_SENDS:
startStopAction(context, query, ActionLocks::PartsSend, false);
break;
case Type::START_REPLICATEDS_SENDS:
startStopAction(context, query, ActionLocks::PartsSend, false);
case Type::START_REPLICATED_SENDS:
startStopAction(context, query, ActionLocks::PartsSend, true);
break;
case Type::STOP_REPLICATION_QUEUES:
startStopAction(context, query, ActionLocks::ReplicationQueue, false);

View File

@ -59,6 +59,9 @@ struct RewriteTablesVisitorData
static bool needRewrite(ASTSelectQuery & select)
{
if (!select.tables)
return false;
auto tables = typeid_cast<const ASTTablesInSelectQuery *>(select.tables.get());
if (!tables)
return false;

View File

@ -298,6 +298,7 @@ struct Settings
M(SettingBool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.") \
M(SettingBool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.") \
M(SettingBool, allow_experimental_multiple_joins_emulation, false, "Emulate multiple joins using subselects") \
M(SettingBool, allow_experimental_cross_to_join_conversion, false, "Convert CROSS JOIN to INNER JOIN if possible") \
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
TYPE NAME {DEFAULT};

View File

@ -0,0 +1,49 @@
#include <Interpreters/SubqueryForSet.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <DataStreams/LazyBlockInputStream.h>
namespace DB
{
void SubqueryForSet::makeSource(std::shared_ptr<InterpreterSelectWithUnionQuery> & interpreter,
const std::list<JoinedColumn> & columns_from_joined_table,
const NameSet & required_columns_from_joined_table)
{
source = std::make_shared<LazyBlockInputStream>(interpreter->getSampleBlock(),
[interpreter]() mutable { return interpreter->execute().in; });
for (const auto & column : columns_from_joined_table)
if (required_columns_from_joined_table.count(column.name_and_type.name))
joined_block_aliases.emplace_back(column.original_name, column.name_and_type.name);
sample_block = source->getHeader();
for (const auto & name_with_alias : joined_block_aliases)
{
if (sample_block.has(name_with_alias.first))
{
auto pos = sample_block.getPositionByName(name_with_alias.first);
auto column = sample_block.getByPosition(pos);
sample_block.erase(pos);
column.name = name_with_alias.second;
sample_block.insert(std::move(column));
}
}
}
void SubqueryForSet::renameColumns(Block & block)
{
for (const auto & name_with_alias : joined_block_aliases)
{
if (block.has(name_with_alias.first))
{
auto pos = block.getPositionByName(name_with_alias.first);
auto column = block.getByPosition(pos);
block.erase(pos);
column.name = name_with_alias.second;
block.insert(std::move(column));
}
}
}
}

View File

@ -0,0 +1,49 @@
#pragma once
#include <Parsers/IAST.h>
#include <Interpreters/PreparedSets.h>
#include <Interpreters/ExpressionActions.h>
namespace DB
{
class Join;
using JoinPtr = std::shared_ptr<Join>;
class InterpreterSelectWithUnionQuery;
struct JoinedColumn;
/// Information on what to do when executing a subquery in the [GLOBAL] IN/JOIN section.
struct SubqueryForSet
{
/// The source is obtained using the InterpreterSelectQuery subquery.
BlockInputStreamPtr source;
/// If set, build it from result.
SetPtr set;
JoinPtr join;
/// Apply this actions to joined block.
ExpressionActionsPtr joined_block_actions;
/// If set, put the result into the table.
/// This is a temporary table for transferring to remote servers for distributed query processing.
StoragePtr table;
void makeSource(std::shared_ptr<InterpreterSelectWithUnionQuery> & interpreter,
const std::list<JoinedColumn> & columns_from_joined_table,
const NameSet & required_columns_from_joined_table);
Block renamedSampleBlock() const { return sample_block; }
void renameColumns(Block & block);
private:
NamesWithAliases joined_block_aliases; /// Rename column from joined block from this list.
Block sample_block; /// source->getHeader() + column renames
};
/// ID of subquery -> what to do with it.
using SubqueriesForSets = std::unordered_map<String, SubqueryForSet>;
}

View File

@ -22,6 +22,7 @@
#include <Parsers/queryToString.h>
#include <Interpreters/JoinToSubqueryTransformVisitor.h>
#include <Interpreters/CrossToInnerJoinVisitor.h>
#include <Interpreters/Quota.h>
#include <Interpreters/InterpreterFactory.h>
#include <Interpreters/ProcessList.h>
@ -199,6 +200,14 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
logQuery(queryToString(*ast), context);
}
if (settings.allow_experimental_cross_to_join_conversion)
{
CrossToInnerJoinVisitor::Data cross_to_inner;
CrossToInnerJoinVisitor(cross_to_inner).visit(ast);
if (cross_to_inner.done)
logQuery(queryToString(*ast), context);
}
/// Check the limits.
checkASTSizeLimits(*ast, settings);
@ -435,7 +444,8 @@ void executeQuery(
WriteBuffer & ostr,
bool allow_into_outfile,
Context & context,
std::function<void(const String &)> set_content_type)
std::function<void(const String &)> set_content_type,
std::function<void(const String &)> set_query_id)
{
PODArray<char> parse_buf;
const char * begin;
@ -518,6 +528,9 @@ void executeQuery(
if (set_content_type)
set_content_type(out->getContentType());
if (set_query_id)
set_query_id(context.getClientInfo().current_query_id);
copyData(*streams.in, *out);
}
}

View File

@ -14,7 +14,8 @@ void executeQuery(
WriteBuffer & ostr, /// Where to write query output to.
bool allow_into_outfile, /// If true and the query contains INTO OUTFILE section, redirect output to that file.
Context & context, /// DB, tables, data types, storage engines, functions, aggregate functions...
std::function<void(const String &)> set_content_type /// If non-empty callback is passed, it will be called with the Content-Type of the result.
std::function<void(const String &)> set_content_type, /// If non-empty callback is passed, it will be called with the Content-Type of the result.
std::function<void(const String &)> set_query_id /// If non-empty callback is passed, it will be called with the query id.
);

View File

@ -3,6 +3,7 @@
#include <Columns/ColumnString.h>
#include <Common/typeid_cast.h>
#include <pdqsort.h>
namespace DB
{
@ -94,7 +95,6 @@ struct PartialSortingLessWithCollation
}
};
void sortBlock(Block & block, const SortDescription & description, size_t limit)
{
if (!block)
@ -151,7 +151,7 @@ void sortBlock(Block & block, const SortDescription & description, size_t limit)
if (limit)
std::partial_sort(perm.begin(), perm.begin() + limit, perm.end(), less_with_collation);
else
std::sort(perm.begin(), perm.end(), less_with_collation);
pdqsort(perm.begin(), perm.end(), less_with_collation);
}
else
{
@ -160,7 +160,7 @@ void sortBlock(Block & block, const SortDescription & description, size_t limit)
if (limit)
std::partial_sort(perm.begin(), perm.begin() + limit, perm.end(), less);
else
std::sort(perm.begin(), perm.end(), less);
pdqsort(perm.begin(), perm.end(), less);
}
size_t columns = block.columns();

View File

@ -45,7 +45,7 @@ try
ReadBufferFromFileDescriptor in(STDIN_FILENO);
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
executeQuery(in, out, /* allow_into_outfile = */ false, context, {});
executeQuery(in, out, /* allow_into_outfile = */ false, context, {}, {});
return 0;
}

View File

@ -59,7 +59,7 @@ const char * ASTSystemQuery::typeToString(Type type)
return "START FETCHES";
case Type::STOP_REPLICATED_SENDS:
return "STOP REPLICATED SENDS";
case Type::START_REPLICATEDS_SENDS:
case Type::START_REPLICATED_SENDS:
return "START REPLICATED SENDS";
case Type::STOP_REPLICATION_QUEUES:
return "STOP REPLICATION QUEUES";
@ -97,7 +97,7 @@ void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &,
|| type == Type::STOP_FETCHES
|| type == Type::START_FETCHES
|| type == Type::STOP_REPLICATED_SENDS
|| type == Type::START_REPLICATEDS_SENDS
|| type == Type::START_REPLICATED_SENDS
|| type == Type::STOP_REPLICATION_QUEUES
|| type == Type::START_REPLICATION_QUEUES)
{

View File

@ -36,7 +36,7 @@ public:
STOP_FETCHES,
START_FETCHES,
STOP_REPLICATED_SENDS,
START_REPLICATEDS_SENDS,
START_REPLICATED_SENDS,
STOP_REPLICATION_QUEUES,
START_REPLICATION_QUEUES,
FLUSH_LOGS,

View File

@ -58,7 +58,7 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
case Type::STOP_FETCHES:
case Type::START_FETCHES:
case Type::STOP_REPLICATED_SENDS:
case Type::START_REPLICATEDS_SENDS:
case Type::START_REPLICATED_SENDS:
case Type::STOP_REPLICATION_QUEUES:
case Type::START_REPLICATION_QUEUES:
parseDatabaseAndTableName(pos, expected, res->target_database, res->target_table);

View File

@ -6,6 +6,7 @@ const char * auto_contributors[] {
"Alex Krash",
"Alex Zatelepin",
"Alexander Avdonkin",
"Alexander GQ Gerasiov",
"Alexander Krasheninnikov",
"Alexander Kuranoff",
"Alexander Lukin",
@ -50,6 +51,7 @@ const char * auto_contributors[] {
"Bogdan",
"Bogdan Voronin",
"Bolinov",
"Boris Granveaud",
"Brett Hoerner",
"Bulat Gaifullin",
"Chen Yufei",
@ -58,6 +60,7 @@ const char * auto_contributors[] {
"CurtizJ",
"Daniel Bershatsky",
"Daniel Dao",
"Danila Kutenin",
"Denis Burlaka",
"Denis Zhuravlev",
"Derek Perkins",
@ -69,15 +72,19 @@ const char * auto_contributors[] {
"Dmitry S..ky / skype: dvska-at-skype",
"Elghazal Ahmed",
"Emmanuel Donin de Rosière",
"Eric",
"Eugene Klimov",
"Eugene Konkov",
"Evgenii Pravda",
"Evgeniy Gatov",
"Evgeniy Udodov",
"Evgeny Konkov",
"Flowyi",
"Fruit of Eden",
"George",
"George G",
"George3d6",
"Gleb Kanterov",
"Guillaume Tassery",
"Hamoon",
"Hiroaki Nakamura",
@ -89,6 +96,7 @@ const char * auto_contributors[] {
"Ilya Khomutov",
"Ilya Korolev",
"Ilya Shipitsin",
"Ilya Skrypitsa",
"Ivan",
"Ivan Babrou",
"Ivan Blinkov",
@ -98,6 +106,7 @@ const char * auto_contributors[] {
"Jason",
"Jean Baptiste Favre",
"Jonatas Freitas",
"Karl Pietrzak",
"Keiji Yoshida",
"Kirill Malev",
"Kirill Shvakov",
@ -112,14 +121,18 @@ const char * auto_contributors[] {
"LiuCong",
"LiuYangkuan",
"Luis Bosque",
"Léo Ercolanelli",
"Maks Skorokhod",
"Maksim",
"Marek Vavrusa",
"Marek Vavruša",
"Marek Vavruša",
"Marsel Arduanov",
"Marti Raudsepp",
"Max",
"Max Akhmedov",
"Max Vetrov",
"Maxim Fedotov",
"Maxim Fridental",
"Maxim Khrisanfov",
"Maxim Nikulin",
@ -127,6 +140,7 @@ const char * auto_contributors[] {
"Michael Furmur",
"Michael Kolupaev",
"Michael Razuvaev",
"Michal Lisowski",
"Mikhail Filimonov",
"Mikhail Salosin",
"Mikhail Surin",
@ -134,11 +148,13 @@ const char * auto_contributors[] {
"Milad Arabi",
"Narek Galstyan",
"Nicolae Vartolomei",
"Nikhil Raman",
"Nikita Vasilev",
"Nikolai Kochetov",
"Nikolay Kirsh",
"Nikolay Vasiliev",
"Nikolay Volosatov",
"Odin Hultgren Van Der Horst",
"Okada Haruki",
"Oleg Komarov",
"Oleg Obleukhov",
@ -152,6 +168,7 @@ const char * auto_contributors[] {
"Pavel Yakunin",
"Pavlo Bashynskiy",
"Pawel Rog",
"Persiyanov Dmitriy Andreevich",
"Ravengg",
"Reto Kromer",
"Roman Lipovsky",
@ -160,6 +177,7 @@ const char * auto_contributors[] {
"Roman Tsisyk",
"Sabyanin Maxim",
"SaltTan",
"Samuel Chou",
"Sergei Tsetlin (rekub)",
"Sergey Elantsev",
"Sergey Fedorov",
@ -209,6 +227,7 @@ const char * auto_contributors[] {
"Yury Stankevich",
"abdrakhmanov",
"abyss7",
"achulkov2",
"alesapin",
"alexey-milovidov",
"ap11",
@ -229,6 +248,7 @@ const char * auto_contributors[] {
"ezhaka",
"f1yegor",
"felixoid",
"fessmage",
"filimonov",
"flow",
"ggerogery",
@ -245,27 +265,33 @@ const char * auto_contributors[] {
"leozhang",
"liuyimin",
"lomberts",
"maiha",
"mf5137",
"mfridental",
"morty",
"moscas",
"nicelulu",
"ns-vasilev",
"ogorbacheva",
"orantius",
"peshkurov",
"proller",
"pyos",
"qianlixiang",
"robot-clickhouse",
"robot-metrika-test",
"root",
"santaux",
"sdk2",
"serebrserg",
"shangshujie",
"shedx",
"stavrolia",
"sundy-li",
"sundyli",
"topvisor",
"velom",
"vicdashkov",
"zamulla",
"zhang2014",
"Георгий Кондратьев",
@ -274,6 +300,7 @@ const char * auto_contributors[] {
"Павел Литвиненко",
"Смитюх Вячеслав",
"Сундуков Алексей",
"小路",
"张健",
"谢磊",
nullptr};

View File

@ -128,7 +128,7 @@ else
TEST_DICT=${TEST_DICT=1}
CLICKHOUSE_CLIENT_QUERY="${CLICKHOUSE_CLIENT} --config ${CLICKHOUSE_CONFIG_CLIENT} --port $CLICKHOUSE_PORT_TCP -m -n -q"
$CLICKHOUSE_CLIENT_QUERY 'SELECT * from system.build_options; SELECT * FROM system.clusters;'
CLICKHOUSE_TEST="env PATH=$PATH:$BIN_DIR ${TEST_DIR}clickhouse-test --binary ${BIN_DIR}${CLICKHOUSE_BINARY} --configclient $CLICKHOUSE_CONFIG_CLIENT --configserver $CLICKHOUSE_CONFIG --tmp $DATA_DIR/tmp --queries $QUERIES_DIR $TEST_OPT0 $TEST_OPT"
CLICKHOUSE_TEST="env ${TEST_DIR}clickhouse-test --binary ${BIN_DIR}${CLICKHOUSE_BINARY} --configclient $CLICKHOUSE_CONFIG_CLIENT --configserver $CLICKHOUSE_CONFIG --tmp $DATA_DIR/tmp --queries $QUERIES_DIR $TEST_OPT0 $TEST_OPT"
CLICKHOUSE_PERFORMANCE_TEST="${BIN_DIR}clickhouse-performance-test --port $CLICKHOUSE_PORT_TCP --recursive $CUR_DIR/performance --skip-tags=long"
if [ "${TEST_RUN_STRESS}" ]; then
# Running test in parallel will fail some results (tests can create/fill/drop same tables)

View File

@ -27,7 +27,6 @@ sudo apt install git
Подробное руководство по использованию Git: https://git-scm.com/book/ru/v2
# Клонирование репозитория на рабочую машину
Затем вам потребуется загрузить исходники для работы на свой компьютер. Это называется "клонирование репозитория", потому что создаёт на вашем компьютере локальную копию репозитория, с которой вы будете работать.
@ -50,6 +49,28 @@ git submodule update
```
Проверить наличие submodules можно с помощью команды `git submodule status`.
Если вы получили сообщение об ошибке:
```
Permission denied (publickey).
fatal: Could not read from remote repository.
Please make sure you have the correct access rights
and the repository exists.
```
Как правило это означает, что отсутствуют ssh ключи для соединения с GitHub. Ключи расположены в директории `~/.ssh`. В интерфейсе GitHub, в настройках, необходимо загрузить публичные ключи, чтобы он их понимал.
Вы также можете клонировать репозиторий по протоколу https:
```
git clone https://github.com/yandex/ClickHouse.git
```
Этот вариант не подходит для отправки изменений на сервер. Вы можете временно его использовать, а затем добавить ssh ключи и заменить адрес репозитория с помощью команды `git remote`.
Вы можете также добавить для своего локального репозитория адрес оригинального репозитория Яндекса, чтобы притягивать оттуда обновления:
```
git remote add upstream git@github.com:yandex/ClickHouse.git
```
После этого, вы сможете добавлять в свой репозиторий обновления из репозитория Яндекса с помощью команды `git pull upstream master`.
# Система сборки
@ -70,6 +91,8 @@ Ninja - система запуска сборочных задач.
brew install cmake ninja
```
Проверьте версию CMake: `cmake --version`. Если версия меньше 3.3, то установите новую версию с сайта https://cmake.org/download/
# Необязательные внешние библиотеки
@ -122,7 +145,6 @@ cmake -D CMAKE_BUILD_TYPE=Debug ..
```
Вы можете изменить вариант сборки, выполнив эту команду в директории build.
Запустите ninja для сборки:
```
ninja clickhouse-server clickhouse-client
@ -166,6 +188,11 @@ ls -l dbms/programs/clickhouse
Для подключения к ClickHouse с помощью clickhouse-client, в соседнем терминале, зайдите в директорию `ClickHouse/build/dbms/programs/` и выполните `clickhouse client`.
Если вы получили сообщение `Connection refused` на Mac OS X или FreeBSD, то укажите для клиента 127.0.0.1 в качестве имени хоста:
```
clickhouse client --host 127.0.0.1
```
Вы можете заменить собранным вами ClickHouse продакшен версию, установленную в системе. Для этого, установите ClickHouse на свою машину по инструкции с официального сайта. Затем выполните:
```
sudo service clickhouse-server stop
@ -173,6 +200,8 @@ sudo cp ClickHouse/build/dbms/programs/clickhouse /usr/bin/
sudo service clickhouse-server start
```
Обратите внимание, что `clickhouse-client`, `clickhouse-server` и другие, являеются симлинками на общий бинарник `clickhouse`.
Также вы можете запустить собранный вами ClickHouse с конфигурационным файлом системного ClickHouse:
```
sudo service clickhouse-server stop
@ -196,3 +225,44 @@ sudo -u clickhouse ClickHouse/build/dbms/programs/clickhouse server --config-fil
Описание архитектуры ClickHouse: https://clickhouse.yandex/docs/ru/development/architecture/
Стиль кода: https://clickhouse.yandex/docs/ru/development/style/
Разработка тестов: https://clickhouse.yandex/docs/ru/development/tests/
Список задач: https://github.com/yandex/ClickHouse/blob/master/dbms/tests/instructions/easy_tasks_sorted_ru.md
# Тестовые данные
Разработка ClickHouse часто требует загрузки реалистичных наборов данных. Особенно это важно для тестирования производительности. Специально для вас мы подготовили набор данных, представляющий собой анонимизированные данные Яндекс.Метрики. Загрузка этих данных потребует ещё 3 GB места на диске. Для выполнения большинства задач разработки, загружать эти данные не обязательно.
```
sudo apt install wget xz-utils
wget https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_v1.tsv.xz
wget https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz
xz -v -d hits_v1.tsv.xz
xz -v -d visits_v1.tsv.xz
clickhouse-client
CREATE TABLE test.hits ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime);
CREATE TABLE test.visits ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), `Goals.ID` Array(UInt32), `Goals.Serial` Array(UInt32), `Goals.EventTime` Array(DateTime), `Goals.Price` Array(Int64), `Goals.OrderID` Array(String), `Goals.CurrencyID` Array(UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, `TraficSource.ID` Array(Int8), `TraficSource.SearchEngineID` Array(UInt16), `TraficSource.AdvEngineID` Array(UInt8), `TraficSource.PlaceID` Array(UInt16), `TraficSource.SocialSourceNetworkID` Array(UInt8), `TraficSource.Domain` Array(String), `TraficSource.SearchPhrase` Array(String), `TraficSource.SocialSourcePage` Array(String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, `ParsedParams.Key1` Array(String), `ParsedParams.Key2` Array(String), `ParsedParams.Key3` Array(String), `ParsedParams.Key4` Array(String), `ParsedParams.Key5` Array(String), `ParsedParams.ValueDouble` Array(Float64), `Market.Type` Array(UInt8), `Market.GoalID` Array(UInt32), `Market.OrderID` Array(String), `Market.OrderPrice` Array(Int64), `Market.PP` Array(UInt32), `Market.DirectPlaceID` Array(UInt32), `Market.DirectOrderID` Array(UInt32), `Market.DirectBannerID` Array(UInt32), `Market.GoodID` Array(String), `Market.GoodName` Array(String), `Market.GoodQuantity` Array(Int32), `Market.GoodPrice` Array(Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) SAMPLE BY intHash32(UserID) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID);
clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.hits FORMAT TSV" < hits_v1.tsv
clickhouse-client --max_insert_block_size 100000 --query "INSERT INTO test.visits FORMAT TSV" < visits_v1.tsv
```
# Создание pull request
Откройте свой форк репозитория в интерфейсе GitHub. Если вы вели разработку в бранче, выберите этот бранч. На странице будет доступна кнопка "Pull request". По сути, это означает "создать заявку на принятие моих изменений в основной репозиторий".
Pull request можно создать, даже если работа над задачей ещё не завершена. В этом случае, добавьте в его название слово "WIP" (work in progress). Название можно будет изменить позже. Это полезно для совместного просмотра и обсуждения изменений, а также для запуска всех имеющихся тестов. Введите краткое описание изменений - впоследствии, оно будет использовано для релизных changelog.
Тесты будут запущены, как только сотрудники Яндекса поставят для pull request тег "Can be tested". Результаты первых проверок (стиль кода) появятся уже через несколько минут. Результаты сборки появятся примерно через пол часа. Результаты основного набора тестов будут доступны в пределах часа.
Система подготовит сборки ClickHouse специально для вашего pull request. Для их получения, нажмите на ссылку "Details" у проверки "Clickhouse build check". Там вы сможете найти прямые ссылки на собранные .deb пакеты ClickHouse, которые, при желании, вы даже сможете установить на свои продакшен серверы (если не страшно).
Вероятнее всего, часть сборок не будет успешной с первого раза. Ведь мы проверяем сборку кода и gcc и clang, а при сборке с помощью clang включаются почти все существующие в природе warnings (всегда с флагом `-Werror`). На той же странице, вы сможете найти логи сборки - вам не обязательно самому собирать ClickHouse всеми возможными способами.

View File

@ -0,0 +1,342 @@
# Простые задачи
## Пустой параметр --password в клиенте должен быть эквивалентен --ask-password.
То есть означать предложение ввести пароль в интерактивном режиме.
`dbms/programs/client/ConnectionParameters.h`
\* кстати, сейчас функциональность реализована плохо: ввод пароля не поддерживает корректную обработку backspace.
## Недостатки юзабилити: у clickhouse-client отсутствует сокращённая опция -C, как вариант --config-file; Недостатки юзабилити, если пользователь не может прочитать конфиг клиента.
`dbms/programs/client/Client.cpp`
Также делаем `chmod 000 /etc/clickhouse-client/config.xml` и смотрим, что получится.
## Оператор NOT BETWEEN.
`SELECT * FROM system.numbers WHERE number NOT BETWEEN 5 AND 10 LIMIT 10`
`ExpressionListParsers.cpp`: `ParserBetweenExpression::parseImpl`
## HTTP заголовок query_id.
`programs/server/HTTPHandler.cpp` - смотрим метод `executeQuery`
`src/Interpreters/executeQuery.h`
`src/Interpreters/executeQuery.cpp` - смотрим колбэк на выставление Content-Type
## Уменьшать max_memory_usage и размеры кэшей при старте, если на сервере мало оперативки.
Смотрим, сколько на сервере оперативки. Если `max_memory_usage`, `max_memory_usage_for_all_queries` ограничены, но больше 90% (настройка) от имеющейся оперативки, то уменьшать их и выводить предупреждение в лог. Аналогично для кэшей: `mark_cache`, `uncompressed_cache`.
`programs/server/Server.cpp` - инициализация сервера, установка размера кэшей
`getMemoryAmount.h` - информация о доступной оперативке
`context.setSetting` - для выставления `max_memory_usage` и других.
## Битовые операции для FixedString.
bitAnd, bitOr, bitNot, bitXor для значения типа FixedString, интерпретируемого как набор бит.
Сделайте сначала в C++ побитовые функции для работы с куском памяти:
```
void memoryBitAnd(const char * a, const char * b, char * result, size_t size);
```
Потом используйте их в вашей функции.
## Функция arrayWithConstant.
`arrayWithConstant(3, 'hello') = ['hello', 'hello', 'hello']`
Смотрите метод `IColumn::replicate` для размножения значений столбца.
## Функция flatten для превращения массивов массивов в массив элементов.
`flatten([[1, 2, 3], [4, 5]]) = [1, 2, 3, 4, 5]`
`ColumnArray` - внимательно изучаем, как устроены массивы в ClickHouse.
## Добавить generic вариант функций least, greatest.
`SELECT least(123, 456)` - работает.
`SELECT least('123', '456')` - не работает. Надо сделать.
Делаем с помощью `IColumn::compareAt` для одинаковых типов и с помощью `castColumn`, `getLeastSuperType` для разных.
## При ATTACH кусков, проверять владельца файлов.
Смотрим, что все файлы в прикрепляемых кусках от правильного пользователя.
## COLLATE должно работать для Nullable(String).
В ClickHouse есть возможность указать collation для сортировки строк. Это не работает для `Nullable(String)`.
## Проверить возможность использования pdqsort вместо std::sort для полной comparison-based сортировки.
В случае, когда есть ORDER BY без LIMIT, это может позволить слегка увеличить производительность.
## Запретить чтение значений типа AggregateFunction по-умолчанию и добавить настройку.
Состояния агрегатных функций могут быть записаны в дамп и считаны из него. Но десериализация состояний агрегатных функций небезопасна. Аккуратно выбранные пользовательские данные могут привести к segfault или порче памяти. Поэтому нужно просто сделать настройку, которая запрещает читать AggregateFunction из пользовательских данных.
## Опции progress и time для clickhouse-local (по аналогии с clickhouse-client).
Возможность выводить время выполнения запроса, а также красивый прогресс-бар для каждого запроса.
## Usability: clickhouse-server должен поддерживать --help.
## В статистику jemalloc добавить информацию по arenas.
В `system.asynchronous_metrics` - суммарный размер арен.
## Добавить агрегатную функцию topKWeighted.
`SELECT topKWeighted(value, weight)` - учитывать каждое значение с весом.
## Функция isValidUTF8, toValidUTF8.
`isValidUTF8` возвращает 1, если строка содержит набор байт в кодировке UTF-8.
`toValidUTF8` - заменяет последовательности байт, не соответствующие кодировке UTF-8, на replacement character.
# Более сложные задачи
## CREATE TABLE AS table_function()
Возможность создать таблицу с таким же типом и структурой, как табличная функция.
`ParserCreateQuery.cpp`, `InterpreterCreateQuery`, `Context::executeTableFunction`
## Layout внешних словарей "direct".
Как cache, но без кэша — всегда прямой запрос в источник.
## Подсказки в фабриках на основе edit distance.
Всевозможные объекты: функции, агрегатные функции, типы данных, движки таблиц, и т. п. достаются по имени из фабрик. Часто пользователь допускает опечатку. Например, вместо `SELECT count(*)` может быть написано `SELECT cunt(*)`. В случае опечатки, необходимо в текст сообщения добавлять указание на ближайшие варианты. Для реализации можно использовать расстояние Левенштейна и полный перебор, или (лучше) - триграмный индекс. Подсказки выдаём, если указанное имя отличается от существующего на 1..2 буквы. Сортируем возможные варианты в порядке похожести. Для того, чтобы это работало во всех фабриках, может быть, потребуется обобщить их.
## Учитывать порядок столбцов в заголовке в форматах CSV и TSV.
В заголовке CSV, TSV могут быть указаны имена столбцов. Сейчас они полностью игнорируются. Надо учитывать, под настройкой.
## Функции randomFixedString, randomBinaryString, fuzzBits, fuzzBytes.
## Функции для geoHash.
Geohash - способ преобразования географических координат в строку, так что отображение обладает свойством локальности. https://en.wikipedia.org/wiki/Geohash В качестве библиотеки следует использовать эту: https://github.com/yinqiwen/geohash-int Необходимо добавить функции для перевода в обе стороны, а также для числового и текстового вариантов.
## Агрегатные функции для статистических тестов (e.g. тест нормальности распределения) и статистик (e.g. энтропия).
Энтропию следует считать по гистограмме. Пример расчёта гистограммы смотрите в реализации функции `quantileExact`.
https://github.com/yandex/ClickHouse/issues/3266
## Функции создания и обновления состояния агрегатной функции по одному кортежу аргументов.
В ClickHouse есть понятие - состояние вычисления агрегатной функции. Состояния агрегатных функций можно записывать в таблицы, складывать, финализировать и т. п. https://clickhouse.yandex/docs/ru/data_types/nested_data_structures/aggregatefunction/
Получить состояние агрегатной функции можно с помощью комбинатора State: https://clickhouse.yandex/docs/ru/query_language/agg_functions/combinators/#-state Но хотелось бы добавить ещё более простой способ получения состояния агрегатной функции.
Например:
`createAggregationState('groupArray')` - создать пустое (начальное) состояние агрегатной функции.
`createAggregationState('groupArray', 1)` - создать состояние агрегатной функции, в котором агрегировано одно значение 1.
`createAggregationState('argMax', ('hello', 123))` - то же самое для агрегатных функций, принимающих несколько аргументов.
## Корректное сравнение Date и DateTime.
https://github.com/yandex/ClickHouse/issues/2011
Нужно сравнивать Date и DateTime так, как будто Date расширено до DateTime на начало суток в том же часовом поясе.
## LEFT ONLY JOIN
## Функции makeDate, makeDateTime.
`makeDate(year, month, day)`
`makeDateTime(year, month, day, hour, minute, second, [timezone])`
## Функции changeYear, changeMonth, ...
`changeYear(datetime, 2019)`
## Исправить мерцание прогресс-бара в clickhouse-client.
Это заметно при работе с серверами с большим пингом.
Прогресс бар не должен мерцать.
Наверное, надо просто вместо очистки строки, перемещать курсор в начало, не очищая её.
## Функция format для вставки значений в строку-шаблон.
`format('Hello {2} World {1}', x, y)`
## Добавить поддержку hyperscan.
https://github.com/intel/hyperscan
Реализовать на основе этой библиотеки функцию для матчинга сразу большого количества регулярных выражений.
## Функция rowNumberForKey.
Возвращает инкрементальное число для повторно встречающихся значений key.
## Агрегатная функция groupConcat.
`groupConcat(x, ',')` - собрать из переданных значений x строку, разделённую запятыми.
## Функции DATE_ADD, DATE_SUB как синонимы для совместимости с SQL.
https://dev.mysql.com/doc/refman/8.0/en/date-and-time-functions.html#function_date-add
## Функции positionReverse, positionUTF8Reverse, positionCaseInsensitiveReverse, positionCaseInsensitiveUTF8Reverse.
position с конца строки.
## Функция indexOf должна поддерживать Enum-ы без cast-а.
`indexOf(arr, 'hello')`, `indexOf(arr, 1)` должны работать, если arr имеет тип `Array(Enum8('hello' = 1, 'world' = 2))`
## Комбинатор агрегатных функций Distinct.
Пример: `avgDistinct(x)` - вычислить среднее по всем различным переданным значениям.
## Проверка набора инструкций при старте сервера.
Если сервер собран с поддержкой SSE 4.2, 4.1, 4, SSSE 3, SSE 3, то как можно ближе к началу работы, запускаем функцию, которая выполняет нужную инструкцию в качестве теста (asm volatile вставка), а до этого ставим обработчик сигнала SIGILL, который в случае невозможности выполнить инструкцию, сделает siglongjmp, позволит нам вывести понятное сообщение в лог и завершить работу. Замечание: /proc/cpuinfo зачастую не содержит актуальную информацию.
## Добавить сжатие Brotli для HTTP интерфейса.
`Content-Encoding: br`
## Метрики количества ошибок.
Добавляем счётчики всех ошибок (ErrorCodes) по аналогии с ProfileEvents. Кроме количества запоминаем также время последней ошибки, стек трейс, сообщение. Добавляем системную таблицу system.errors. Отправка в Graphite.
## Добавить Lizard, LZSSE и density в качестве вариантов алгоритмов сжатия.
Экспериментальные алгоритмы сжатия. Сейчас ClickHouse поддерживает только lz4 и zstd.
## Запрос CREATE OR REPLACE TABLE/VIEW.
Атомарно (под блокировкой) удаляет таблицу перед созданием новой, если такая была.
## Приведение типов для IN (subquery).
`SELECT 1 IN (SELECT -1 UNION ALL SELECT 1)`
- сейчас не работает.
## Возможность задать смещение для LIMIT BY.
https://clickhouse.yandex/docs/ru/query_language/select/#limit-n-by
`LIMIT 100, 10 BY RegionID` - выдать не более 10 строк для каждого RegionID, но пропустив первые 100 строк.
## Возможность вставки значений типа AggregateFunction в виде кортежа значений аргументов, а не бинарного дампа состояния, под настройкой.
Во входных данных в запросе INSERT должна быть возможность передать значение типа AggregateFunction не в виде сериализованного состояния, а в виде аргументов, которые будут агрегированы, для формирования этого состояния.
## Возможность использовать ALIAS столбцы при INSERT.
https://clickhouse.yandex/docs/en/query_language/create/#create-table
`INSERT INTO table (column1, column2, ...)`
- если column - это ALIAS столбец, и если выражение для ALIAS тривиально (просто ссылается на другой столбец), то разрешить использовать его вместо другого столбца в запросе INSERT.
## Запрос ALTER TABLE LOCK/UNLOCK PARTITION.
Запретить модификацию данных в партиции. На партицию ставится флаг, что она заблокирована. В неё нельзя делать INSERT и ALTER. С файлов снимается доступ на запись.
## Поддержка произвольных константных выражений в LIMIT.
Возможность писать `LIMIT 1 + 2`. То же самое для `LIMIT BY`.
## Добавить информацию об exp-smoothed количестве ошибок соединений с репликами в таблицу system.clusters.
У нас есть счётчик ошибок соединения с серверами для failover. Надо сделать его видимым для пользователя.
## Настройка join_use_nulls: поддержка для LEFT ARRAY JOIN.
## Внешние словари из Redis/Aerospike/Couchbase/Cassandra (на выбор).
Подключить одну из key-value БД как источник.
## Движок таблиц Mongo, табличная функция mongo.
Возможность легко импортировать данные из MongoDB.
## Возможность использования нескольких потоков для INSERT при INSERT SELECT.
При INSERT SELECT, запрос SELECT может выполняться параллельно, но все данные будут передаваться на вставку в INSERT в один поток. Хотя некоторые таблицы (семейства MergeTree) поддерживают параллельную вставку. Необходимо сделать настройку для максимального количества потоков для INSERT.
## Корректная обработка multiline значений в Pretty форматах.
SELECT 'hello\nworld' AS x, 123 AS y
```
┌─x──────────┬───y─┐
│ hello
world │ 123 │
└────────────┴─────┘
```
А надо так:
```
┌─x─────┬───y─┐
│ hello…│ 123 │
│…world │ │
└───────┴─────┘
```
## Писать логи ClickHouse в ClickHouse.
Пишем текстовые логи ClickHouse в системную таблицу в структурированном виде.
См. SystemLog.h, cpp.
## Работоспособность внешних данных на время сессии.
https://clickhouse.yandex/docs/en/operations/table_engines/external_data/
Не работает, если открыть clickhouse-client в интерактивном режиме и делать несколько запросов.
## Настройка для возможности получить частичный результат при cancel-е.
Хотим по Ctrl+C получить те данные, которые успели обработаться.
## Раскрытие кортежей в функциях высшего порядка.
## Табличная функция loop.
`SELECT * FROM loop(database, table)`
Читает данные из таблицы в бесконечном цикле.
## Настройка, позволяющая обратиться ко всем репликам кластера, как к разным шардам.
## Возможность ATTACH партиции с меньшим или большим количеством столбцов.
## Поддержка неконстантного аргумента с тайм-зоной у некоторых функций для работы с датой и временем.
## Возможность задавать параметры соединений для табличных функций, движков таблиц и для реплик из отдельных разделов конфигурации.
## Настройка rollup_use_nulls.
## Настройка cast_keep_nullable.
## Функция bitEquals для сравнения произвольных типов данных побитово.
## Функция serialize для implementation specific non portable non backwards compatible сериализации любого типа данных в набор байт.
## Функция arrayEnumerateUniqDeep
Как arrayEnumerateUniq, но смотрит на самые глубокие элементы вложенных массивов.
## Функция bitEquals и оператор <=>.
## Параллельный ALTER MODIFY COLUMN.

View File

@ -2,9 +2,10 @@
<name>trim_whitespaces</name>
<type>loop</type>
<preconditions>
<table_exists>whitespaces</table_exists>
</preconditions>
<create_query>CREATE TABLE IF NOT EXISTS whitespaces(value String) ENGINE = MergeTree() PARTITION BY tuple() ORDER BY tuple()</create_query>
<fill_query> INSERT INTO whitespaces SELECT value FROM (SELECT arrayStringConcat(groupArray(' ')) AS spaces, concat(spaces, toString(any(number)), spaces) AS value FROM numbers(100000000) GROUP BY pow(number, intHash32(number) % 4) % 12345678)</fill_query>
<fill_query> INSERT INTO whitespaces SELECT value FROM (SELECT arrayStringConcat(groupArray(' ')) AS spaces, concat(spaces, toString(any(number)), spaces) AS value FROM numbers(100000000) GROUP BY pow(number, intHash32(number) % 4) % 12345678)</fill_query>
<fill_query> INSERT INTO whitespaces SELECT value FROM (SELECT arrayStringConcat(groupArray(' ')) AS spaces, concat(spaces, toString(any(number)), spaces) AS value FROM numbers(100000000) GROUP BY pow(number, intHash32(number) % 4) % 12345678)</fill_query>
<stop_conditions>
<all_of>
@ -32,4 +33,6 @@
</substitutions>
<query>SELECT count() FROM whitespaces WHERE NOT ignore({func})</query>
<drop_query>DROP TABLE IF EXISTS whitespaces</drop_query>
</test>

View File

@ -1,17 +0,0 @@
CREATE TABLE whitespaces
(
value String
)
ENGINE = MergeTree()
PARTITION BY tuple()
ORDER BY tuple()
INSERT INTO whitespaces SELECT value
FROM
(
SELECT
arrayStringConcat(groupArray(' ')) AS spaces,
concat(spaces, toString(any(number)), spaces) AS value
FROM numbers(100000000)
GROUP BY pow(number, intHash32(number) % 4) % 12345678
) -- repeat something like this multiple times and/or just copy whitespaces table into itself

View File

@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
( ${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}?query=SELECT%201";
${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}?query=select+*+from+system.numbers+limit+1000000" ) | grep -v "Date:" | grep -v "X-ClickHouse-Server-Display-Name:"
${CLICKHOUSE_CURL} -s --head "${CLICKHOUSE_URL}?query=select+*+from+system.numbers+limit+1000000" ) | grep -v "Date:" | grep -v "X-ClickHouse-Server-Display-Name:" | grep -v "Query-Id:"
if [[ `${CLICKHOUSE_CURL} -sS -X POST -I "${CLICKHOUSE_URL}?query=SELECT+1" | grep -c '411 Length Required'` -ne 1 ]]; then
echo FAIL

View File

@ -1,118 +1,253 @@
inner
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
9 I 9 i
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
9 l9 \N 9 r9 nr9
inner subs
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
9 I 9 i
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
9 l9 \N 9 r9 nr9
inner expr
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
9 I 9 i
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
9 l9 \N 9 r9 nr9
left
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
5 G 0
8 H 0
9 I 9 i
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 9 r9 nr9
left subs
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
5 G 0
8 H 0
9 I 9 i
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 9 r9 nr9
left expr
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
5 G 0
8 H 0
9 I 9 i
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 9 r9 nr9
right
0 6 g
0 7 h
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
9 I 9 i
0 \N 6 r7 nr7
0 \N 7 r8 nr8
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
9 l9 \N 9 r9 nr9
right subs
0 6 g
0 7 h
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
9 I 9 i
0 \N 6 r7 nr7
0 \N 7 r8 nr8
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
9 l9 \N 9 r9 nr9
full
0 6 g
0 7 h
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
5 G 0
8 H 0
9 I 9 i
0 \N 6 r7 nr7
0 \N 7 r8 nr8
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 9 r9 nr9
full subs
0 6 g
0 7 h
1 A 1 a
1 A 1 b
2 B 2 c
2 C 2 c
3 D 3 d
3 D 3 e
4 E 4 f
4 F 4 f
5 G 0
8 H 0
9 I 9 i
0 \N 6 r7 nr7
0 \N 7 r8 nr8
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 9 r9 nr9
self inner
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l2 2 2 l3 3
2 l3 3 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
4 l5 \N 4 l5 \N
4 l5 \N 4 l6 \N
4 l6 \N 4 l5 \N
4 l6 \N 4 l6 \N
5 l7 \N 5 l7 \N
8 l8 \N 8 l8 \N
9 l9 \N 9 l9 \N
self inner nullable
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
self inner nullable vs not nullable
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l2 2
3 l4 4 2 l3 3
4 l5 \N 3 l4 4
4 l6 \N 3 l4 4
self inner nullable vs not nullable 2
4 r6 nr6 4 r6 nr6
6 r7 nr7 6 r7 nr7
7 r8 nr8 7 r8 nr8
9 r9 nr9 9 r9 nr9
self left
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l2 2 2 l3 3
2 l3 3 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
4 l5 \N 4 l5 \N
4 l5 \N 4 l6 \N
4 l6 \N 4 l5 \N
4 l6 \N 4 l6 \N
5 l7 \N 5 l7 \N
8 l8 \N 8 l8 \N
9 l9 \N 9 l9 \N
self left nullable
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
4 l5 \N 0 \N
4 l6 \N 0 \N
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 0 \N
self left nullable vs not nullable
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l2 2
3 l4 4 2 l3 3
4 l5 \N 3 l4 4
4 l6 \N 3 l4 4
5 l7 \N 0 0
8 l8 \N 0 0
9 l9 \N 0 0
self left nullable vs not nullable 2
1 r1 \N 0
1 r2 \N 0
2 r3 \N 0
3 r4 \N 0
3 r5 \N 0
4 r6 nr6 4 r6 nr6
6 r7 nr7 6 r7 nr7
7 r8 nr8 7 r8 nr8
9 r9 nr9 9 r9 nr9
self right
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l2 2 2 l3 3
2 l3 3 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
4 l5 \N 4 l5 \N
4 l5 \N 4 l6 \N
4 l6 \N 4 l5 \N
4 l6 \N 4 l6 \N
5 l7 \N 5 l7 \N
8 l8 \N 8 l8 \N
9 l9 \N 9 l9 \N
self right nullable
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
self right nullable vs not nullable
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l2 2
3 l4 4 2 l3 3
4 l5 \N 3 l4 4
4 l6 \N 3 l4 4
self full
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l2 2 2 l3 3
2 l3 3 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
4 l5 \N 4 l5 \N
4 l5 \N 4 l6 \N
4 l6 \N 4 l5 \N
4 l6 \N 4 l6 \N
5 l7 \N 5 l7 \N
8 l8 \N 8 l8 \N
9 l9 \N 9 l9 \N
self full nullable
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
4 l5 \N 0 \N
4 l6 \N 0 \N
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 0 \N
self full nullable vs not nullable
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l2 2
3 l4 4 2 l3 3
4 l5 \N 3 l4 4
4 l6 \N 3 l4 4
5 l7 \N 0 0
8 l8 \N 0 0
9 l9 \N 0 0

View File

@ -2,25 +2,27 @@ use test;
drop table if exists X;
drop table if exists Y;
create table X (id Int32, x_name String) engine Memory;
create table Y (id Int32, y_name String) engine Memory;
create table X (id Int32, x_a String, x_b Nullable(Int32)) engine Memory;
create table Y (id Int32, y_a String, y_b Nullable(String)) engine Memory;
insert into X (id, x_name) values (1, 'A'), (2, 'B'), (2, 'C'), (3, 'D'), (4, 'E'), (4, 'F'), (5, 'G'), (8, 'H'), (9, 'I');
insert into Y (id, y_name) values (1, 'a'), (1, 'b'), (2, 'c'), (3, 'd'), (3, 'e'), (4, 'f'), (6, 'g'), (7, 'h'), (9, 'i');
insert into X (id, x_a, x_b) values (1, 'l1', 1), (2, 'l2', 2), (2, 'l3', 3), (3, 'l4', 4);
insert into X (id, x_a) values (4, 'l5'), (4, 'l6'), (5, 'l7'), (8, 'l8'), (9, 'l9');
insert into Y (id, y_a) values (1, 'r1'), (1, 'r2'), (2, 'r3'), (3, 'r4'), (3, 'r5');
insert into Y (id, y_a, y_b) values (4, 'r6', 'nr6'), (6, 'r7', 'nr7'), (7, 'r8', 'nr8'), (9, 'r9', 'nr9');
select 'inner';
select X.*, Y.* from X inner join Y on X.id = Y.id;
select X.*, Y.* from X inner join Y on X.id = Y.id order by id;
select 'inner subs';
select s.*, j.* from (select * from X) as s inner join (select * from Y) as j on s.id = j.id;
select s.*, j.* from (select * from X) as s inner join (select * from Y) as j on s.id = j.id order by id;
select 'inner expr';
select X.*, Y.* from X inner join Y on (X.id + 1) = (Y.id + 1);
select X.*, Y.* from X inner join Y on (X.id + 1) = (Y.id + 1) order by id;
select 'left';
select X.*, Y.* from X left join Y on X.id = Y.id;
select X.*, Y.* from X left join Y on X.id = Y.id order by id;
select 'left subs';
select s.*, j.* from (select * from X) as s left join (select * from Y) as j on s.id = j.id;
select s.*, j.* from (select * from X) as s left join (select * from Y) as j on s.id = j.id order by id;
select 'left expr';
select X.*, Y.* from X left join Y on (X.id + 1) = (Y.id + 1);
select X.*, Y.* from X left join Y on (X.id + 1) = (Y.id + 1) order by id;
select 'right';
select X.*, Y.* from X right join Y on X.id = Y.id order by id;
@ -36,5 +38,43 @@ select s.*, j.* from (select * from X) as s full join (select * from Y) as j on
--select 'full expr';
--select X.*, Y.* from X full join Y on (X.id + 1) = (Y.id + 1) order by id;
select 'self inner';
select X.*, s.* from X inner join (select * from X) as s on X.id = s.id order by X.id;
select 'self inner nullable';
select X.*, s.* from X inner join (select * from X) as s on X.x_b = s.x_b order by X.id;
select 'self inner nullable vs not nullable';
select X.*, s.* from X inner join (select * from X) as s on X.id = s.x_b order by X.id;
-- TODO: s.y_b == '' instead of NULL
select 'self inner nullable vs not nullable 2';
select Y.*, s.* from Y inner join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by id;
select 'self left';
select X.*, s.* from X left join (select * from X) as s on X.id = s.id order by X.id;
select 'self left nullable';
select X.*, s.* from X left join (select * from X) as s on X.x_b = s.x_b order by X.id;
select 'self left nullable vs not nullable';
select X.*, s.* from X left join (select * from X) as s on X.id = s.x_b order by X.id;
-- TODO: s.y_b == '' instead of NULL
select 'self left nullable vs not nullable 2';
select Y.*, s.* from Y left join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by id;
select 'self right';
select X.*, s.* from X right join (select * from X) as s on X.id = s.id order by X.id;
select 'self right nullable';
select X.*, s.* from X right join (select * from X) as s on X.x_b = s.x_b order by X.id;
select 'self right nullable vs not nullable';
select X.*, s.* from X right join (select * from X) as s on X.id = s.x_b order by X.id;
--select 'self right nullable vs not nullable 2';
--select Y.*, s.* from Y right join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by id;
select 'self full';
select X.*, s.* from X full join (select * from X) as s on X.id = s.id order by X.id;
select 'self full nullable';
select X.*, s.* from X full join (select * from X) as s on X.x_b = s.x_b order by X.id;
select 'self full nullable vs not nullable';
select X.*, s.* from X full join (select * from X) as s on X.id = s.x_b order by X.id;
--select 'self full nullable vs not nullable 2';
--select Y.*, s.* from Y full join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by id;
drop table X;
drop table Y;

File diff suppressed because one or more lines are too long

View File

@ -0,0 +1,5 @@
a 2018-01-01 00:00:00 0000-00-00 00:00:00
b 2018-01-01 00:00:00 b 2018-01-01 00:00:00
c 2018-01-01 00:00:00 c 2018-01-01 00:00:00
b 2018-01-01 00:00:00 b 2018-01-01 00:00:00
c 2018-01-01 00:00:00 c 2018-01-01 00:00:00

View File

@ -0,0 +1,16 @@
USE test;
DROP TABLE IF EXISTS test.using1;
DROP TABLE IF EXISTS test.using2;
CREATE TABLE test.using1(a String, b DateTime) ENGINE=MergeTree order by a;
CREATE TABLE test.using2(c String, a String, d DateTime) ENGINE=MergeTree order by c;
INSERT INTO test.using1 VALUES ('a', '2018-01-01 00:00:00') ('b', '2018-01-01 00:00:00') ('c', '2018-01-01 00:00:00');
INSERT INTO test.using2 VALUES ('d', 'd', '2018-01-01 00:00:00') ('b', 'b', '2018-01-01 00:00:00') ('c', 'c', '2018-01-01 00:00:00');
SELECT * FROM test.using1 t1 ALL LEFT JOIN (SELECT *, c as a, d as b FROM test.using2) t2 USING (a, b) ORDER BY d;
SELECT * FROM test.using1 t1 ALL INNER JOIN (SELECT *, c as a, d as b FROM test.using2) t2 USING (a, b) ORDER BY d;
DROP TABLE test.using1;
DROP TABLE test.using2;

View File

@ -17,6 +17,8 @@ INSERT INTO table5 SELECT number * 5, number * 50, number * 500 FROM numbers(10)
SET allow_experimental_multiple_joins_emulation = 1;
SELECT 1 LIMIT 0;
-- FIXME: wrong names qualification
select a, b, c from table1 as t1 join table2 as t2 on t1.a = t2.a join table3 as t3 on b = t3.b;
select a, b, c from table1 as t1 join table2 as t2 on t1.a = t2.a join table5 as t5 on a = t5.a AND b = t5.b;

View File

@ -0,0 +1 @@
Query-Id

View File

@ -0,0 +1,7 @@
#!/usr/bin/env bash
set -e
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
${CLICKHOUSE_CURL_COMMAND} -I -sSg ${CLICKHOUSE_URL}?query=SELECT%201 | grep -o Query-Id

View File

@ -0,0 +1,23 @@
cross
1 1 1 1
2 2 2 \N
cross nullable
1 1 1 1
cross nullable vs not nullable
1 1 1 1
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableExpression (children 1)\n Identifier t2\n TableJoin\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 3)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableExpression (children 1)\n Identifier t2\n TableJoin\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 2)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableJoin (children 1)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n TableExpression (children 1)\n Identifier t2\n
Explain ParsedAST (children 1)\n SelectWithUnionQuery (children 1)\n ExpressionList (children 1)\n SelectQuery (children 2)\n ExpressionList (children 1)\n Asterisk\n TablesInSelectQuery (children 2)\n TablesInSelectQueryElement (children 1)\n TableExpression (children 1)\n Identifier t1\n TablesInSelectQueryElement (children 2)\n TableJoin (children 1)\n Function equals (children 1)\n ExpressionList (children 2)\n Identifier t1.a\n Identifier t2.a\n TableExpression (children 1)\n Identifier t2\n
cross
1 1 1 1
2 2 2 \N
cross nullable
1 1 1 1
cross nullable vs not nullable
1 1 1 1
comma
1 1 1 1
2 2 2 \N
comma nullable
1 1 1 1

View File

@ -0,0 +1,42 @@
USE test;
DROP TABLE IF EXISTS t1;
DROP TABLE IF EXISTS t2;
CREATE TABLE t1 (a Int8, b Nullable(Int8)) ENGINE = Memory;
CREATE TABLE t2 (a Int8, b Nullable(Int8)) ENGINE = Memory;
INSERT INTO t1 values (1,1), (2,2);
INSERT INTO t2 values (1,1);
INSERT INTO t2 (a) values (2), (3);
SELECT 'cross';
SELECT * FROM t1 cross join t2 where t1.a = t2.a;
SELECT 'cross nullable';
SELECT * FROM t1 cross join t2 where t1.b = t2.b;
SELECT 'cross nullable vs not nullable';
SELECT * FROM t1 cross join t2 where t1.a = t2.b;
SET enable_debug_queries = 1;
AST SELECT * FROM t1 cross join t2 where t1.a = t2.a;
AST SELECT * FROM t1, t2 where t1.a = t2.a;
SET allow_experimental_cross_to_join_conversion = 1;
AST SELECT * FROM t1 cross join t2 where t1.a = t2.a;
AST SELECT * FROM t1, t2 where t1.a = t2.a;
SELECT 'cross';
SELECT * FROM t1 cross join t2 where t1.a = t2.a;
SELECT 'cross nullable';
SELECT * FROM t1 cross join t2 where t1.b = t2.b;
SELECT 'cross nullable vs not nullable';
SELECT * FROM t1 cross join t2 where t1.a = t2.b;
SELECT 'comma';
SELECT * FROM t1, t2 where t1.a = t2.a;
SELECT 'comma nullable';
SELECT * FROM t1, t2 where t1.b = t2.b;
DROP TABLE t1;
DROP TABLE t2;

View File

@ -32,6 +32,7 @@
- [Prometheus](https://prometheus.io/)
- [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter)
- [PromHouse](https://github.com/Percona-Lab/PromHouse)
- [clickhouse_exporter](https://github.com/hot-wifi/clickhouse_exporter) (uses [Go client](https://github.com/kshvakov/clickhouse/))
- [Nagios](https://www.nagios.org/)
- [check_clickhouse](https://github.com/exogroup/check_clickhouse/)
- Logging

View File

@ -31,6 +31,7 @@
- [Prometheus](https://prometheus.io/)
- [clickhouse_exporter](https://github.com/f1yegor/clickhouse_exporter)
- [PromHouse](https://github.com/Percona-Lab/PromHouse)
- [clickhouse_exporter](https://github.com/hot-wifi/clickhouse_exporter) (использует [Go client](https://github.com/kshvakov/clickhouse/))
- [Nagios](https://www.nagios.org/)
- [check_clickhouse](https://github.com/exogroup/check_clickhouse/)
- Логирование

View File

@ -19,7 +19,7 @@ add_library (common ${LINK_MODE}
src/JSON.cpp
src/getMemoryAmount.cpp
src/demangle.cpp
src/SetTerminalEcho.cpp
src/setTerminalEcho.cpp
include/common/Types.h
include/common/DayNum.h
@ -37,7 +37,7 @@ add_library (common ${LINK_MODE}
include/common/JSON.h
include/common/getMemoryAmount.h
include/common/demangle.h
include/common/SetTerminalEcho.h
include/common/setTerminalEcho.h
include/common/find_symbols.h
include/common/constexpr_helpers.h

View File

@ -1,4 +1,4 @@
#pragma once
/// Enable or disable echoing of typed characters. Throws std::runtime_error on error.
void SetTerminalEcho(bool enable);
void setTerminalEcho(bool enable);

View File

@ -1,6 +1,6 @@
// https://stackoverflow.com/questions/1413445/reading-a-password-from-stdcin
#include <common/SetTerminalEcho.h>
#include <common/setTerminalEcho.h>
#include <stdexcept>
#include <cstring>
#include <string>
@ -13,13 +13,13 @@
#include <errno.h>
#endif
void SetTerminalEcho(bool enable)
void setTerminalEcho(bool enable)
{
#ifdef WIN32
auto handle = GetStdHandle(STD_INPUT_HANDLE);
DWORD mode;
if (!GetConsoleMode(handle, &mode))
throw std::runtime_error(std::string("SetTerminalEcho failed get: ") + std::to_string(GetLastError()));
throw std::runtime_error(std::string("setTerminalEcho failed get: ") + std::to_string(GetLastError()));
if (!enable)
mode &= ~ENABLE_ECHO_INPUT;
@ -27,11 +27,11 @@ void SetTerminalEcho(bool enable)
mode |= ENABLE_ECHO_INPUT;
if (!SetConsoleMode(handle, mode))
throw std::runtime_error(std::string("SetTerminalEcho failed set: ") + std::to_string(GetLastError()));
throw std::runtime_error(std::string("setTerminalEcho failed set: ") + std::to_string(GetLastError()));
#else
struct termios tty;
if (tcgetattr(STDIN_FILENO, &tty))
throw std::runtime_error(std::string("SetTerminalEcho failed get: ") + strerror(errno));
throw std::runtime_error(std::string("setTerminalEcho failed get: ") + strerror(errno));
if (!enable)
tty.c_lflag &= ~ECHO;
else
@ -39,6 +39,6 @@ void SetTerminalEcho(bool enable)
auto ret = tcsetattr(STDIN_FILENO, TCSANOW, &tty);
if (ret)
throw std::runtime_error(std::string("SetTerminalEcho failed set: ") + strerror(errno));
throw std::runtime_error(std::string("setTerminalEcho failed set: ") + strerror(errno));
#endif
}

View File

@ -231,6 +231,10 @@ private:
/// Previous value of logger element in config. It is used to reinitialize loggers whenever the value changed.
std::string config_logger;
/// Check SSE and others instructions availability
/// Calls exit on fail
void checkRequiredInstructions();
};

View File

@ -68,7 +68,6 @@
#include <Common/CurrentThread.h>
#include <Poco/Net/RemoteSyslogChannel.h>
/** For transferring information from signal handler to a separate thread.
* If you need to do something serious in case of a signal (example: write a message to the log),
* then sending information to a separate thread through pipe and doing all the stuff asynchronously
@ -597,8 +596,10 @@ void BaseDaemon::reloadConfiguration()
}
/// For creating and destroying unique_ptr of incomplete type.
BaseDaemon::BaseDaemon() = default;
BaseDaemon::BaseDaemon()
{
checkRequiredInstructions();
}
BaseDaemon::~BaseDaemon()
@ -609,6 +610,127 @@ BaseDaemon::~BaseDaemon()
}
enum class InstructionFail
{
NONE = 0,
SSE3 = 1,
SSSE3 = 2,
SSE4_1 = 3,
SSE4_2 = 4,
AVX = 5,
AVX2 = 6,
AVX512 = 7
};
static std::string instructionFailToString(InstructionFail fail)
{
switch(fail)
{
case InstructionFail::NONE:
return "NONE";
case InstructionFail::SSE3:
return "SSE3";
case InstructionFail::SSSE3:
return "SSSE3";
case InstructionFail::SSE4_1:
return "SSE4.1";
case InstructionFail::SSE4_2:
return "SSE4.2";
case InstructionFail::AVX:
return "AVX";
case InstructionFail::AVX2:
return "AVX2";
case InstructionFail::AVX512:
return "AVX512";
}
__builtin_unreachable();
}
static sigjmp_buf jmpbuf;
static void sigIllCheckHandler(int sig, siginfo_t * info, void * context)
{
siglongjmp(jmpbuf, 1);
}
/// Check if necessary sse extensions are available by trying to execute some sse instructions.
/// If instruction is unavailable, SIGILL will be sent by kernel.
static void checkRequiredInstructions(volatile InstructionFail & fail)
{
#if __SSE3__
fail = InstructionFail::SSE3;
__asm__ volatile ("addsubpd %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if __SSSE3__
fail = InstructionFail::SSSE3;
__asm__ volatile ("pabsw %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if __SSE4_1__
fail = InstructionFail::SSE4_1;
__asm__ volatile ("pmaxud %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if __SSE4_2__
fail = InstructionFail::SSE4_2;
__asm__ volatile ("pcmpgtq %%xmm0, %%xmm0" : : : "xmm0");
#endif
#if __AVX__
fail = InstructionFail::AVX;
__asm__ volatile ("vaddpd %%ymm0, %%ymm0" : : : "ymm0");
#endif
#if __AVX2__
fail = InstructionFail::AVX2;
__asm__ volatile ("vpabsw %%ymm0, %%ymm0" : : : "ymm0");
#endif
#if __AVX512__
fail = InstructionFail::AVX512;
__asm__ volatile ("vpabsw %%zmm0, %%zmm0" : : : "zmm0");
#endif
fail = InstructionFail::NONE;
}
void BaseDaemon::checkRequiredInstructions()
{
struct sigaction sa{};
struct sigaction sa_old{};
sa.sa_sigaction = sigIllCheckHandler;
sa.sa_flags = SA_SIGINFO;
auto signal = SIGILL;
if (sigemptyset(&sa.sa_mask) != 0
|| sigaddset(&sa.sa_mask, signal) != 0
|| sigaction(signal, &sa, &sa_old) != 0)
{
std::cerr << "Can not set signal handler\n";
exit(1);
}
volatile InstructionFail fail = InstructionFail::NONE;
if (sigsetjmp(jmpbuf, 1))
{
std::cerr << "Instruction check fail. There is no " << instructionFailToString(fail) << " instruction set\n";
exit(1);
}
::checkRequiredInstructions(fail);
if (sigaction(signal, &sa_old, nullptr))
{
std::cerr << "Can not set signal handler\n";
exit(1);
}
}
void BaseDaemon::terminate()
{
getTaskManager().cancelAll();
@ -889,16 +1011,15 @@ void BaseDaemon::initialize(Application & self)
reloadConfiguration();
/// This must be done before creation of any files (including logs).
mode_t umask_num = 0027;
if (config().has("umask"))
{
std::string umask_str = config().getString("umask");
mode_t umask_num = 0;
std::stringstream stream;
stream << umask_str;
stream >> std::oct >> umask_num;
umask(umask_num);
}
umask(umask_num);
DB::ConfigProcessor(config_path).savePreprocessedConfig(loaded_config, "");