mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-12 02:23:14 +00:00
Merge branch 'ClickHouse:master' into master
This commit is contained in:
commit
5a4580c454
@ -1,5 +1,5 @@
|
|||||||
#!/bin/bash
|
#!/bin/bash
|
||||||
# shellcheck disable=SC2086
|
# shellcheck disable=SC2086,SC2001
|
||||||
|
|
||||||
set -eux
|
set -eux
|
||||||
set -o pipefail
|
set -o pipefail
|
||||||
@ -76,7 +76,10 @@ function filter_exists_and_template
|
|||||||
local path
|
local path
|
||||||
for path in "$@"; do
|
for path in "$@"; do
|
||||||
if [ -e "$path" ]; then
|
if [ -e "$path" ]; then
|
||||||
echo "$path" | sed -n 's/\.sql\.j2$/.gen.sql/'
|
# SC2001 shellcheck suggests:
|
||||||
|
# echo ${path//.sql.j2/.gen.sql}
|
||||||
|
# but it doesn't allow to use regex
|
||||||
|
echo "$path" | sed 's/\.sql\.j2$/.gen.sql/'
|
||||||
else
|
else
|
||||||
echo "'$path' does not exists" >&2
|
echo "'$path' does not exists" >&2
|
||||||
fi
|
fi
|
||||||
|
@ -84,28 +84,55 @@ bool incrementMetricIfLessThanMax(std::atomic<Int64> & atomic_value, Int64 max_v
|
|||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// This is a RAII class which only decrements metric.
|
||||||
|
/// It is added because after all other fixes a bug non-executing merges was occurred again.
|
||||||
|
/// Last hypothesis: task was successfully added to pool, however, was not executed because of internal exception in it.
|
||||||
|
class ParanoidMetricDecrementor
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
explicit ParanoidMetricDecrementor(CurrentMetrics::Metric metric_) : metric(metric_) {}
|
||||||
|
void alarm() { is_alarmed = true; }
|
||||||
|
void decrement()
|
||||||
|
{
|
||||||
|
if (is_alarmed.exchange(false))
|
||||||
|
{
|
||||||
|
CurrentMetrics::values[metric]--;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
~ParanoidMetricDecrementor() { decrement(); }
|
||||||
|
|
||||||
|
private:
|
||||||
|
|
||||||
|
CurrentMetrics::Metric metric;
|
||||||
|
std::atomic_bool is_alarmed = false;
|
||||||
|
};
|
||||||
|
|
||||||
void IBackgroundJobExecutor::execute(JobAndPool job_and_pool)
|
void IBackgroundJobExecutor::execute(JobAndPool job_and_pool)
|
||||||
try
|
try
|
||||||
{
|
{
|
||||||
auto & pool_config = pools_configs[job_and_pool.pool_type];
|
auto & pool_config = pools_configs[job_and_pool.pool_type];
|
||||||
const auto max_pool_size = pool_config.get_max_pool_size();
|
const auto max_pool_size = pool_config.get_max_pool_size();
|
||||||
|
|
||||||
|
auto metric_decrementor = std::make_shared<ParanoidMetricDecrementor>(pool_config.tasks_metric);
|
||||||
|
|
||||||
/// If corresponding pool is not full increment metric and assign new job
|
/// If corresponding pool is not full increment metric and assign new job
|
||||||
if (incrementMetricIfLessThanMax(CurrentMetrics::values[pool_config.tasks_metric], max_pool_size))
|
if (incrementMetricIfLessThanMax(CurrentMetrics::values[pool_config.tasks_metric], max_pool_size))
|
||||||
{
|
{
|
||||||
|
metric_decrementor->alarm();
|
||||||
try /// this try required because we have to manually decrement metric
|
try /// this try required because we have to manually decrement metric
|
||||||
{
|
{
|
||||||
/// Synchronize pool size, because config could be reloaded
|
/// Synchronize pool size, because config could be reloaded
|
||||||
pools[job_and_pool.pool_type].setMaxThreads(max_pool_size);
|
pools[job_and_pool.pool_type].setMaxThreads(max_pool_size);
|
||||||
pools[job_and_pool.pool_type].setQueueSize(max_pool_size);
|
pools[job_and_pool.pool_type].setQueueSize(max_pool_size);
|
||||||
|
|
||||||
pools[job_and_pool.pool_type].scheduleOrThrowOnError([this, pool_config, job{std::move(job_and_pool.job)}] ()
|
pools[job_and_pool.pool_type].scheduleOrThrowOnError([this, metric_decrementor, job{std::move(job_and_pool.job)}] ()
|
||||||
{
|
{
|
||||||
try /// We don't want exceptions in background pool
|
try /// We don't want exceptions in background pool
|
||||||
{
|
{
|
||||||
bool job_success = job();
|
bool job_success = job();
|
||||||
/// Job done, decrement metric and reset no_work counter
|
/// Job done, decrement metric and reset no_work counter
|
||||||
CurrentMetrics::values[pool_config.tasks_metric]--;
|
metric_decrementor->decrement();
|
||||||
|
|
||||||
if (job_success)
|
if (job_success)
|
||||||
{
|
{
|
||||||
@ -121,7 +148,7 @@ try
|
|||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
CurrentMetrics::values[pool_config.tasks_metric]--;
|
metric_decrementor->decrement();
|
||||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||||
scheduleTask(/* with_backoff = */ true);
|
scheduleTask(/* with_backoff = */ true);
|
||||||
}
|
}
|
||||||
@ -133,7 +160,7 @@ try
|
|||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
/// With our Pool settings scheduleOrThrowOnError shouldn't throw exceptions, but for safety catch added here
|
/// With our Pool settings scheduleOrThrowOnError shouldn't throw exceptions, but for safety catch added here
|
||||||
CurrentMetrics::values[pool_config.tasks_metric]--;
|
metric_decrementor->decrement();
|
||||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||||
scheduleTask(/* with_backoff = */ true);
|
scheduleTask(/* with_backoff = */ true);
|
||||||
}
|
}
|
||||||
|
@ -192,7 +192,7 @@ void ReplicatedMergeTreePartCheckThread::searchForMissingPartAndFetchIfPossible(
|
|||||||
if (missing_part_search_result == MissingPartSearchResult::LostForever)
|
if (missing_part_search_result == MissingPartSearchResult::LostForever)
|
||||||
{
|
{
|
||||||
auto lost_part_info = MergeTreePartInfo::fromPartName(part_name, storage.format_version);
|
auto lost_part_info = MergeTreePartInfo::fromPartName(part_name, storage.format_version);
|
||||||
if (lost_part_info.level != 0)
|
if (lost_part_info.level != 0 || lost_part_info.mutation != 0)
|
||||||
{
|
{
|
||||||
Strings source_parts;
|
Strings source_parts;
|
||||||
bool part_in_queue = storage.queue.checkPartInQueueAndGetSourceParts(part_name, source_parts);
|
bool part_in_queue = storage.queue.checkPartInQueueAndGetSourceParts(part_name, source_parts);
|
||||||
|
@ -4,25 +4,25 @@ SET joined_subquery_requires_alias = 0;
|
|||||||
|
|
||||||
SET join_algorithm = '{{ join_algorithm }}';
|
SET join_algorithm = '{{ join_algorithm }}';
|
||||||
|
|
||||||
SELECT 'IN empty set',count() FROM system.numbers WHERE number IN (SELECT toUInt64(1) WHERE 0);
|
SELECT 'IN empty set', count() FROM system.numbers WHERE number IN (SELECT toUInt64(1) WHERE 0);
|
||||||
SELECT 'IN non-empty set',count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 WHERE t1.number IN (SELECT toUInt64(1) WHERE 1);
|
SELECT 'IN non-empty set', count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 WHERE t1.number IN (SELECT toUInt64(1) WHERE 1);
|
||||||
SELECT 'NOT IN empty set',count() FROM (SELECT number FROM system.numbers LIMIT 10) WHERE number NOT IN (SELECT toUInt64(1) WHERE 0);
|
SELECT 'NOT IN empty set', count() FROM (SELECT number FROM system.numbers LIMIT 10) WHERE number NOT IN (SELECT toUInt64(1) WHERE 0);
|
||||||
|
|
||||||
SELECT 'INNER JOIN empty set',count() FROM system.numbers INNER JOIN (SELECT toUInt64(1) AS x WHERE 0) ON system.numbers.number = x;
|
SELECT 'INNER JOIN empty set', count() FROM system.numbers INNER JOIN (SELECT toUInt64(1) AS x WHERE 0) ON system.numbers.number = x;
|
||||||
SELECT 'INNER JOIN non-empty set',count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 INNER JOIN (SELECT toUInt64(1) AS x WHERE 1) ON t1.number = x;
|
SELECT 'INNER JOIN non-empty set', count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 INNER JOIN (SELECT toUInt64(1) AS x WHERE 1) ON t1.number = x;
|
||||||
|
|
||||||
SELECT 'RIGHT JOIN empty set',count() FROM system.numbers RIGHT JOIN (SELECT toUInt64(1) AS x WHERE 0) ON system.numbers.number = x;
|
SELECT 'RIGHT JOIN empty set', count() FROM system.numbers RIGHT JOIN (SELECT toUInt64(1) AS x WHERE 0) ON system.numbers.number = x;
|
||||||
SELECT 'RIGHT JOIN non-empty set',count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 RIGHT JOIN (SELECT toUInt64(1) AS x WHERE 1) ON t1.number = x;
|
SELECT 'RIGHT JOIN non-empty set', count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 RIGHT JOIN (SELECT toUInt64(1) AS x WHERE 1) ON t1.number = x;
|
||||||
|
|
||||||
SELECT 'LEFT JOIN empty set',count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 LEFT JOIN (SELECT toUInt64(1) AS x WHERE 0) ON t1.number = x;
|
SELECT 'LEFT JOIN empty set', count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 LEFT JOIN (SELECT toUInt64(1) AS x WHERE 0) ON t1.number = x;
|
||||||
SELECT 'LEFT JOIN non-empty set',count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 LEFT JOIN (SELECT toUInt64(1) AS x WHERE 1) ON t1.number = x;
|
SELECT 'LEFT JOIN non-empty set', count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 LEFT JOIN (SELECT toUInt64(1) AS x WHERE 1) ON t1.number = x;
|
||||||
|
|
||||||
SELECT 'multiple sets IN empty set OR IN non-empty set',count() FROM (SELECT number FROM system.numbers LIMIT 10) WHERE number IN (SELECT toUInt64(1) WHERE 0) OR number IN (SELECT toUInt64(1) WHERE 1);
|
SELECT 'multiple sets IN empty set OR IN non-empty set', count() FROM (SELECT number FROM system.numbers LIMIT 10) WHERE number IN (SELECT toUInt64(1) WHERE 0) OR number IN (SELECT toUInt64(1) WHERE 1);
|
||||||
SELECT 'multiple sets IN empty set OR NOT IN non-empty set',count() FROM (SELECT number FROM system.numbers LIMIT 10) WHERE number IN (SELECT toUInt64(1) WHERE 0) OR number NOT IN (SELECT toUInt64(1) WHERE 1);
|
SELECT 'multiple sets IN empty set OR NOT IN non-empty set', count() FROM (SELECT number FROM system.numbers LIMIT 10) WHERE number IN (SELECT toUInt64(1) WHERE 0) OR number NOT IN (SELECT toUInt64(1) WHERE 1);
|
||||||
SELECT 'multiple sets NOT IN empty set AND IN non-empty set',count() FROM (SELECT number FROM system.numbers LIMIT 10) WHERE number NOT IN (SELECT toUInt64(1) WHERE 0) AND number IN (SELECT toUInt64(1) WHERE 1);
|
SELECT 'multiple sets NOT IN empty set AND IN non-empty set', count() FROM (SELECT number FROM system.numbers LIMIT 10) WHERE number NOT IN (SELECT toUInt64(1) WHERE 0) AND number IN (SELECT toUInt64(1) WHERE 1);
|
||||||
SELECT 'multiple sets INNER JOIN empty set AND IN empty set',count() FROM system.numbers INNER JOIN (SELECT toUInt64(1) AS x WHERE 0) ON system.numbers.number = x WHERE system.numbers.number IN (SELECT toUInt64(1) WHERE 0);
|
SELECT 'multiple sets INNER JOIN empty set AND IN empty set', count() FROM system.numbers INNER JOIN (SELECT toUInt64(1) AS x WHERE 0) ON system.numbers.number = x WHERE system.numbers.number IN (SELECT toUInt64(1) WHERE 0);
|
||||||
SELECT 'multiple sets INNER JOIN empty set AND IN non-empty set',count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 INNER JOIN (SELECT toUInt64(1) AS x WHERE 0) ON t1.number = x WHERE t1.number IN (SELECT toUInt64(1) WHERE 1);
|
SELECT 'multiple sets INNER JOIN empty set AND IN non-empty set', count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 INNER JOIN (SELECT toUInt64(1) AS x WHERE 0) ON t1.number = x WHERE t1.number IN (SELECT toUInt64(1) WHERE 1);
|
||||||
SELECT 'multiple sets INNER JOIN non-empty set AND IN non-empty set',count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 INNER JOIN (SELECT toUInt64(1) AS x WHERE 1) ON t1.number = x WHERE t1.number IN (SELECT toUInt64(1) WHERE 1);
|
SELECT 'multiple sets INNER JOIN non-empty set AND IN non-empty set', count() FROM (SELECT number FROM system.numbers LIMIT 10) t1 INNER JOIN (SELECT toUInt64(1) AS x WHERE 1) ON t1.number = x WHERE t1.number IN (SELECT toUInt64(1) WHERE 1);
|
||||||
|
|
||||||
SELECT 'IN empty set equals 0', count() FROM numbers(10) WHERE (number IN (SELECT toUInt64(1) WHERE 0)) = 0;
|
SELECT 'IN empty set equals 0', count() FROM numbers(10) WHERE (number IN (SELECT toUInt64(1) WHERE 0)) = 0;
|
||||||
SELECT 'IN empty set sum if', sum(if(number IN (SELECT toUInt64(1) WHERE 0), 2, 1)) FROM numbers(10);
|
SELECT 'IN empty set sum if', sum(if(number IN (SELECT toUInt64(1) WHERE 0), 2, 1)) FROM numbers(10);
|
||||||
|
Loading…
Reference in New Issue
Block a user