mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Merge branch 'master' into alternative-keeper-configs
This commit is contained in:
commit
9ea8dc4e98
@ -353,12 +353,14 @@ if (COMPILER_CLANG)
|
||||
|
||||
set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-U,_inside_main")
|
||||
|
||||
# The LLVM MachO linker (ld64.lld) generates by default unwind info in 'compact' format which the internal unwinder doesn't support
|
||||
# and the server will not come up ('invalid compact unwind encoding'). Disable it.
|
||||
# You will see warning during the build "ld64.lld: warning: Option `-no_compact_unwind' is undocumented. Should lld implement it?".
|
||||
# Yes, ld64.lld does not document the option, likely for compat with Apple's system ld after which ld64.lld is modeled after and
|
||||
# which also does not document it.
|
||||
set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-no_compact_unwind")
|
||||
# The LLVM MachO linker (ld64.lld, used in native builds) generates by default unwind info in 'compact' format which the internal
|
||||
# unwinder doesn't support and the server will not come up ('invalid compact unwind encoding'). Disable it. You will see warning
|
||||
# during the build "ld64.lld: warning: Option `-no_compact_unwind' is undocumented. Should lld implement it?". Yes, ld64.lld does
|
||||
# not document the option, likely for compat with Apple's system ld after which ld64.lld is modeled after and which also does not
|
||||
# document it.
|
||||
if (NOT CMAKE_CROSSCOMPILING)
|
||||
set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-no_compact_unwind")
|
||||
endif ()
|
||||
endif()
|
||||
|
||||
# Display absolute paths in error messages. Otherwise KDevelop fails to navigate to correct file and opens a new file instead.
|
||||
|
@ -31,6 +31,40 @@
|
||||
#define BIG_CONSTANT(x) (x##LLU)
|
||||
|
||||
#endif // !defined(_MSC_VER)
|
||||
//
|
||||
//-----------------------------------------------------------------------------
|
||||
// Block read - on little-endian machines this is a single load,
|
||||
// while on big-endian or unknown machines the byte accesses should
|
||||
// still get optimized into the most efficient instruction.
|
||||
static inline uint32_t getblock ( const uint32_t * p )
|
||||
{
|
||||
#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__)
|
||||
return *p;
|
||||
#else
|
||||
const uint8_t *c = (const uint8_t *)p;
|
||||
return (uint32_t)c[0] |
|
||||
(uint32_t)c[1] << 8 |
|
||||
(uint32_t)c[2] << 16 |
|
||||
(uint32_t)c[3] << 24;
|
||||
#endif
|
||||
}
|
||||
|
||||
static inline uint64_t getblock ( const uint64_t * p )
|
||||
{
|
||||
#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__)
|
||||
return *p;
|
||||
#else
|
||||
const uint8_t *c = (const uint8_t *)p;
|
||||
return (uint64_t)c[0] |
|
||||
(uint64_t)c[1] << 8 |
|
||||
(uint64_t)c[2] << 16 |
|
||||
(uint64_t)c[3] << 24 |
|
||||
(uint64_t)c[4] << 32 |
|
||||
(uint64_t)c[5] << 40 |
|
||||
(uint64_t)c[6] << 48 |
|
||||
(uint64_t)c[7] << 56;
|
||||
#endif
|
||||
}
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
|
||||
@ -52,7 +86,7 @@ uint32_t MurmurHash2 ( const void * key, size_t len, uint32_t seed )
|
||||
|
||||
while(len >= 4)
|
||||
{
|
||||
uint32_t k = *(uint32_t*)data;
|
||||
uint32_t k = getblock((const uint32_t *)data);
|
||||
|
||||
k *= m;
|
||||
k ^= k >> r;
|
||||
@ -105,7 +139,7 @@ uint64_t MurmurHash64A ( const void * key, size_t len, uint64_t seed )
|
||||
|
||||
while(data != end)
|
||||
{
|
||||
uint64_t k = *data++;
|
||||
uint64_t k = getblock(data++);
|
||||
|
||||
k *= m;
|
||||
k ^= k >> r;
|
||||
@ -151,12 +185,12 @@ uint64_t MurmurHash64B ( const void * key, size_t len, uint64_t seed )
|
||||
|
||||
while(len >= 8)
|
||||
{
|
||||
uint32_t k1 = *data++;
|
||||
uint32_t k1 = getblock(data++);
|
||||
k1 *= m; k1 ^= k1 >> r; k1 *= m;
|
||||
h1 *= m; h1 ^= k1;
|
||||
len -= 4;
|
||||
|
||||
uint32_t k2 = *data++;
|
||||
uint32_t k2 = getblock(data++);
|
||||
k2 *= m; k2 ^= k2 >> r; k2 *= m;
|
||||
h2 *= m; h2 ^= k2;
|
||||
len -= 4;
|
||||
@ -164,7 +198,7 @@ uint64_t MurmurHash64B ( const void * key, size_t len, uint64_t seed )
|
||||
|
||||
if(len >= 4)
|
||||
{
|
||||
uint32_t k1 = *data++;
|
||||
uint32_t k1 = getblock(data++);
|
||||
k1 *= m; k1 ^= k1 >> r; k1 *= m;
|
||||
h1 *= m; h1 ^= k1;
|
||||
len -= 4;
|
||||
@ -215,7 +249,7 @@ uint32_t MurmurHash2A ( const void * key, size_t len, uint32_t seed )
|
||||
|
||||
while(len >= 4)
|
||||
{
|
||||
uint32_t k = *(uint32_t*)data;
|
||||
uint32_t k = getblock((const uint32_t *)data);
|
||||
|
||||
mmix(h,k);
|
||||
|
||||
@ -278,7 +312,7 @@ public:
|
||||
|
||||
while(len >= 4)
|
||||
{
|
||||
uint32_t k = *(uint32_t*)data;
|
||||
uint32_t k = getblock((const uint32_t *)data);
|
||||
|
||||
mmix(m_hash,k);
|
||||
|
||||
@ -427,7 +461,7 @@ uint32_t MurmurHashAligned2 ( const void * key, size_t len, uint32_t seed )
|
||||
|
||||
while(len >= 4)
|
||||
{
|
||||
d = *(uint32_t *)data;
|
||||
d = getblock((const uint32_t *)data);
|
||||
t = (t >> sr) | (d << sl);
|
||||
|
||||
uint32_t k = t;
|
||||
@ -492,7 +526,7 @@ uint32_t MurmurHashAligned2 ( const void * key, size_t len, uint32_t seed )
|
||||
{
|
||||
while(len >= 4)
|
||||
{
|
||||
uint32_t k = *(uint32_t *)data;
|
||||
uint32_t k = getblock((const uint32_t *)data);
|
||||
|
||||
MIX(h,k,m);
|
||||
|
||||
|
@ -55,14 +55,32 @@ inline uint64_t rotl64 ( uint64_t x, int8_t r )
|
||||
|
||||
FORCE_INLINE uint32_t getblock32 ( const uint32_t * p, int i )
|
||||
{
|
||||
uint32_t res;
|
||||
memcpy(&res, p + i, sizeof(res));
|
||||
return res;
|
||||
#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__)
|
||||
return p[i];
|
||||
#else
|
||||
const uint8_t *c = (const uint8_t *)&p[i];
|
||||
return (uint32_t)c[0] |
|
||||
(uint32_t)c[1] << 8 |
|
||||
(uint32_t)c[2] << 16 |
|
||||
(uint32_t)c[3] << 24;
|
||||
#endif
|
||||
}
|
||||
|
||||
FORCE_INLINE uint64_t getblock64 ( const uint64_t * p, int i )
|
||||
{
|
||||
#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__)
|
||||
return p[i];
|
||||
#else
|
||||
const uint8_t *c = (const uint8_t *)&p[i];
|
||||
return (uint64_t)c[0] |
|
||||
(uint64_t)c[1] << 8 |
|
||||
(uint64_t)c[2] << 16 |
|
||||
(uint64_t)c[3] << 24 |
|
||||
(uint64_t)c[4] << 32 |
|
||||
(uint64_t)c[5] << 40 |
|
||||
(uint64_t)c[6] << 48 |
|
||||
(uint64_t)c[7] << 56;
|
||||
#endif
|
||||
}
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
@ -329,9 +347,13 @@ void MurmurHash3_x64_128 ( const void * key, const size_t len,
|
||||
|
||||
h1 += h2;
|
||||
h2 += h1;
|
||||
|
||||
#if defined(__BYTE_ORDER__) && (__BYTE_ORDER__ == __ORDER_LITTLE_ENDIAN__)
|
||||
((uint64_t*)out)[0] = h1;
|
||||
((uint64_t*)out)[1] = h2;
|
||||
#else
|
||||
((uint64_t*)out)[0] = h2;
|
||||
((uint64_t*)out)[1] = h1;
|
||||
#endif
|
||||
}
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
|
@ -128,7 +128,7 @@ function run_tests()
|
||||
set +e
|
||||
|
||||
if [[ -n "$USE_PARALLEL_REPLICAS" ]] && [[ "$USE_PARALLEL_REPLICAS" -eq 1 ]]; then
|
||||
clickhouse-test --client="clickhouse-client --use_hedged_requests=0 --allow_experimental_parallel_reading_from_replicas=1 \
|
||||
clickhouse-test --client="clickhouse-client --use_hedged_requests=0 --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 \
|
||||
--max_parallel_replicas=100 --cluster_for_parallel_replicas='parallel_replicas'" \
|
||||
-j 2 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --no-parallel-replicas --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \
|
||||
"$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
|
||||
|
@ -15,7 +15,7 @@ Usage examples:
|
||||
## Usage in ClickHouse Server {#usage-in-clickhouse-server}
|
||||
|
||||
``` sql
|
||||
ENGINE = GenerateRandom([random_seed] [,max_string_length] [,max_array_length])
|
||||
ENGINE = GenerateRandom([random_seed [,max_string_length [,max_array_length]]])
|
||||
```
|
||||
|
||||
The `max_array_length` and `max_string_length` parameters specify maximum length of all
|
||||
|
@ -810,9 +810,12 @@ bool BackupCoordinationRemote::hasConcurrentBackups(const std::atomic<size_t> &)
|
||||
if (existing_backup_uuid == toString(backup_uuid))
|
||||
continue;
|
||||
|
||||
const auto status = zk->get(root_zookeeper_path + "/" + existing_backup_path + "/stage");
|
||||
if (status != Stage::COMPLETED)
|
||||
return true;
|
||||
String status;
|
||||
if (zk->tryGet(root_zookeeper_path + "/" + existing_backup_path + "/stage", status))
|
||||
{
|
||||
if (status != Stage::COMPLETED)
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
zk->createIfNotExists(backup_stage_path, "");
|
||||
|
@ -441,7 +441,7 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt
|
||||
|
||||
if (restore_settings.async)
|
||||
{
|
||||
backups_thread_pool.scheduleOrThrowOnError(
|
||||
restores_thread_pool.scheduleOrThrowOnError(
|
||||
[this, restore_query, restore_id, backup_name_for_logging, backup_info, restore_settings, restore_coordination, context_in_use]
|
||||
{
|
||||
doRestore(
|
||||
|
@ -46,8 +46,8 @@ ThreadPoolImpl<Thread>::ThreadPoolImpl(size_t max_threads_)
|
||||
template <typename Thread>
|
||||
ThreadPoolImpl<Thread>::ThreadPoolImpl(size_t max_threads_, size_t max_free_threads_, size_t queue_size_, bool shutdown_on_exception_)
|
||||
: max_threads(max_threads_)
|
||||
, max_free_threads(max_free_threads_)
|
||||
, queue_size(queue_size_)
|
||||
, max_free_threads(std::min(max_free_threads_, max_threads))
|
||||
, queue_size(queue_size_ ? std::max(queue_size_, max_threads) : 0 /* zero means the queue is unlimited */)
|
||||
, shutdown_on_exception(shutdown_on_exception_)
|
||||
{
|
||||
}
|
||||
@ -56,10 +56,26 @@ template <typename Thread>
|
||||
void ThreadPoolImpl<Thread>::setMaxThreads(size_t value)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
bool need_start_threads = (value > max_threads);
|
||||
bool need_finish_free_threads = (value < max_free_threads);
|
||||
|
||||
max_threads = value;
|
||||
max_free_threads = std::min(max_free_threads, max_threads);
|
||||
|
||||
/// We have to also adjust queue size, because it limits the number of scheduled and already running jobs in total.
|
||||
queue_size = std::max(queue_size, max_threads);
|
||||
queue_size = queue_size ? std::max(queue_size, max_threads) : 0;
|
||||
jobs.reserve(queue_size);
|
||||
|
||||
if (need_start_threads)
|
||||
{
|
||||
/// Start new threads while there are more scheduled jobs in the queue and the limit `max_threads` is not reached.
|
||||
startNewThreadsNoLock();
|
||||
}
|
||||
else if (need_finish_free_threads)
|
||||
{
|
||||
/// Wake up free threads so they can finish themselves.
|
||||
new_job_or_shutdown.notify_all();
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Thread>
|
||||
@ -73,14 +89,22 @@ template <typename Thread>
|
||||
void ThreadPoolImpl<Thread>::setMaxFreeThreads(size_t value)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
max_free_threads = value;
|
||||
bool need_finish_free_threads = (value < max_free_threads);
|
||||
|
||||
max_free_threads = std::min(value, max_threads);
|
||||
|
||||
if (need_finish_free_threads)
|
||||
{
|
||||
/// Wake up free threads so they can finish themselves.
|
||||
new_job_or_shutdown.notify_all();
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Thread>
|
||||
void ThreadPoolImpl<Thread>::setQueueSize(size_t value)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
queue_size = value;
|
||||
queue_size = value ? std::max(value, max_threads) : 0;
|
||||
/// Reserve memory to get rid of allocations
|
||||
jobs.reserve(queue_size);
|
||||
}
|
||||
@ -159,11 +183,42 @@ ReturnType ThreadPoolImpl<Thread>::scheduleImpl(Job job, ssize_t priority, std::
|
||||
++scheduled_jobs;
|
||||
}
|
||||
|
||||
/// Wake up a free thread to run the new job.
|
||||
new_job_or_shutdown.notify_one();
|
||||
|
||||
return static_cast<ReturnType>(true);
|
||||
}
|
||||
|
||||
template <typename Thread>
|
||||
void ThreadPoolImpl<Thread>::startNewThreadsNoLock()
|
||||
{
|
||||
if (shutdown)
|
||||
return;
|
||||
|
||||
/// Start new threads while there are more scheduled jobs in the queue and the limit `max_threads` is not reached.
|
||||
while (threads.size() < std::min(scheduled_jobs, max_threads))
|
||||
{
|
||||
try
|
||||
{
|
||||
threads.emplace_front();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
break; /// failed to start more threads
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
threads.front() = Thread([this, it = threads.begin()] { worker(it); });
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
threads.pop_front();
|
||||
break; /// failed to start more threads
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Thread>
|
||||
void ThreadPoolImpl<Thread>::scheduleOrThrowOnError(Job job, ssize_t priority)
|
||||
{
|
||||
@ -185,20 +240,18 @@ void ThreadPoolImpl<Thread>::scheduleOrThrow(Job job, ssize_t priority, uint64_t
|
||||
template <typename Thread>
|
||||
void ThreadPoolImpl<Thread>::wait()
|
||||
{
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
/// Signal here just in case.
|
||||
/// If threads are waiting on condition variables, but there are some jobs in the queue
|
||||
/// then it will prevent us from deadlock.
|
||||
new_job_or_shutdown.notify_all();
|
||||
job_finished.wait(lock, [this] { return scheduled_jobs == 0; });
|
||||
std::unique_lock lock(mutex);
|
||||
/// Signal here just in case.
|
||||
/// If threads are waiting on condition variables, but there are some jobs in the queue
|
||||
/// then it will prevent us from deadlock.
|
||||
new_job_or_shutdown.notify_all();
|
||||
job_finished.wait(lock, [this] { return scheduled_jobs == 0; });
|
||||
|
||||
if (first_exception)
|
||||
{
|
||||
std::exception_ptr exception;
|
||||
std::swap(exception, first_exception);
|
||||
std::rethrow_exception(exception);
|
||||
}
|
||||
if (first_exception)
|
||||
{
|
||||
std::exception_ptr exception;
|
||||
std::swap(exception, first_exception);
|
||||
std::rethrow_exception(exception);
|
||||
}
|
||||
}
|
||||
|
||||
@ -219,10 +272,14 @@ void ThreadPoolImpl<Thread>::finalize()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
shutdown = true;
|
||||
/// We don't want threads to remove themselves from `threads` anymore, otherwise `thread.join()` will go wrong below in this function.
|
||||
threads_remove_themselves = false;
|
||||
}
|
||||
|
||||
/// Wake up threads so they can finish themselves.
|
||||
new_job_or_shutdown.notify_all();
|
||||
|
||||
/// Wait for all currently running jobs to finish (we don't wait for all scheduled jobs here like the function wait() does).
|
||||
for (auto & thread : threads)
|
||||
thread.join();
|
||||
|
||||
@ -268,38 +325,53 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
|
||||
CurrentMetrics::Increment metric_all_threads(
|
||||
std::is_same_v<Thread, std::thread> ? CurrentMetrics::GlobalThread : CurrentMetrics::LocalThread);
|
||||
|
||||
/// Remove this thread from `threads` and detach it, that must be done before exiting from this worker.
|
||||
/// We can't wrap the following lambda function into `SCOPE_EXIT` because it requires `mutex` to be locked.
|
||||
auto detach_thread = [this, thread_it]
|
||||
{
|
||||
/// `mutex` is supposed to be already locked.
|
||||
if (threads_remove_themselves)
|
||||
{
|
||||
thread_it->detach();
|
||||
threads.erase(thread_it);
|
||||
}
|
||||
};
|
||||
|
||||
/// We'll run jobs in this worker while there are scheduled jobs and until some special event occurs (e.g. shutdown, or decreasing the number of max_threads).
|
||||
/// And if `max_free_threads > 0` we keep this number of threads even when there are no jobs for them currently.
|
||||
while (true)
|
||||
{
|
||||
/// This is inside the loop to also reset previous thread names set inside the jobs.
|
||||
setThreadName("ThreadPool");
|
||||
|
||||
Job job;
|
||||
bool need_shutdown = false;
|
||||
|
||||
/// A copy of parent trace context
|
||||
DB::OpenTelemetry::TracingContextOnThread parent_thead_trace_context;
|
||||
|
||||
/// Get a job from the queue.
|
||||
Job job;
|
||||
std::exception_ptr exception_from_job;
|
||||
bool need_shutdown = false;
|
||||
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
new_job_or_shutdown.wait(lock, [this] { return shutdown || !jobs.empty(); });
|
||||
new_job_or_shutdown.wait(lock, [&] { return !jobs.empty() || shutdown || (threads.size() > std::min(max_threads, scheduled_jobs + max_free_threads)); });
|
||||
need_shutdown = shutdown;
|
||||
|
||||
if (!jobs.empty())
|
||||
if (jobs.empty())
|
||||
{
|
||||
/// boost::priority_queue does not provide interface for getting non-const reference to an element
|
||||
/// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority::job.
|
||||
job = std::move(const_cast<Job &>(jobs.top().job));
|
||||
parent_thead_trace_context = std::move(const_cast<DB::OpenTelemetry::TracingContextOnThread &>(jobs.top().thread_trace_context));
|
||||
jobs.pop();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// shutdown is true, simply finish the thread.
|
||||
/// No jobs and either `shutdown` is set or this thread is excessive. The worker will stop.
|
||||
detach_thread();
|
||||
return;
|
||||
}
|
||||
|
||||
/// boost::priority_queue does not provide interface for getting non-const reference to an element
|
||||
/// to prevent us from modifying its priority. We have to use const_cast to force move semantics on JobWithPriority::job.
|
||||
job = std::move(const_cast<Job &>(jobs.top().job));
|
||||
parent_thead_trace_context = std::move(const_cast<DB::OpenTelemetry::TracingContextOnThread &>(jobs.top().thread_trace_context));
|
||||
jobs.pop();
|
||||
}
|
||||
|
||||
/// Run the job. We don't run jobs after `shutdown` is set.
|
||||
if (!need_shutdown)
|
||||
{
|
||||
ALLOW_ALLOCATIONS_IN_SCOPE;
|
||||
@ -326,46 +398,47 @@ void ThreadPoolImpl<Thread>::worker(typename std::list<Thread>::iterator thread_
|
||||
/// job should be reset before decrementing scheduled_jobs to
|
||||
/// ensure that the Job destroyed before wait() returns.
|
||||
job = {};
|
||||
parent_thead_trace_context.reset();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
thread_trace_context.root_span.addAttribute(std::current_exception());
|
||||
exception_from_job = std::current_exception();
|
||||
thread_trace_context.root_span.addAttribute(exception_from_job);
|
||||
|
||||
/// job should be reset before decrementing scheduled_jobs to
|
||||
/// ensure that the Job destroyed before wait() returns.
|
||||
job = {};
|
||||
parent_thead_trace_context.reset();
|
||||
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (!first_exception)
|
||||
first_exception = std::current_exception(); // NOLINT
|
||||
if (shutdown_on_exception)
|
||||
shutdown = true;
|
||||
--scheduled_jobs;
|
||||
}
|
||||
|
||||
job_finished.notify_all();
|
||||
new_job_or_shutdown.notify_all();
|
||||
return;
|
||||
}
|
||||
|
||||
parent_thead_trace_context.reset();
|
||||
}
|
||||
|
||||
/// The job is done.
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (exception_from_job)
|
||||
{
|
||||
if (!first_exception)
|
||||
first_exception = exception_from_job;
|
||||
if (shutdown_on_exception)
|
||||
shutdown = true;
|
||||
}
|
||||
|
||||
--scheduled_jobs;
|
||||
|
||||
if (threads.size() > scheduled_jobs + max_free_threads)
|
||||
if (threads.size() > std::min(max_threads, scheduled_jobs + max_free_threads))
|
||||
{
|
||||
thread_it->detach();
|
||||
threads.erase(thread_it);
|
||||
/// This thread is excessive. The worker will stop.
|
||||
detach_thread();
|
||||
job_finished.notify_all();
|
||||
if (shutdown)
|
||||
new_job_or_shutdown.notify_all(); /// `shutdown` was set, wake up other threads so they can finish themselves.
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
job_finished.notify_all();
|
||||
job_finished.notify_all();
|
||||
if (shutdown)
|
||||
new_job_or_shutdown.notify_all(); /// `shutdown` was set, wake up other threads so they can finish themselves.
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -102,6 +102,7 @@ private:
|
||||
|
||||
size_t scheduled_jobs = 0;
|
||||
bool shutdown = false;
|
||||
bool threads_remove_themselves = true;
|
||||
const bool shutdown_on_exception = true;
|
||||
|
||||
struct JobWithPriority
|
||||
@ -129,6 +130,9 @@ private:
|
||||
|
||||
void worker(typename std::list<Thread>::iterator thread_it);
|
||||
|
||||
/// Tries to start new threads if there are scheduled jobs and the limit `max_threads` is not reached. Must be called with `mutex` locked.
|
||||
void startNewThreadsNoLock();
|
||||
|
||||
void finalize();
|
||||
void onDestroy();
|
||||
};
|
||||
@ -260,6 +264,11 @@ public:
|
||||
return true;
|
||||
}
|
||||
|
||||
std::thread::id get_id() const
|
||||
{
|
||||
return state ? state->thread_id.load() : std::thread::id{};
|
||||
}
|
||||
|
||||
protected:
|
||||
struct State
|
||||
{
|
||||
|
@ -152,6 +152,7 @@ class IColumn;
|
||||
M(String, cluster_for_parallel_replicas, "default", "Cluster for a shard in which current server is located", 0) \
|
||||
M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \
|
||||
M(Float, parallel_replicas_single_task_marks_count_multiplier, 2, "A multiplier which will be added during calculation for minimal number of marks to retrieve from coordinator. This will be applied only for remote replicas.", 0) \
|
||||
M(Bool, parallel_replicas_for_non_replicated_merge_tree, false, "If true, ClickHouse will use parallel replicas algorithm also for non-replicated MergeTree tables", 0) \
|
||||
\
|
||||
M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \
|
||||
\
|
||||
|
@ -2,8 +2,10 @@
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDate32.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
||||
#include <Functions/DateTimeTransforms.h>
|
||||
#include <Functions/TransformDateTime64.h>
|
||||
@ -60,6 +62,9 @@ public:
|
||||
|
||||
const auto * type_ptr = &type;
|
||||
|
||||
if (const auto * lc_type = checkAndGetDataType<DataTypeLowCardinality>(type_ptr))
|
||||
type_ptr = lc_type->getDictionaryType().get();
|
||||
|
||||
if (const auto * nullable_type = checkAndGetDataType<DataTypeNullable>(type_ptr))
|
||||
type_ptr = nullable_type->getNestedType().get();
|
||||
|
||||
|
@ -396,7 +396,7 @@ void WriteBufferFromS3::completeMultipartUpload()
|
||||
if (outcome.IsSuccess())
|
||||
{
|
||||
LOG_TRACE(log, "Multipart upload has completed. Bucket: {}, Key: {}, Upload_id: {}, Parts: {}", bucket, key, multipart_upload_id, tags.size());
|
||||
break;
|
||||
return;
|
||||
}
|
||||
else if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY)
|
||||
{
|
||||
@ -412,6 +412,11 @@ void WriteBufferFromS3::completeMultipartUpload()
|
||||
outcome.GetError().GetMessage(), key, bucket, fmt::join(tags.begin(), tags.end(), " "));
|
||||
}
|
||||
}
|
||||
|
||||
throw S3Exception(
|
||||
Aws::S3::S3Errors::NO_SUCH_KEY,
|
||||
"Message: Multipart upload failed with NO_SUCH_KEY error, retries {}, Key: {}, Bucket: {}",
|
||||
max_retry, key, bucket);
|
||||
}
|
||||
|
||||
void WriteBufferFromS3::makeSinglepartUpload()
|
||||
@ -508,7 +513,7 @@ void WriteBufferFromS3::processPutRequest(const PutObjectTask & task)
|
||||
if (outcome.IsSuccess())
|
||||
{
|
||||
LOG_TRACE(log, "Single part upload has completed. Bucket: {}, Key: {}, Object size: {}, WithPool: {}", bucket, key, task.req.GetContentLength(), with_pool);
|
||||
break;
|
||||
return;
|
||||
}
|
||||
else if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY)
|
||||
{
|
||||
@ -525,6 +530,11 @@ void WriteBufferFromS3::processPutRequest(const PutObjectTask & task)
|
||||
outcome.GetError().GetMessage(), key, bucket, task.req.GetContentLength(), with_pool);
|
||||
}
|
||||
}
|
||||
|
||||
throw S3Exception(
|
||||
Aws::S3::S3Errors::NO_SUCH_KEY,
|
||||
"Message: Single part upload failed with NO_SUCH_KEY error, retries {}, Key: {}, Bucket: {}",
|
||||
max_retry, key, bucket);
|
||||
}
|
||||
|
||||
void WriteBufferFromS3::waitForReadyBackGroundTasks()
|
||||
|
@ -119,7 +119,6 @@ ASTPtr ASTGrantQuery::clone() const
|
||||
void ASTGrantQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
|
||||
{
|
||||
settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << (attach_mode ? "ATTACH " : "")
|
||||
<< (settings.hilite ? hilite_keyword : "") << ((!is_revoke && (replace_access || replace_granted_roles)) ? "REPLACE " : "") << (settings.hilite ? hilite_none : "")
|
||||
<< (settings.hilite ? hilite_keyword : "") << (is_revoke ? "REVOKE" : "GRANT")
|
||||
<< (settings.hilite ? IAST::hilite_none : "");
|
||||
|
||||
@ -161,6 +160,9 @@ void ASTGrantQuery::formatImpl(const FormatSettings & settings, FormatState &, F
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH GRANT OPTION" << (settings.hilite ? hilite_none : "");
|
||||
else if (admin_option)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH ADMIN OPTION" << (settings.hilite ? hilite_none : "");
|
||||
|
||||
if (replace_access || replace_granted_roles)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH REPLACE OPTION" << (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -76,7 +76,7 @@ ISource::Status RemoteSource::prepare()
|
||||
return status;
|
||||
}
|
||||
|
||||
if (status == Status::PortFull)
|
||||
if (status == Status::PortFull || status == Status::Ready)
|
||||
{
|
||||
/// Also push empty chunk to dependency to signal that we read data from remote source
|
||||
/// or answered to the incoming request from parallel replica
|
||||
|
@ -318,6 +318,9 @@ bool PostgreSQLHandler::isEmptyQuery(const String & query)
|
||||
{
|
||||
if (query.empty())
|
||||
return true;
|
||||
/// golang driver pgx sends ";"
|
||||
if (query == ";")
|
||||
return true;
|
||||
|
||||
Poco::RegularExpression regex(R"(\A\s*\z)");
|
||||
return regex.match(query);
|
||||
|
@ -456,6 +456,14 @@ StorageInMemoryMetadata ReplicatedMergeTreeTableMetadata::Diff::getNewMetadata(c
|
||||
new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST(
|
||||
new_metadata.table_ttl.definition_ast, new_metadata.columns, context, new_metadata.primary_key);
|
||||
|
||||
if (!projections_changed)
|
||||
{
|
||||
ProjectionsDescription recalculated_projections;
|
||||
for (const auto & projection : new_metadata.projections)
|
||||
recalculated_projections.add(ProjectionDescription::getProjectionFromAST(projection.definition_ast, new_metadata.columns, context));
|
||||
new_metadata.projections = std::move(recalculated_projections);
|
||||
}
|
||||
|
||||
return new_metadata;
|
||||
}
|
||||
|
||||
|
@ -214,7 +214,7 @@ void StorageMergeTree::read(
|
||||
size_t max_block_size,
|
||||
size_t num_streams)
|
||||
{
|
||||
if (local_context->canUseParallelReplicasOnInitiator())
|
||||
if (local_context->canUseParallelReplicasOnInitiator() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree)
|
||||
{
|
||||
auto table_id = getStorageID();
|
||||
|
||||
@ -245,10 +245,12 @@ void StorageMergeTree::read(
|
||||
}
|
||||
else
|
||||
{
|
||||
const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() && local_context->getSettingsRef().parallel_replicas_for_non_replicated_merge_tree;
|
||||
|
||||
if (auto plan = reader.read(
|
||||
column_names, storage_snapshot, query_info,
|
||||
local_context, max_block_size, num_streams,
|
||||
processed_stage, nullptr, /*enable_parallel_reading*/local_context->canUseParallelReplicasOnFollower()))
|
||||
processed_stage, nullptr, enable_parallel_reading))
|
||||
query_plan = std::move(*plan);
|
||||
}
|
||||
|
||||
|
@ -1,264 +0,0 @@
|
||||
[
|
||||
"test_atomic_drop_table/test.py::test_atomic_delete_with_stopped_zookeeper",
|
||||
"test_attach_without_fetching/test.py::test_attach_without_fetching",
|
||||
"test_broken_part_during_merge/test.py::test_merge_and_part_corruption",
|
||||
"test_cleanup_dir_after_bad_zk_conn/test.py::test_attach_without_zk",
|
||||
"test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_bad_zk_conn",
|
||||
"test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_replica_name",
|
||||
"test_cleanup_dir_after_bad_zk_conn/test.py::test_cleanup_dir_after_wrong_zk_path",
|
||||
"test_consistent_parts_after_clone_replica/test.py::test_inconsistent_parts_if_drop_while_replica_not_active",
|
||||
"test_cross_replication/test.py::test",
|
||||
"test_ddl_worker_non_leader/test.py::test_non_leader_replica",
|
||||
"test_delayed_replica_failover/test.py::test",
|
||||
"test_dictionaries_update_field/test.py::test_update_field[complex_key_hashed_update_field_dictionary-HASHED]",
|
||||
"test_dictionaries_update_field/test.py::test_update_field[flat_update_field_dictionary-FLAT]",
|
||||
"test_dictionaries_update_field/test.py::test_update_field[simple_key_hashed_update_field_dictionary-HASHED]",
|
||||
"test_dictionary_allow_read_expired_keys/test_default_reading.py::test_default_reading",
|
||||
"test_dictionary_allow_read_expired_keys/test_default_string.py::test_return_real_values",
|
||||
"test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py::test_simple_dict_get_or_default",
|
||||
"test_dictionary_allow_read_expired_keys/test_dict_get.py::test_simple_dict_get",
|
||||
"test_disabled_mysql_server/test.py::test_disabled_mysql_server",
|
||||
"test_distributed_ddl_on_cross_replication/test.py::test_alter_ddl",
|
||||
"test_distributed_ddl_on_cross_replication/test.py::test_atomic_database",
|
||||
"test_distributed_ddl_parallel/test.py::test_all_in_parallel",
|
||||
"test_distributed_ddl_parallel/test.py::test_slow_dict_load_7",
|
||||
"test_distributed_ddl_parallel/test.py::test_smoke",
|
||||
"test_distributed_ddl_parallel/test.py::test_smoke_parallel",
|
||||
"test_distributed_ddl_parallel/test.py::test_smoke_parallel_dict_reload",
|
||||
"test_distributed_ddl_parallel/test.py::test_two_in_parallel_two_queued",
|
||||
"test_distributed_ddl_password/test.py::test_alter",
|
||||
"test_distributed_ddl_password/test.py::test_truncate",
|
||||
"test_distributed_ddl/test.py::test_allowed_databases[configs]",
|
||||
"test_distributed_ddl/test.py::test_allowed_databases[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_create_as_select[configs]",
|
||||
"test_distributed_ddl/test.py::test_create_as_select[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_create_reserved[configs]",
|
||||
"test_distributed_ddl/test.py::test_create_reserved[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_create_view[configs]",
|
||||
"test_distributed_ddl/test.py::test_create_view[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_default_database[configs]",
|
||||
"test_distributed_ddl/test.py::test_default_database[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_detach_query[configs]",
|
||||
"test_distributed_ddl/test.py::test_detach_query[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_implicit_macros[configs]",
|
||||
"test_distributed_ddl/test.py::test_implicit_macros[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_kill_query[configs]",
|
||||
"test_distributed_ddl/test.py::test_kill_query[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_macro[configs]",
|
||||
"test_distributed_ddl/test.py::test_macro[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_on_connection_loss[configs]",
|
||||
"test_distributed_ddl/test.py::test_on_connection_loss[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_on_server_fail[configs]",
|
||||
"test_distributed_ddl/test.py::test_on_server_fail[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_on_session_expired[configs]",
|
||||
"test_distributed_ddl/test.py::test_on_session_expired[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_optimize_query[configs]",
|
||||
"test_distributed_ddl/test.py::test_optimize_query[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_rename[configs]",
|
||||
"test_distributed_ddl/test.py::test_rename[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_replicated_without_arguments[configs]",
|
||||
"test_distributed_ddl/test.py::test_replicated_without_arguments[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_simple_alters[configs]",
|
||||
"test_distributed_ddl/test.py::test_simple_alters[configs_secure]",
|
||||
"test_distributed_ddl/test.py::test_socket_timeout[configs]",
|
||||
"test_distributed_ddl/test.py::test_socket_timeout[configs_secure]",
|
||||
"test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs]",
|
||||
"test_distributed_ddl/test_replicated_alter.py::test_replicated_alters[configs_secure]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-distributed]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node1-remote]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-distributed]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-default-node2-remote]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-distributed]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node1-remote]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-distributed]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs-ready_to_wait-node2-remote]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-distributed]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node1-remote]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-distributed]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-default-node2-remote]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-distributed]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node1-remote]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-distributed]",
|
||||
"test_distributed_respect_user_timeouts/test.py::test_reconnect[configs_secure-ready_to_wait-node2-remote]",
|
||||
"test_drop_replica/test.py::test_drop_replica",
|
||||
"test_hedged_requests_parallel/test.py::test_combination1",
|
||||
"test_hedged_requests_parallel/test.py::test_combination2",
|
||||
"test_hedged_requests_parallel/test.py::test_query_with_no_data_to_sample",
|
||||
"test_hedged_requests_parallel/test.py::test_send_data",
|
||||
"test_hedged_requests_parallel/test.py::test_send_table_status_sleep",
|
||||
"test_hedged_requests/test.py::test_combination1",
|
||||
"test_hedged_requests/test.py::test_combination2",
|
||||
"test_hedged_requests/test.py::test_combination3",
|
||||
"test_hedged_requests/test.py::test_combination4",
|
||||
"test_hedged_requests/test.py::test_long_query",
|
||||
"test_hedged_requests/test.py::test_receive_timeout1",
|
||||
"test_hedged_requests/test.py::test_receive_timeout2",
|
||||
"test_hedged_requests/test.py::test_send_data",
|
||||
"test_hedged_requests/test.py::test_send_data2",
|
||||
"test_hedged_requests/test.py::test_send_table_status_sleep",
|
||||
"test_hedged_requests/test.py::test_send_table_status_sleep2",
|
||||
"test_hedged_requests/test.py::test_stuck_replica",
|
||||
"test_https_replication/test.py::test_both_http",
|
||||
"test_https_replication/test.py::test_both_https",
|
||||
"test_https_replication/test.py::test_mixed_protocol",
|
||||
"test_https_replication/test.py::test_replication_after_partition",
|
||||
"test_insert_into_distributed_sync_async/test.py::test_async_inserts_into_local_shard",
|
||||
"test_insert_into_distributed_sync_async/test.py::test_insertion_sync",
|
||||
"test_insert_into_distributed_sync_async/test.py::test_insertion_sync_fails_with_timeout",
|
||||
"test_insert_into_distributed_sync_async/test.py::test_insertion_sync_with_disabled_timeout",
|
||||
"test_insert_into_distributed_sync_async/test.py::test_insertion_without_sync_ignores_timeout",
|
||||
"test_insert_into_distributed/test.py::test_inserts_batching",
|
||||
"test_insert_into_distributed/test.py::test_inserts_local",
|
||||
"test_insert_into_distributed/test.py::test_inserts_low_cardinality",
|
||||
"test_insert_into_distributed/test.py::test_inserts_single_replica_internal_replication",
|
||||
"test_insert_into_distributed/test.py::test_inserts_single_replica_local_internal_replication",
|
||||
"test_insert_into_distributed/test.py::test_inserts_single_replica_no_internal_replication",
|
||||
"test_insert_into_distributed/test.py::test_prefer_localhost_replica",
|
||||
"test_insert_into_distributed/test.py::test_reconnect",
|
||||
"test_insert_into_distributed/test.py::test_table_function",
|
||||
"test_insert_into_distributed_through_materialized_view/test.py::test_inserts_batching SKIPPED",
|
||||
"test_insert_into_distributed_through_materialized_view/test.py::test_inserts_local",
|
||||
"test_insert_into_distributed_through_materialized_view/test.py::test_reconnect",
|
||||
"test_keeper_multinode_blocade_leader/test.py::test_blocade_leader",
|
||||
"test_keeper_multinode_blocade_leader/test.py::test_blocade_leader_twice",
|
||||
"test_keeper_multinode_simple/test.py::test_follower_restart",
|
||||
"test_keeper_multinode_simple/test.py::test_read_write_multinode",
|
||||
"test_keeper_multinode_simple/test.py::test_session_expiration",
|
||||
"test_keeper_multinode_simple/test.py::test_simple_replicated_table",
|
||||
"test_keeper_multinode_simple/test.py::test_watch_on_follower",
|
||||
"test_limited_replicated_fetches/test.py::test_limited_fetches",
|
||||
"test_materialized_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_multi_table_update[clickhouse_node0]",
|
||||
"test_materialized_mysql_database/test.py::test_multi_table_update[clickhouse_node1]",
|
||||
"test_materialized_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_mysql_settings[clickhouse_node0]",
|
||||
"test_materialized_mysql_database/test.py::test_mysql_settings[clickhouse_node1]",
|
||||
"test_materialized_mysql_database/test.py::test_network_partition_5_7[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_network_partition_5_7[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_network_partition_8_0[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_network_partition_8_0[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_select_without_columns_5_7[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_select_without_columns_5_7[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_select_without_columns_8_0[atomic]",
|
||||
"test_materialized_mysql_database/test.py::test_select_without_columns_8_0[ordinary]",
|
||||
"test_materialized_mysql_database/test.py::test_system_parts_table[clickhouse_node0]",
|
||||
"test_materialized_mysql_database/test.py::test_system_parts_table[clickhouse_node1]",
|
||||
"test_materialized_mysql_database/test.py::test_system_tables_table[clickhouse_node0]",
|
||||
"test_materialized_mysql_database/test.py::test_system_tables_table[clickhouse_node1]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_with_column_comments[clickhouse_node0]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_with_column_comments[clickhouse_node1]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_with_enum[clickhouse_node0]",
|
||||
"test_materialized_mysql_database/test.py::test_materialize_with_enum[clickhouse_node1]",
|
||||
"test_materialized_mysql_database/test.py::test_utf8mb4[clickhouse_node0]",
|
||||
"test_materialized_mysql_database/test.py::test_utf8mb4[clickhouse_node1]",
|
||||
"test_parts_delete_zookeeper/test.py::test_merge_doesnt_work_without_zookeeper",
|
||||
"test_polymorphic_parts/test.py::test_compact_parts_only",
|
||||
"test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_compact-Compact]",
|
||||
"test_polymorphic_parts/test.py::test_different_part_types_on_replicas[polymorphic_table_wide-Wide]",
|
||||
"test_polymorphic_parts/test.py::test_in_memory",
|
||||
"test_polymorphic_parts/test.py::test_in_memory_alters",
|
||||
"test_polymorphic_parts/test.py::test_in_memory_deduplication",
|
||||
"test_polymorphic_parts/test.py::test_in_memory_wal_rotate",
|
||||
"test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node0-second_node0]",
|
||||
"test_polymorphic_parts/test.py::test_polymorphic_parts_basics[first_node1-second_node1]",
|
||||
"test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions_2 SKIPPED",
|
||||
"test_polymorphic_parts/test.py::test_polymorphic_parts_diff_versions SKIPPED",
|
||||
"test_polymorphic_parts/test.py::test_polymorphic_parts_index",
|
||||
"test_polymorphic_parts/test.py::test_polymorphic_parts_non_adaptive",
|
||||
"test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_parallel",
|
||||
"test_quorum_inserts_parallel/test.py::test_parallel_quorum_actually_quorum",
|
||||
"test_random_inserts/test.py::test_insert_multithreaded",
|
||||
"test_random_inserts/test.py::test_random_inserts",
|
||||
"test_reload_clusters_config/test.py::test_add_cluster",
|
||||
"test_reload_clusters_config/test.py::test_delete_cluster",
|
||||
"test_reload_clusters_config/test.py::test_simple_reload",
|
||||
"test_reload_clusters_config/test.py::test_update_one_cluster",
|
||||
"test_replace_partition/test.py::test_drop_failover",
|
||||
"test_replace_partition/test.py::test_normal_work",
|
||||
"test_replace_partition/test.py::test_replace_after_replace_failover",
|
||||
"test_replicated_database/test.py::test_alters_from_different_replicas",
|
||||
"test_replicated_database/test.py::test_create_replicated_table",
|
||||
"test_replicated_database/test.py::test_recover_staled_replica",
|
||||
"test_replicated_database/test.py::test_simple_alter_table[MergeTree]",
|
||||
"test_replicated_database/test.py::test_simple_alter_table[ReplicatedMergeTree]",
|
||||
"test_replicated_database/test.py::test_startup_without_zk",
|
||||
"test_replicated_fetches_timeouts/test.py::test_no_stall",
|
||||
"test_storage_kafka/test.py::test_bad_reschedule",
|
||||
"test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop",
|
||||
"test_storage_kafka/test.py::test_exception_from_destructor",
|
||||
"test_storage_kafka/test.py::test_kafka_commit_on_block_write",
|
||||
"test_storage_kafka/test.py::test_kafka_consumer_hang",
|
||||
"test_storage_kafka/test.py::test_kafka_consumer_hang2",
|
||||
"test_storage_kafka/test.py::test_kafka_csv_with_delimiter",
|
||||
"test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer",
|
||||
"test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed",
|
||||
"test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream",
|
||||
"test_storage_kafka/test.py::test_kafka_engine_put_errors_to_stream_with_random_malformed_json",
|
||||
"test_storage_kafka/test.py::test_kafka_flush_by_block_size",
|
||||
"test_storage_kafka/test.py::test_kafka_flush_by_time",
|
||||
"test_storage_kafka/test.py::test_kafka_flush_on_big_message",
|
||||
"test_storage_kafka/test.py::test_kafka_formats",
|
||||
"test_storage_kafka/test.py::test_kafka_formats_with_broken_message",
|
||||
"test_storage_kafka/test.py::test_kafka_insert",
|
||||
"test_storage_kafka/test.py::test_kafka_issue11308",
|
||||
"test_storage_kafka/test.py::test_kafka_issue14202",
|
||||
"test_storage_kafka/test.py::test_kafka_issue4116",
|
||||
"test_storage_kafka/test.py::test_kafka_json_as_string",
|
||||
"test_storage_kafka/test.py::test_kafka_json_without_delimiter",
|
||||
"test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk",
|
||||
"test_storage_kafka/test.py::test_kafka_many_materialized_views",
|
||||
"test_storage_kafka/test.py::test_kafka_materialized_view",
|
||||
"test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery",
|
||||
"test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed",
|
||||
"test_storage_kafka/test.py::test_kafka_produce_consume",
|
||||
"test_storage_kafka/test.py::test_kafka_produce_key_timestamp",
|
||||
"test_storage_kafka/test.py::test_kafka_protobuf",
|
||||
"test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter",
|
||||
"test_storage_kafka/test.py::test_kafka_rebalance",
|
||||
"test_storage_kafka/test.py::test_kafka_select_empty",
|
||||
"test_storage_kafka/test.py::test_kafka_settings_new_syntax",
|
||||
"test_storage_kafka/test.py::test_kafka_settings_old_syntax",
|
||||
"test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf",
|
||||
"test_storage_kafka/test.py::test_kafka_tsv_with_delimiter",
|
||||
"test_storage_kafka/test.py::test_kafka_unavailable",
|
||||
"test_storage_kafka/test.py::test_kafka_virtual_columns",
|
||||
"test_storage_kafka/test.py::test_kafka_virtual_columns2",
|
||||
"test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view",
|
||||
"test_storage_kafka/test.py::test_librdkafka_compression",
|
||||
"test_storage_kafka/test.py::test_premature_flush_on_eof",
|
||||
"test_storage_kerberized_kafka/test.py::test_kafka_json_as_string",
|
||||
"test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc",
|
||||
"test_system_clusters_actual_information/test.py::test",
|
||||
"test_system_metrics/test.py::test_readonly_metrics",
|
||||
"test_system_replicated_fetches/test.py::test_system_replicated_fetches"
|
||||
]
|
@ -51,5 +51,19 @@
|
||||
"test_global_overcommit_tracker/test.py::test_global_overcommit",
|
||||
|
||||
"test_user_ip_restrictions/test.py::test_ipv4",
|
||||
"test_user_ip_restrictions/test.py::test_ipv6"
|
||||
"test_user_ip_restrictions/test.py::test_ipv6",
|
||||
|
||||
"test_server_reload/test.py::test_change_grpc_port",
|
||||
"test_server_reload/test.py::test_change_http_handlers",
|
||||
"test_server_reload/test.py::test_change_http_port",
|
||||
"test_server_reload/test.py::test_change_listen_host",
|
||||
"test_server_reload/test.py::test_change_mysql_port",
|
||||
"test_server_reload/test.py::test_change_postgresql_port",
|
||||
"test_server_reload/test.py::test_change_tcp_port",
|
||||
"test_server_reload/test.py::test_reload_via_client",
|
||||
"test_server_reload/test.py::test_remove_grpc_port",
|
||||
"test_server_reload/test.py::test_remove_http_port",
|
||||
"test_server_reload/test.py::test_remove_mysql_port",
|
||||
"test_server_reload/test.py::test_remove_postgresql_port",
|
||||
"test_server_reload/test.py::test_remove_tcp_port"
|
||||
]
|
||||
|
@ -9,13 +9,14 @@ from helpers.test_tools import TSV, assert_eq_with_retry
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
num_nodes = 10
|
||||
num_nodes = 4
|
||||
ddl_task_timeout = 640
|
||||
|
||||
|
||||
def generate_cluster_def():
|
||||
path = os.path.join(
|
||||
os.path.dirname(os.path.realpath(__file__)),
|
||||
"./_gen/cluster_for_concurrency_test.xml",
|
||||
"./_gen/cluster_for_disallow_concurrency_test.xml",
|
||||
)
|
||||
os.makedirs(os.path.dirname(path), exist_ok=True)
|
||||
with open(path, "w") as f:
|
||||
@ -85,7 +86,7 @@ def drop_after_test():
|
||||
node0.query(
|
||||
"DROP TABLE IF EXISTS tbl ON CLUSTER 'cluster' NO DELAY",
|
||||
settings={
|
||||
"distributed_ddl_task_timeout": 360,
|
||||
"distributed_ddl_task_timeout": ddl_task_timeout,
|
||||
},
|
||||
)
|
||||
|
||||
@ -100,6 +101,7 @@ def new_backup_name():
|
||||
|
||||
|
||||
def create_and_fill_table():
|
||||
node0.query("SET mutations_sync=2")
|
||||
node0.query(
|
||||
"CREATE TABLE tbl ON CLUSTER 'cluster' ("
|
||||
"x UInt64"
|
||||
@ -107,7 +109,10 @@ def create_and_fill_table():
|
||||
"ORDER BY x"
|
||||
)
|
||||
for i in range(num_nodes):
|
||||
nodes[i].query(f"INSERT INTO tbl SELECT number FROM numbers(40000000)")
|
||||
nodes[i].query(f"INSERT INTO tbl SELECT number FROM numbers(100000000)")
|
||||
nodes[i].query(
|
||||
f"INSERT INTO tbl SELECT number+100000000 FROM numbers(100000000)"
|
||||
)
|
||||
|
||||
|
||||
# All the tests have concurrent backup/restores with same backup names
|
||||
@ -138,6 +143,8 @@ def test_concurrent_backups_on_same_node():
|
||||
nodes[0],
|
||||
f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'",
|
||||
"BACKUP_CREATED",
|
||||
retry_count=100,
|
||||
sleep_time=1,
|
||||
)
|
||||
|
||||
# This restore part is added to confirm creating an internal backup & restore work
|
||||
@ -145,7 +152,7 @@ def test_concurrent_backups_on_same_node():
|
||||
nodes[0].query(
|
||||
f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY",
|
||||
settings={
|
||||
"distributed_ddl_task_timeout": 360,
|
||||
"distributed_ddl_task_timeout": ddl_task_timeout,
|
||||
},
|
||||
)
|
||||
nodes[0].query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}")
|
||||
@ -174,6 +181,8 @@ def test_concurrent_backups_on_different_nodes():
|
||||
nodes[1],
|
||||
f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'",
|
||||
"BACKUP_CREATED",
|
||||
retry_count=100,
|
||||
sleep_time=1,
|
||||
)
|
||||
|
||||
|
||||
@ -197,12 +206,14 @@ def test_concurrent_restores_on_same_node():
|
||||
nodes[0],
|
||||
f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'",
|
||||
"BACKUP_CREATED",
|
||||
retry_count=100,
|
||||
sleep_time=1,
|
||||
)
|
||||
|
||||
nodes[0].query(
|
||||
f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY",
|
||||
settings={
|
||||
"distributed_ddl_task_timeout": 360,
|
||||
"distributed_ddl_task_timeout": ddl_task_timeout,
|
||||
},
|
||||
)
|
||||
restore_id = (
|
||||
@ -226,44 +237,46 @@ def test_concurrent_restores_on_different_node():
|
||||
backup_name = new_backup_name()
|
||||
|
||||
id = (
|
||||
nodes[0]
|
||||
nodes[1]
|
||||
.query(f"BACKUP TABLE tbl ON CLUSTER 'cluster' TO {backup_name} ASYNC")
|
||||
.split("\t")[0]
|
||||
)
|
||||
assert_eq_with_retry(
|
||||
nodes[0],
|
||||
nodes[1],
|
||||
f"SELECT status FROM system.backups WHERE status == 'CREATING_BACKUP' AND id = '{id}'",
|
||||
"CREATING_BACKUP",
|
||||
)
|
||||
|
||||
assert_eq_with_retry(
|
||||
nodes[0],
|
||||
nodes[1],
|
||||
f"SELECT status FROM system.backups WHERE status == 'BACKUP_CREATED' AND id = '{id}'",
|
||||
"BACKUP_CREATED",
|
||||
retry_count=100,
|
||||
sleep_time=1,
|
||||
)
|
||||
|
||||
nodes[0].query(
|
||||
nodes[1].query(
|
||||
f"DROP TABLE tbl ON CLUSTER 'cluster' NO DELAY",
|
||||
settings={
|
||||
"distributed_ddl_task_timeout": 360,
|
||||
"distributed_ddl_task_timeout": ddl_task_timeout,
|
||||
},
|
||||
)
|
||||
restore_id = (
|
||||
nodes[0]
|
||||
nodes[1]
|
||||
.query(f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name} ASYNC")
|
||||
.split("\t")[0]
|
||||
)
|
||||
assert_eq_with_retry(
|
||||
nodes[0],
|
||||
f"SELECT status FROM system.backups WHERE status == 'RESTORING'",
|
||||
nodes[1],
|
||||
f"SELECT status FROM system.backups WHERE status == 'RESTORING' AND id == '{restore_id}'",
|
||||
"RESTORING",
|
||||
)
|
||||
assert "Concurrent restores not supported" in nodes[1].query_and_get_error(
|
||||
assert "Concurrent restores not supported" in nodes[0].query_and_get_error(
|
||||
f"RESTORE TABLE tbl ON CLUSTER 'cluster' FROM {backup_name}"
|
||||
)
|
||||
|
||||
assert_eq_with_retry(
|
||||
nodes[0],
|
||||
nodes[1],
|
||||
f"SELECT status FROM system.backups WHERE status == 'RESTORED' AND id == '{restore_id}'",
|
||||
"RESTORED",
|
||||
)
|
||||
|
@ -25,5 +25,5 @@
|
||||
9631199822919835226
|
||||
4334672815104069193
|
||||
4334672815104069193
|
||||
6145F501578671E2877DBA2BE487AF7E
|
||||
16FE7483905CCE7A85670E43E4678877
|
||||
1
|
||||
1
|
||||
|
@ -32,5 +32,7 @@ SELECT gccMurmurHash('foo');
|
||||
SELECT gccMurmurHash('\x01');
|
||||
SELECT gccMurmurHash(1);
|
||||
|
||||
SELECT hex(murmurHash3_128('foo'));
|
||||
SELECT hex(murmurHash3_128('\x01'));
|
||||
-- Comparison with reverse for big endian
|
||||
SELECT hex(murmurHash3_128('foo')) = hex(reverse(unhex('6145F501578671E2877DBA2BE487AF7E'))) or hex(murmurHash3_128('foo')) = '6145F501578671E2877DBA2BE487AF7E';
|
||||
-- Comparison with reverse for big endian
|
||||
SELECT hex(murmurHash3_128('\x01')) = hex(reverse(unhex('16FE7483905CCE7A85670E43E4678877'))) or hex(murmurHash3_128('\x01')) = '16FE7483905CCE7A85670E43E4678877';
|
||||
|
@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
|
||||
${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 -nm < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null
|
||||
${CLICKHOUSE_CLIENT} --allow_experimental_parallel_reading_from_replicas=1 --parallel_replicas_for_non_replicated_merge_tree=1 -nm < "$CURDIR"/01099_parallel_distributed_insert_select.sql > /dev/null
|
||||
|
@ -15,6 +15,7 @@ as select * from numbers(1);
|
||||
# Logical error: 'Coordinator for parallel reading from replicas is not initialized'.
|
||||
opts=(
|
||||
--allow_experimental_parallel_reading_from_replicas 1
|
||||
--parallel_replicas_for_non_replicated_merge_tree 1
|
||||
--max_parallel_replicas 3
|
||||
|
||||
--iterations 1
|
||||
|
@ -61,6 +61,7 @@ create table pr_t(a UInt64, b UInt64) engine=MergeTree order by a;
|
||||
insert into pr_t select number % 1000, number % 1000 from numbers_mt(1e6);
|
||||
|
||||
set allow_experimental_parallel_reading_from_replicas = 1;
|
||||
set parallel_replicas_for_non_replicated_merge_tree = 1;
|
||||
set max_parallel_replicas = 3;
|
||||
set use_hedged_requests = 0;
|
||||
set cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost';
|
||||
|
@ -0,0 +1 @@
|
||||
GRANT SELECT ON *.* TO A WITH REPLACE OPTION
|
11
tests/queries/0_stateless/02675_grant_query_formatting.sh
Executable file
11
tests/queries/0_stateless/02675_grant_query_formatting.sh
Executable file
@ -0,0 +1,11 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
set -e
|
||||
|
||||
format="$CLICKHOUSE_FORMAT --oneline"
|
||||
|
||||
echo "grant select on *.* to A with replace option" | $format
|
@ -1 +1,2 @@
|
||||
22 0 1
|
||||
1970-01-01 02:00:02
|
||||
|
@ -1,4 +1,5 @@
|
||||
DROP TABLE IF EXISTS 02680_datetime64_monotonic_check;
|
||||
DROP TABLE IF EXISTS 02680_datetime_monotonic_check_lc;
|
||||
|
||||
CREATE TABLE 02680_datetime64_monotonic_check (`t` DateTime64(3), `x` Nullable(Decimal(18, 14)))
|
||||
ENGINE = MergeTree
|
||||
@ -13,3 +14,15 @@ WHERE toHour_Israel = 0
|
||||
GROUP BY toHour_UTC, toHour_Israel;
|
||||
|
||||
DROP TABLE 02680_datetime64_monotonic_check;
|
||||
|
||||
SET allow_suspicious_low_cardinality_types = 1;
|
||||
CREATE TABLE 02680_datetime_monotonic_check_lc (`timestamp` LowCardinality(UInt32))
|
||||
ENGINE = MergeTree
|
||||
ORDER BY timestamp
|
||||
SETTINGS index_granularity = 1;
|
||||
|
||||
INSERT INTO 02680_datetime_monotonic_check_lc VALUES (2);
|
||||
|
||||
SELECT toDateTime(timestamp, 'Asia/Jerusalem') FROM 02680_datetime_monotonic_check_lc WHERE toHour(toDateTime(timestamp, 'Asia/Jerusalem')) = 2;
|
||||
|
||||
DROP TABLE 02680_datetime_monotonic_check_lc
|
||||
|
@ -0,0 +1,11 @@
|
||||
DROP TABLE IF EXISTS 02691_drop_column_replicated;
|
||||
|
||||
CREATE TABLE 02691_drop_column_replicated (col1 Int64, col2 Int64, PROJECTION 02691_drop_column_replicated (SELECT * ORDER BY col1 ))
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/02691_drop_column', 'r1')
|
||||
ORDER BY col1;
|
||||
|
||||
INSERT INTO 02691_drop_column_replicated VALUES (1, 2);
|
||||
|
||||
ALTER TABLE 02691_drop_column_replicated DROP COLUMN col2 SETTINGS alter_sync = 2;
|
||||
|
||||
DROP TABLE 02691_drop_column_replicated;
|
@ -34,7 +34,7 @@ test1() {
|
||||
GROUP BY CounterID, URL, EventDate
|
||||
ORDER BY URL, EventDate
|
||||
LIMIT 5 OFFSET 10
|
||||
SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, use_hedged_requests = 0"
|
||||
SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, use_hedged_requests = 0"
|
||||
check_replicas_read_in_order $query_id
|
||||
}
|
||||
|
||||
@ -51,7 +51,7 @@ test2() {
|
||||
GROUP BY URL, EventDate
|
||||
ORDER BY URL, EventDate
|
||||
LIMIT 5 OFFSET 10
|
||||
SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, use_hedged_requests = 0, query_plan_aggregation_in_order = 1"
|
||||
SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, use_hedged_requests = 0, query_plan_aggregation_in_order = 1"
|
||||
check_replicas_read_in_order $query_id
|
||||
}
|
||||
|
||||
@ -67,7 +67,7 @@ test3() {
|
||||
FROM test.hits
|
||||
WHERE CounterID = 1704509 AND UserID = 4322253409885123546
|
||||
GROUP BY URL, EventDate
|
||||
SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, use_hedged_requests = 0
|
||||
SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, use_hedged_requests = 0
|
||||
)
|
||||
WHERE explain LIKE '%Aggr%Transform%' OR explain LIKE '%InOrder%'"
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user