mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Merge branch 'master' into no-export-dynamic
This commit is contained in:
commit
10aa12d3b3
@ -744,11 +744,12 @@ try
|
||||
[&]() -> std::vector<ProtocolServerMetrics>
|
||||
{
|
||||
std::vector<ProtocolServerMetrics> metrics;
|
||||
metrics.reserve(servers_to_start_before_tables.size());
|
||||
|
||||
std::lock_guard lock(servers_lock);
|
||||
metrics.reserve(servers_to_start_before_tables.size() + servers.size());
|
||||
for (const auto & server : servers_to_start_before_tables)
|
||||
metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()});
|
||||
|
||||
std::lock_guard lock(servers_lock);
|
||||
for (const auto & server : servers)
|
||||
metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()});
|
||||
return metrics;
|
||||
@ -1311,7 +1312,7 @@ try
|
||||
global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config);
|
||||
|
||||
std::lock_guard lock(servers_lock);
|
||||
updateServers(*config, server_pool, async_metrics, servers);
|
||||
updateServers(*config, server_pool, async_metrics, servers, servers_to_start_before_tables);
|
||||
}
|
||||
|
||||
global_context->updateStorageConfiguration(*config);
|
||||
@ -1413,11 +1414,28 @@ try
|
||||
|
||||
}
|
||||
|
||||
{
|
||||
std::lock_guard lock(servers_lock);
|
||||
/// We should start interserver communications before (and more imporant shutdown after) tables.
|
||||
/// Because server can wait for a long-running queries (for example in tcp_handler) after interserver handler was already shut down.
|
||||
/// In this case we will have replicated tables which are unable to send any parts to other replicas, but still can
|
||||
/// communicate with zookeeper, execute merges, etc.
|
||||
createInterserverServers(
|
||||
config(),
|
||||
interserver_listen_hosts,
|
||||
listen_try,
|
||||
server_pool,
|
||||
async_metrics,
|
||||
servers_to_start_before_tables,
|
||||
/* start_servers= */ false);
|
||||
|
||||
|
||||
for (auto & server : servers_to_start_before_tables)
|
||||
{
|
||||
server.start();
|
||||
LOG_INFO(log, "Listening for {}", server.getDescription());
|
||||
}
|
||||
}
|
||||
|
||||
/// Initialize access storages.
|
||||
auto & access_control = global_context->getAccessControl();
|
||||
@ -1536,11 +1554,14 @@ try
|
||||
{
|
||||
LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish.");
|
||||
size_t current_connections = 0;
|
||||
{
|
||||
std::lock_guard lock(servers_lock);
|
||||
for (auto & server : servers_to_start_before_tables)
|
||||
{
|
||||
server.stop();
|
||||
current_connections += server.currentConnections();
|
||||
}
|
||||
}
|
||||
|
||||
if (current_connections)
|
||||
LOG_INFO(log, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections);
|
||||
@ -1712,7 +1733,7 @@ try
|
||||
|
||||
{
|
||||
std::lock_guard lock(servers_lock);
|
||||
createServers(config(), listen_hosts, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers);
|
||||
createServers(config(), listen_hosts, listen_try, server_pool, async_metrics, servers);
|
||||
if (servers.empty())
|
||||
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG,
|
||||
"No servers started (add valid listen_host and 'tcp_port' or 'http_port' "
|
||||
@ -1970,7 +1991,6 @@ HTTPContextPtr Server::httpContext() const
|
||||
void Server::createServers(
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
const Strings & listen_hosts,
|
||||
const Strings & interserver_listen_hosts,
|
||||
bool listen_try,
|
||||
Poco::ThreadPool & server_pool,
|
||||
AsynchronousMetrics & async_metrics,
|
||||
@ -2192,6 +2212,23 @@ void Server::createServers(
|
||||
httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
void Server::createInterserverServers(
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
const Strings & interserver_listen_hosts,
|
||||
bool listen_try,
|
||||
Poco::ThreadPool & server_pool,
|
||||
AsynchronousMetrics & async_metrics,
|
||||
std::vector<ProtocolServerAdapter> & servers,
|
||||
bool start_servers)
|
||||
{
|
||||
const Settings & settings = global_context->getSettingsRef();
|
||||
|
||||
Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0);
|
||||
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
|
||||
http_params->setTimeout(settings.http_receive_timeout);
|
||||
http_params->setKeepAliveTimeout(keep_alive_timeout);
|
||||
|
||||
/// Now iterate over interserver_listen_hosts
|
||||
for (const auto & interserver_listen_host : interserver_listen_hosts)
|
||||
@ -2240,14 +2277,14 @@ void Server::createServers(
|
||||
#endif
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void Server::updateServers(
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
Poco::ThreadPool & server_pool,
|
||||
AsynchronousMetrics & async_metrics,
|
||||
std::vector<ProtocolServerAdapter> & servers)
|
||||
std::vector<ProtocolServerAdapter> & servers,
|
||||
std::vector<ProtocolServerAdapter> & servers_to_start_before_tables)
|
||||
{
|
||||
Poco::Logger * log = &logger();
|
||||
|
||||
@ -2273,11 +2310,19 @@ void Server::updateServers(
|
||||
|
||||
Poco::Util::AbstractConfiguration & previous_config = latest_config ? *latest_config : this->config();
|
||||
|
||||
std::vector<ProtocolServerAdapter *> all_servers;
|
||||
all_servers.reserve(servers.size() + servers_to_start_before_tables.size());
|
||||
for (auto & server : servers)
|
||||
all_servers.push_back(&server);
|
||||
|
||||
for (auto & server : servers_to_start_before_tables)
|
||||
all_servers.push_back(&server);
|
||||
|
||||
for (auto * server : all_servers)
|
||||
{
|
||||
if (!server.isStopping())
|
||||
if (!server->isStopping())
|
||||
{
|
||||
std::string port_name = server.getPortName();
|
||||
std::string port_name = server->getPortName();
|
||||
bool has_host = false;
|
||||
bool is_http = false;
|
||||
if (port_name.starts_with("protocols."))
|
||||
@ -2315,27 +2360,29 @@ void Server::updateServers(
|
||||
/// NOTE: better to compare using getPortName() over using
|
||||
/// dynamic_cast<> since HTTPServer is also used for prometheus and
|
||||
/// internal replication communications.
|
||||
is_http = server.getPortName() == "http_port" || server.getPortName() == "https_port";
|
||||
is_http = server->getPortName() == "http_port" || server->getPortName() == "https_port";
|
||||
}
|
||||
|
||||
if (!has_host)
|
||||
has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server.getListenHost()) != listen_hosts.end();
|
||||
has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server->getListenHost()) != listen_hosts.end();
|
||||
bool has_port = !config.getString(port_name, "").empty();
|
||||
bool force_restart = is_http && !isSameConfiguration(previous_config, config, "http_handlers");
|
||||
if (force_restart)
|
||||
LOG_TRACE(log, "<http_handlers> had been changed, will reload {}", server.getDescription());
|
||||
LOG_TRACE(log, "<http_handlers> had been changed, will reload {}", server->getDescription());
|
||||
|
||||
if (!has_host || !has_port || config.getInt(server.getPortName()) != server.portNumber() || force_restart)
|
||||
if (!has_host || !has_port || config.getInt(server->getPortName()) != server->portNumber() || force_restart)
|
||||
{
|
||||
server.stop();
|
||||
LOG_INFO(log, "Stopped listening for {}", server.getDescription());
|
||||
server->stop();
|
||||
LOG_INFO(log, "Stopped listening for {}", server->getDescription());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
createServers(config, listen_hosts, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true);
|
||||
createServers(config, listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true);
|
||||
createInterserverServers(config, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ true);
|
||||
|
||||
std::erase_if(servers, std::bind_front(check_server, ""));
|
||||
std::erase_if(servers_to_start_before_tables, std::bind_front(check_server, ""));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -102,6 +102,14 @@ private:
|
||||
void createServers(
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
const Strings & listen_hosts,
|
||||
bool listen_try,
|
||||
Poco::ThreadPool & server_pool,
|
||||
AsynchronousMetrics & async_metrics,
|
||||
std::vector<ProtocolServerAdapter> & servers,
|
||||
bool start_servers = false);
|
||||
|
||||
void createInterserverServers(
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
const Strings & interserver_listen_hosts,
|
||||
bool listen_try,
|
||||
Poco::ThreadPool & server_pool,
|
||||
@ -113,7 +121,8 @@ private:
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
Poco::ThreadPool & server_pool,
|
||||
AsynchronousMetrics & async_metrics,
|
||||
std::vector<ProtocolServerAdapter> & servers);
|
||||
std::vector<ProtocolServerAdapter> & servers,
|
||||
std::vector<ProtocolServerAdapter> & servers_to_start_before_tables);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -548,15 +548,17 @@ void ExpressionAnalyzer::getRootActionsForWindowFunctions(const ASTPtr & ast, bo
|
||||
|
||||
void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, AggregateDescriptions & descriptions)
|
||||
{
|
||||
for (const ASTFunction * node : aggregates())
|
||||
for (const ASTPtr & ast : aggregates())
|
||||
{
|
||||
const ASTFunction & node = typeid_cast<const ASTFunction &>(*ast);
|
||||
|
||||
AggregateDescription aggregate;
|
||||
if (node->arguments)
|
||||
getRootActionsNoMakeSet(node->arguments, actions);
|
||||
if (node.arguments)
|
||||
getRootActionsNoMakeSet(node.arguments, actions);
|
||||
|
||||
aggregate.column_name = node->getColumnName();
|
||||
aggregate.column_name = node.getColumnName();
|
||||
|
||||
const ASTs & arguments = node->arguments ? node->arguments->children : ASTs();
|
||||
const ASTs & arguments = node.arguments ? node.arguments->children : ASTs();
|
||||
aggregate.argument_names.resize(arguments.size());
|
||||
DataTypes types(arguments.size());
|
||||
|
||||
@ -568,7 +570,7 @@ void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, Aggr
|
||||
{
|
||||
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
|
||||
"Unknown identifier '{}' in aggregate function '{}'",
|
||||
name, node->formatForErrorMessage());
|
||||
name, node.formatForErrorMessage());
|
||||
}
|
||||
|
||||
types[i] = dag_node->result_type;
|
||||
@ -576,8 +578,8 @@ void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, Aggr
|
||||
}
|
||||
|
||||
AggregateFunctionProperties properties;
|
||||
aggregate.parameters = (node->parameters) ? getAggregateFunctionParametersArray(node->parameters, "", getContext()) : Array();
|
||||
aggregate.function = AggregateFunctionFactory::instance().get(node->name, types, aggregate.parameters, properties);
|
||||
aggregate.parameters = (node.parameters) ? getAggregateFunctionParametersArray(node.parameters, "", getContext()) : Array();
|
||||
aggregate.function = AggregateFunctionFactory::instance().get(node.name, types, aggregate.parameters, properties);
|
||||
|
||||
descriptions.push_back(aggregate);
|
||||
}
|
||||
@ -744,12 +746,13 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
|
||||
}
|
||||
|
||||
// Window functions
|
||||
for (const ASTFunction * function_node : syntax->window_function_asts)
|
||||
for (const ASTPtr & ast : syntax->window_function_asts)
|
||||
{
|
||||
assert(function_node->is_window_function);
|
||||
const ASTFunction & function_node = typeid_cast<const ASTFunction &>(*ast);
|
||||
assert(function_node.is_window_function);
|
||||
|
||||
WindowFunctionDescription window_function;
|
||||
window_function.function_node = function_node;
|
||||
window_function.function_node = &function_node;
|
||||
window_function.column_name
|
||||
= window_function.function_node->getColumnName();
|
||||
window_function.function_parameters
|
||||
@ -760,7 +763,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
|
||||
|
||||
// Requiring a constant reference to a shared pointer to non-const AST
|
||||
// doesn't really look sane, but the visitor does indeed require it.
|
||||
// Hence we clone the node (not very sane either, I know).
|
||||
// Hence, we clone the node (not very sane either, I know).
|
||||
getRootActionsNoMakeSet(window_function.function_node->clone(), actions);
|
||||
|
||||
const ASTs & arguments
|
||||
@ -793,22 +796,22 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions)
|
||||
// Find the window corresponding to this function. It may be either
|
||||
// referenced by name and previously defined in WINDOW clause, or it
|
||||
// may be defined inline.
|
||||
if (!function_node->window_name.empty())
|
||||
if (!function_node.window_name.empty())
|
||||
{
|
||||
auto it = window_descriptions.find(function_node->window_name);
|
||||
auto it = window_descriptions.find(function_node.window_name);
|
||||
if (it == std::end(window_descriptions))
|
||||
{
|
||||
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
|
||||
"Window '{}' is not defined (referenced by '{}')",
|
||||
function_node->window_name,
|
||||
function_node->formatForErrorMessage());
|
||||
function_node.window_name,
|
||||
function_node.formatForErrorMessage());
|
||||
}
|
||||
|
||||
it->second.window_functions.push_back(window_function);
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto & definition = function_node->window_definition->as<
|
||||
const auto & definition = function_node.window_definition->as<
|
||||
const ASTWindowDefinition &>();
|
||||
WindowDescription desc;
|
||||
desc.window_name = definition.getDefaultWindowName();
|
||||
@ -1323,11 +1326,14 @@ void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(Expression
|
||||
GetAggregatesVisitor(data).visit(select_query->orderBy());
|
||||
|
||||
/// TODO: data.aggregates -> aggregates()
|
||||
for (const ASTFunction * node : data.aggregates)
|
||||
if (node->arguments)
|
||||
for (auto & argument : node->arguments->children)
|
||||
for (const ASTPtr & ast : data.aggregates)
|
||||
{
|
||||
const ASTFunction & node = typeid_cast<const ASTFunction &>(*ast);
|
||||
if (node.arguments)
|
||||
for (auto & argument : node.arguments->children)
|
||||
getRootActions(argument, only_types, step.actions());
|
||||
}
|
||||
}
|
||||
|
||||
void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments(
|
||||
ExpressionActionsChain & chain, bool /* only_types */)
|
||||
|
@ -168,7 +168,7 @@ protected:
|
||||
const ConstStoragePtr & storage() const { return syntax->storage; } /// The main table in FROM clause, if exists.
|
||||
const TableJoin & analyzedJoin() const { return *syntax->analyzed_join; }
|
||||
const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; }
|
||||
const std::vector<const ASTFunction *> & aggregates() const { return syntax->aggregates; }
|
||||
const ASTs & aggregates() const { return syntax->aggregates; }
|
||||
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
|
||||
void initGlobalSubqueriesAndExternalTables(bool do_global, bool is_explain);
|
||||
|
||||
|
@ -26,8 +26,8 @@ public:
|
||||
// Explicit empty initializers are needed to make designated initializers
|
||||
// work on GCC 10.
|
||||
std::unordered_set<String> uniq_names {};
|
||||
std::vector<const ASTFunction *> aggregates {};
|
||||
std::vector<const ASTFunction *> window_functions {};
|
||||
ASTs aggregates;
|
||||
ASTs window_functions;
|
||||
};
|
||||
|
||||
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child)
|
||||
@ -61,7 +61,7 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
static void visit(const ASTFunction & node, const ASTPtr &, Data & data)
|
||||
static void visit(const ASTFunction & node, const ASTPtr & ast, Data & data)
|
||||
{
|
||||
if (isAggregateFunction(node))
|
||||
{
|
||||
@ -74,7 +74,7 @@ private:
|
||||
return;
|
||||
|
||||
data.uniq_names.insert(column_name);
|
||||
data.aggregates.push_back(&node);
|
||||
data.aggregates.push_back(ast);
|
||||
}
|
||||
else if (node.is_window_function)
|
||||
{
|
||||
@ -87,7 +87,7 @@ private:
|
||||
return;
|
||||
|
||||
data.uniq_names.insert(column_name);
|
||||
data.window_functions.push_back(&node);
|
||||
data.window_functions.push_back(ast);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -731,7 +731,7 @@ void expandGroupByAll(ASTSelectQuery * select_query)
|
||||
select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, group_expression_list);
|
||||
}
|
||||
|
||||
std::vector<const ASTFunction *> getAggregates(ASTPtr & query, const ASTSelectQuery & select_query)
|
||||
ASTs getAggregates(ASTPtr & query, const ASTSelectQuery & select_query)
|
||||
{
|
||||
/// There can not be aggregate functions inside the WHERE and PREWHERE.
|
||||
if (select_query.where())
|
||||
@ -743,11 +743,12 @@ std::vector<const ASTFunction *> getAggregates(ASTPtr & query, const ASTSelectQu
|
||||
GetAggregatesVisitor(data).visit(query);
|
||||
|
||||
/// There can not be other aggregate functions within the aggregate functions.
|
||||
for (const ASTFunction * node : data.aggregates)
|
||||
for (const ASTPtr & ast : data.aggregates)
|
||||
{
|
||||
if (node->arguments)
|
||||
const ASTFunction & node = typeid_cast<const ASTFunction &>(*ast);
|
||||
if (node.arguments)
|
||||
{
|
||||
for (auto & arg : node->arguments->children)
|
||||
for (auto & arg : node.arguments->children)
|
||||
{
|
||||
assertNoAggregates(arg, "inside another aggregate function");
|
||||
// We also can't have window functions inside aggregate functions,
|
||||
@ -759,7 +760,7 @@ std::vector<const ASTFunction *> getAggregates(ASTPtr & query, const ASTSelectQu
|
||||
return data.aggregates;
|
||||
}
|
||||
|
||||
std::vector<const ASTFunction *> getWindowFunctions(ASTPtr & query, const ASTSelectQuery & select_query)
|
||||
ASTs getWindowFunctions(ASTPtr & query, const ASTSelectQuery & select_query)
|
||||
{
|
||||
/// There can not be window functions inside the WHERE, PREWHERE and HAVING
|
||||
if (select_query.having())
|
||||
@ -777,20 +778,16 @@ std::vector<const ASTFunction *> getWindowFunctions(ASTPtr & query, const ASTSel
|
||||
/// Window functions cannot be inside aggregates or other window functions.
|
||||
/// Aggregate functions can be inside window functions because they are
|
||||
/// calculated earlier.
|
||||
for (const ASTFunction * node : data.window_functions)
|
||||
for (const ASTPtr & ast : data.window_functions)
|
||||
{
|
||||
if (node->arguments)
|
||||
{
|
||||
for (auto & arg : node->arguments->children)
|
||||
{
|
||||
assertNoWindows(arg, "inside another window function");
|
||||
}
|
||||
}
|
||||
const ASTFunction & node = typeid_cast<const ASTFunction &>(*ast);
|
||||
|
||||
if (node->window_definition)
|
||||
{
|
||||
assertNoWindows(node->window_definition, "inside window definition");
|
||||
}
|
||||
if (node.arguments)
|
||||
for (auto & arg : node.arguments->children)
|
||||
assertNoWindows(arg, "inside another window function");
|
||||
|
||||
if (node.window_definition)
|
||||
assertNoWindows(node.window_definition, "inside window definition");
|
||||
}
|
||||
|
||||
return data.window_functions;
|
||||
@ -1357,8 +1354,8 @@ TreeRewriterResultPtr TreeRewriter::analyze(
|
||||
GetAggregatesVisitor(data).visit(query);
|
||||
|
||||
/// There can not be other aggregate functions within the aggregate functions.
|
||||
for (const ASTFunction * node : data.aggregates)
|
||||
for (auto & arg : node->arguments->children)
|
||||
for (const ASTPtr & node : data.aggregates)
|
||||
for (auto & arg : typeid_cast<const ASTFunction &>(*node).arguments->children)
|
||||
assertNoAggregates(arg, "inside another aggregate function");
|
||||
result.aggregates = data.aggregates;
|
||||
}
|
||||
|
@ -41,8 +41,8 @@ struct TreeRewriterResult
|
||||
|
||||
Aliases aliases;
|
||||
|
||||
std::vector<const ASTFunction *> aggregates;
|
||||
std::vector<const ASTFunction *> window_function_asts;
|
||||
ASTs aggregates;
|
||||
ASTs window_function_asts;
|
||||
ASTs expressions_with_window_function;
|
||||
|
||||
/// Which column is needed to be ARRAY-JOIN'ed to get the specified.
|
||||
|
@ -190,7 +190,7 @@ def clear_ip_tables_and_restart_daemons():
|
||||
try:
|
||||
logging.info("Killing all alive docker containers")
|
||||
subprocess.check_output(
|
||||
"timeout -s 9 10m docker ps --quiet | xargs --no-run-if-empty docker kill",
|
||||
"timeout --signal=KILL 10m docker ps --quiet | xargs --no-run-if-empty docker kill",
|
||||
shell=True,
|
||||
)
|
||||
except subprocess.CalledProcessError as err:
|
||||
@ -199,7 +199,7 @@ def clear_ip_tables_and_restart_daemons():
|
||||
try:
|
||||
logging.info("Removing all docker containers")
|
||||
subprocess.check_output(
|
||||
"timeout -s 9 10m docker ps --all --quiet | xargs --no-run-if-empty docker rm --force",
|
||||
"timeout --signal=KILL 10m docker ps --all --quiet | xargs --no-run-if-empty docker rm --force",
|
||||
shell=True,
|
||||
)
|
||||
except subprocess.CalledProcessError as err:
|
||||
@ -321,7 +321,7 @@ class ClickhouseIntegrationTestsRunner:
|
||||
|
||||
cmd = (
|
||||
"cd {repo_path}/tests/integration && "
|
||||
"timeout -s 9 1h ./runner {runner_opts} {image_cmd} --pre-pull --command '{command}' ".format(
|
||||
"timeout --signal=KILL 1h ./runner {runner_opts} {image_cmd} --pre-pull --command '{command}' ".format(
|
||||
repo_path=repo_path,
|
||||
runner_opts=self._get_runner_opts(),
|
||||
image_cmd=image_cmd,
|
||||
@ -433,9 +433,9 @@ class ClickhouseIntegrationTestsRunner:
|
||||
out_file_full = os.path.join(self.result_path, "runner_get_all_tests.log")
|
||||
cmd = (
|
||||
"cd {repo_path}/tests/integration && "
|
||||
"timeout -s 9 1h ./runner {runner_opts} {image_cmd} -- --setup-plan "
|
||||
"| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' "
|
||||
"| grep -v 'SKIPPED' | sort -u > {out_file}".format(
|
||||
"timeout --signal=KILL 1h ./runner {runner_opts} {image_cmd} -- --setup-plan "
|
||||
"| tee '{out_file_full}' | grep -F '::' | sed -r 's/ \(fixtures used:.*//g; s/^ *//g; s/ *$//g' "
|
||||
"| grep -v -F 'SKIPPED' | sort --unique > {out_file}".format(
|
||||
repo_path=repo_path,
|
||||
runner_opts=self._get_runner_opts(),
|
||||
image_cmd=image_cmd,
|
||||
@ -677,7 +677,7 @@ class ClickhouseIntegrationTestsRunner:
|
||||
# -E -- (E)rror
|
||||
# -p -- (p)assed
|
||||
# -s -- (s)kipped
|
||||
cmd = "cd {}/tests/integration && timeout -s 9 1h ./runner {} {} -t {} {} -- -rfEps --run-id={} --color=no --durations=0 {} | tee {}".format(
|
||||
cmd = "cd {}/tests/integration && timeout --signal=KILL 1h ./runner {} {} -t {} {} -- -rfEps --run-id={} --color=no --durations=0 {} | tee {}".format(
|
||||
repo_path,
|
||||
self._get_runner_opts(),
|
||||
image_cmd,
|
||||
|
@ -1,11 +1,10 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g')
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS check;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE check (x UInt64, y UInt64 DEFAULT throwIf(x > 1500000)) ENGINE = Memory;"
|
||||
|
@ -1,11 +1,10 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g')
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="""
|
||||
SELECT * FROM (SELECT number % 5 AS a, count() AS b, c FROM numbers(10)
|
||||
ARRAY JOIN [1,2] AS c GROUP BY a,c) AS table
|
||||
|
@ -4,11 +4,10 @@
|
||||
set -e
|
||||
|
||||
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CUR_DIR"/../shell_config.sh
|
||||
|
||||
CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g')
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t"
|
||||
${CLICKHOUSE_CLIENT} --query="CREATE TABLE t (x Int8) ENGINE = MergeTree ORDER BY tuple()"
|
||||
|
||||
|
@ -2,9 +2,8 @@
|
||||
# Tags: race
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=debug
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g')
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query="SELECT logTrace('logTrace Function Test');" 2>&1 | grep -q "logTrace Function Test" && echo "OK" || echo "FAIL"
|
||||
|
@ -1,11 +1,10 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g')
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS check;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="CREATE TABLE check (x UInt64) ENGINE = Memory;"
|
||||
|
@ -1,11 +1,11 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
[ ! -z "$CLICKHOUSE_CLIENT_REDEFINED" ] && CLICKHOUSE_CLIENT=$CLICKHOUSE_CLIENT_REDEFINED
|
||||
|
||||
CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=trace/g')
|
||||
regexp="executeQuery|InterpreterSelectQuery"
|
||||
$CLICKHOUSE_CLIENT --send_logs_source_regexp "$regexp" -q "SELECT 1;" 2> >(grep -v -E "$regexp" 1>&2)
|
||||
|
Loading…
Reference in New Issue
Block a user