mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-18 04:12:19 +00:00
Merge pull request #21318 from azat/servers-pool-wait
Fix SIGSEGV by waiting servers thread pool
This commit is contained in:
commit
f8ce5547aa
@ -173,18 +173,24 @@ int waitServersToFinish(std::vector<DB::ProtocolServerAdapter> & servers, size_t
|
|||||||
const int sleep_one_ms = 100;
|
const int sleep_one_ms = 100;
|
||||||
int sleep_current_ms = 0;
|
int sleep_current_ms = 0;
|
||||||
int current_connections = 0;
|
int current_connections = 0;
|
||||||
while (sleep_current_ms < sleep_max_ms)
|
for (;;)
|
||||||
{
|
{
|
||||||
current_connections = 0;
|
current_connections = 0;
|
||||||
|
|
||||||
for (auto & server : servers)
|
for (auto & server : servers)
|
||||||
{
|
{
|
||||||
server.stop();
|
server.stop();
|
||||||
current_connections += server.currentConnections();
|
current_connections += server.currentConnections();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!current_connections)
|
if (!current_connections)
|
||||||
break;
|
break;
|
||||||
|
|
||||||
sleep_current_ms += sleep_one_ms;
|
sleep_current_ms += sleep_one_ms;
|
||||||
std::this_thread::sleep_for(std::chrono::milliseconds(sleep_one_ms));
|
if (sleep_current_ms < sleep_max_ms)
|
||||||
|
std::this_thread::sleep_for(std::chrono::milliseconds(sleep_one_ms));
|
||||||
|
else
|
||||||
|
break;
|
||||||
}
|
}
|
||||||
return current_connections;
|
return current_connections;
|
||||||
}
|
}
|
||||||
@ -951,6 +957,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
|||||||
global_context->shutdownKeeperStorageDispatcher();
|
global_context->shutdownKeeperStorageDispatcher();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Wait server pool to avoid use-after-free of destroyed context in the handlers
|
||||||
|
server_pool.joinAll();
|
||||||
|
|
||||||
/** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available.
|
/** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available.
|
||||||
* At this moment, no one could own shared part of Context.
|
* At this moment, no one could own shared part of Context.
|
||||||
*/
|
*/
|
||||||
|
@ -373,7 +373,7 @@ struct ContextSharedPart
|
|||||||
std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
|
std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
|
||||||
String format_schema_path; /// Path to a directory that contains schema files used by input formats.
|
String format_schema_path; /// Path to a directory that contains schema files used by input formats.
|
||||||
ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers
|
ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers
|
||||||
std::optional<SystemLogs> system_logs; /// Used to log queries and operations on parts
|
std::unique_ptr<SystemLogs> system_logs; /// Used to log queries and operations on parts
|
||||||
std::optional<StorageS3Settings> storage_s3_settings; /// Settings of S3 storage
|
std::optional<StorageS3Settings> storage_s3_settings; /// Settings of S3 storage
|
||||||
|
|
||||||
RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml
|
RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml
|
||||||
@ -442,24 +442,32 @@ struct ContextSharedPart
|
|||||||
|
|
||||||
DatabaseCatalog::shutdown();
|
DatabaseCatalog::shutdown();
|
||||||
|
|
||||||
/// Preemptive destruction is important, because these objects may have a refcount to ContextShared (cyclic reference).
|
std::unique_ptr<SystemLogs> delete_system_logs;
|
||||||
/// TODO: Get rid of this.
|
{
|
||||||
|
auto lock = std::lock_guard(mutex);
|
||||||
|
|
||||||
system_logs.reset();
|
/// Preemptive destruction is important, because these objects may have a refcount to ContextShared (cyclic reference).
|
||||||
embedded_dictionaries.reset();
|
/// TODO: Get rid of this.
|
||||||
external_dictionaries_loader.reset();
|
|
||||||
models_repository_guard.reset();
|
|
||||||
external_models_loader.reset();
|
|
||||||
buffer_flush_schedule_pool.reset();
|
|
||||||
schedule_pool.reset();
|
|
||||||
distributed_schedule_pool.reset();
|
|
||||||
message_broker_schedule_pool.reset();
|
|
||||||
ddl_worker.reset();
|
|
||||||
|
|
||||||
/// Stop trace collector if any
|
delete_system_logs = std::move(system_logs);
|
||||||
trace_collector.reset();
|
embedded_dictionaries.reset();
|
||||||
/// Stop zookeeper connection
|
external_dictionaries_loader.reset();
|
||||||
zookeeper.reset();
|
models_repository_guard.reset();
|
||||||
|
external_models_loader.reset();
|
||||||
|
buffer_flush_schedule_pool.reset();
|
||||||
|
schedule_pool.reset();
|
||||||
|
distributed_schedule_pool.reset();
|
||||||
|
message_broker_schedule_pool.reset();
|
||||||
|
ddl_worker.reset();
|
||||||
|
|
||||||
|
/// Stop trace collector if any
|
||||||
|
trace_collector.reset();
|
||||||
|
/// Stop zookeeper connection
|
||||||
|
zookeeper.reset();
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Can be removed w/o context lock
|
||||||
|
delete_system_logs.reset();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool hasTraceCollector() const
|
bool hasTraceCollector() const
|
||||||
@ -1910,7 +1918,7 @@ void Context::setCluster(const String & cluster_name, const std::shared_ptr<Clus
|
|||||||
void Context::initializeSystemLogs()
|
void Context::initializeSystemLogs()
|
||||||
{
|
{
|
||||||
auto lock = getLock();
|
auto lock = getLock();
|
||||||
shared->system_logs.emplace(getGlobalContext(), getConfigRef());
|
shared->system_logs = std::make_unique<SystemLogs>(getGlobalContext(), getConfigRef());
|
||||||
}
|
}
|
||||||
|
|
||||||
void Context::initializeTraceCollector()
|
void Context::initializeTraceCollector()
|
||||||
|
@ -152,6 +152,8 @@ public:
|
|||||||
void shutdown() override
|
void shutdown() override
|
||||||
{
|
{
|
||||||
stopFlushThread();
|
stopFlushThread();
|
||||||
|
if (table)
|
||||||
|
table->shutdown();
|
||||||
}
|
}
|
||||||
|
|
||||||
String getName() override
|
String getName() override
|
||||||
|
@ -0,0 +1,35 @@
|
|||||||
|
<?xml version="1.0"?>
|
||||||
|
<yandex>
|
||||||
|
<logger>
|
||||||
|
<level>trace</level>
|
||||||
|
<console>true</console>
|
||||||
|
</logger>
|
||||||
|
|
||||||
|
<tcp_port>9000</tcp_port>
|
||||||
|
|
||||||
|
<path>./</path>
|
||||||
|
|
||||||
|
<mark_cache_size>0</mark_cache_size>
|
||||||
|
|
||||||
|
<users>
|
||||||
|
<default>
|
||||||
|
<password></password>
|
||||||
|
|
||||||
|
<networks>
|
||||||
|
<ip>::/0</ip>
|
||||||
|
</networks>
|
||||||
|
|
||||||
|
<profile>default</profile>
|
||||||
|
<quota>default</quota>
|
||||||
|
<access_management>1</access_management>
|
||||||
|
</default>
|
||||||
|
</users>
|
||||||
|
|
||||||
|
<profiles>
|
||||||
|
<default/>
|
||||||
|
</profiles>
|
||||||
|
|
||||||
|
<quotas>
|
||||||
|
<default />
|
||||||
|
</quotas>
|
||||||
|
</yandex>
|
83
tests/queries/0_stateless/01737_clickhouse_server_wait_server_pool_long.sh
Executable file
83
tests/queries/0_stateless/01737_clickhouse_server_wait_server_pool_long.sh
Executable file
@ -0,0 +1,83 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
|
||||||
|
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
# shellcheck source=../shell_config.sh
|
||||||
|
. "$CUR_DIR"/../shell_config.sh
|
||||||
|
|
||||||
|
server_opts=(
|
||||||
|
"--config-file=$CUR_DIR/$(basename "${BASH_SOURCE[0]}" .sh).config.xml"
|
||||||
|
"--"
|
||||||
|
# to avoid multiple listen sockets (complexity for port discovering)
|
||||||
|
"--listen_host=127.1"
|
||||||
|
# we will discover the real port later.
|
||||||
|
"--tcp_port=0"
|
||||||
|
"--shutdown_wait_unfinished=0"
|
||||||
|
)
|
||||||
|
CLICKHOUSE_WATCHDOG_ENABLE=0 $CLICKHOUSE_SERVER_BINARY "${server_opts[@]}" >& clickhouse-server.log &
|
||||||
|
server_pid=$!
|
||||||
|
|
||||||
|
trap cleanup EXIT
|
||||||
|
function cleanup()
|
||||||
|
{
|
||||||
|
kill -9 $server_pid
|
||||||
|
kill -9 $client_pid
|
||||||
|
|
||||||
|
echo "Test failed. Server log:"
|
||||||
|
cat clickhouse-server.log
|
||||||
|
rm -f clickhouse-server.log
|
||||||
|
|
||||||
|
exit 1
|
||||||
|
}
|
||||||
|
|
||||||
|
server_port=
|
||||||
|
i=0 retries=300
|
||||||
|
# wait until server will start to listen (max 30 seconds)
|
||||||
|
while [[ -z $server_port ]] && [[ $i -lt $retries ]]; do
|
||||||
|
server_port=$(lsof -n -a -P -i tcp -s tcp:LISTEN -p $server_pid 2>/dev/null | awk -F'[ :]' '/LISTEN/ { print $(NF-1) }')
|
||||||
|
((++i))
|
||||||
|
sleep 0.1
|
||||||
|
done
|
||||||
|
if [[ -z $server_port ]]; then
|
||||||
|
echo "Cannot wait for LISTEN socket" >&2
|
||||||
|
exit 1
|
||||||
|
fi
|
||||||
|
|
||||||
|
# wait for the server to start accepting tcp connections (max 30 seconds)
|
||||||
|
i=0 retries=300
|
||||||
|
while ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null -q 'select 1' 2>/dev/null && [[ $i -lt $retries ]]; do
|
||||||
|
sleep 0.1
|
||||||
|
done
|
||||||
|
if ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null -q 'select 1'; then
|
||||||
|
echo "Cannot wait until server will start accepting connections on <tcp_port>" >&2
|
||||||
|
exit 1
|
||||||
|
fi
|
||||||
|
|
||||||
|
query_id="$CLICKHOUSE_DATABASE-$SECONDS"
|
||||||
|
$CLICKHOUSE_CLIENT_BINARY --query_id "$query_id" --host 127.1 --port "$server_port" --format Null -q 'select sleepEachRow(1) from numbers(10)' 2>/dev/null &
|
||||||
|
client_pid=$!
|
||||||
|
|
||||||
|
# wait until the query will appear in processlist (max 10 second)
|
||||||
|
# (it is enough to trigger the problem)
|
||||||
|
i=0 retries=1000
|
||||||
|
while [[ $($CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" -q "select count() from system.processes where query_id = '$query_id'") != "1" ]] && [[ $i -lt $retries ]]; do
|
||||||
|
sleep 0.01
|
||||||
|
done
|
||||||
|
if [[ $($CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" -q "select count() from system.processes where query_id = '$query_id'") != "1" ]]; then
|
||||||
|
echo "Cannot wait until the query will start" >&2
|
||||||
|
exit 1
|
||||||
|
fi
|
||||||
|
|
||||||
|
# send TERM and save the error code to ensure that it is 0 (EXIT_SUCCESS)
|
||||||
|
kill $server_pid
|
||||||
|
wait $server_pid
|
||||||
|
return_code=$?
|
||||||
|
|
||||||
|
wait $client_pid
|
||||||
|
|
||||||
|
trap '' EXIT
|
||||||
|
if [ $return_code != 0 ]; then
|
||||||
|
cat clickhouse-server.log
|
||||||
|
fi
|
||||||
|
rm -f clickhouse-server.log
|
||||||
|
|
||||||
|
exit $return_code
|
@ -23,14 +23,21 @@ export CLICKHOUSE_TEST_ZOOKEEPER_PREFIX="${CLICKHOUSE_TEST_NAME}_${CLICKHOUSE_DA
|
|||||||
[ -v CLICKHOUSE_LOG_COMMENT ] && CLICKHOUSE_BENCHMARK_OPT0+=" --log_comment='${CLICKHOUSE_LOG_COMMENT}' "
|
[ -v CLICKHOUSE_LOG_COMMENT ] && CLICKHOUSE_BENCHMARK_OPT0+=" --log_comment='${CLICKHOUSE_LOG_COMMENT}' "
|
||||||
|
|
||||||
export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="clickhouse"}
|
export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="clickhouse"}
|
||||||
|
# client
|
||||||
[ -x "$CLICKHOUSE_BINARY-client" ] && CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:=$CLICKHOUSE_BINARY-client}
|
[ -x "$CLICKHOUSE_BINARY-client" ] && CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:=$CLICKHOUSE_BINARY-client}
|
||||||
[ -x "$CLICKHOUSE_BINARY" ] && CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:=$CLICKHOUSE_BINARY client}
|
[ -x "$CLICKHOUSE_BINARY" ] && CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:=$CLICKHOUSE_BINARY client}
|
||||||
export CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:=$CLICKHOUSE_BINARY-client}
|
export CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:=$CLICKHOUSE_BINARY-client}
|
||||||
export CLICKHOUSE_CLIENT_OPT="${CLICKHOUSE_CLIENT_OPT0:-} ${CLICKHOUSE_CLIENT_OPT:-}"
|
export CLICKHOUSE_CLIENT_OPT="${CLICKHOUSE_CLIENT_OPT0:-} ${CLICKHOUSE_CLIENT_OPT:-}"
|
||||||
export CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="$CLICKHOUSE_CLIENT_BINARY ${CLICKHOUSE_CLIENT_OPT:-}"}
|
export CLICKHOUSE_CLIENT=${CLICKHOUSE_CLIENT:="$CLICKHOUSE_CLIENT_BINARY ${CLICKHOUSE_CLIENT_OPT:-}"}
|
||||||
|
# local
|
||||||
[ -x "${CLICKHOUSE_BINARY}-local" ] && CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY}-local"}
|
[ -x "${CLICKHOUSE_BINARY}-local" ] && CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY}-local"}
|
||||||
[ -x "${CLICKHOUSE_BINARY}" ] && CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY} local"}
|
[ -x "${CLICKHOUSE_BINARY}" ] && CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY} local"}
|
||||||
export CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY}-local"}
|
export CLICKHOUSE_LOCAL=${CLICKHOUSE_LOCAL:="${CLICKHOUSE_BINARY}-local"}
|
||||||
|
# server
|
||||||
|
[ -x "${CLICKHOUSE_BINARY}-server" ] && CLICKHOUSE_SERVER_BINARY=${CLICKHOUSE_SERVER_BINARY:="${CLICKHOUSE_BINARY}-server"}
|
||||||
|
[ -x "${CLICKHOUSE_BINARY}" ] && CLICKHOUSE_SERVER_BINARY=${CLICKHOUSE_SERVER_BINARY:="${CLICKHOUSE_BINARY} server"}
|
||||||
|
export CLICKHOUSE_SERVER_BINARY=${CLICKHOUSE_SERVER_BINARY:="${CLICKHOUSE_BINARY}-server"}
|
||||||
|
# others
|
||||||
export CLICKHOUSE_OBFUSCATOR=${CLICKHOUSE_OBFUSCATOR:="${CLICKHOUSE_BINARY}-obfuscator"}
|
export CLICKHOUSE_OBFUSCATOR=${CLICKHOUSE_OBFUSCATOR:="${CLICKHOUSE_BINARY}-obfuscator"}
|
||||||
export CLICKHOUSE_COMPRESSOR=${CLICKHOUSE_COMPRESSOR:="${CLICKHOUSE_BINARY}-compressor"}
|
export CLICKHOUSE_COMPRESSOR=${CLICKHOUSE_COMPRESSOR:="${CLICKHOUSE_BINARY}-compressor"}
|
||||||
export CLICKHOUSE_BENCHMARK=${CLICKHOUSE_BENCHMARK:="${CLICKHOUSE_BINARY}-benchmark ${CLICKHOUSE_BENCHMARK_OPT0:-}"}
|
export CLICKHOUSE_BENCHMARK=${CLICKHOUSE_BENCHMARK:="${CLICKHOUSE_BINARY}-benchmark ${CLICKHOUSE_BENCHMARK_OPT0:-}"}
|
||||||
|
@ -696,6 +696,7 @@
|
|||||||
"01682_cache_dictionary_complex_key",
|
"01682_cache_dictionary_complex_key",
|
||||||
"01684_ssd_cache_dictionary_simple_key",
|
"01684_ssd_cache_dictionary_simple_key",
|
||||||
"01685_ssd_cache_dictionary_complex_key",
|
"01685_ssd_cache_dictionary_complex_key",
|
||||||
|
"01737_clickhouse_server_wait_server_pool_long", // This test is fully compatible to run in parallel, however under ASAN processes are pretty heavy and may fail under flaky adress check.
|
||||||
"01760_system_dictionaries",
|
"01760_system_dictionaries",
|
||||||
"01760_polygon_dictionaries",
|
"01760_polygon_dictionaries",
|
||||||
"01778_hierarchical_dictionaries",
|
"01778_hierarchical_dictionaries",
|
||||||
|
Loading…
Reference in New Issue
Block a user