From e4a653b41cf109c7e4a885b3763a8f68f3080d89 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 15 May 2023 18:49:34 +0000 Subject: [PATCH 01/46] Multiple pools support for AsyncLoader --- src/Common/AsyncLoader.cpp | 218 +++++++++++++++++++++++------------- src/Common/AsyncLoader.h | 219 +++++++++++++++++++------------------ 2 files changed, 259 insertions(+), 178 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 6c19b1910eb..ccb4944b4c1 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -41,9 +42,9 @@ std::exception_ptr LoadJob::exception() const return load_exception; } -ssize_t LoadJob::priority() const +size_t LoadJob::pool() const { - return load_priority; + return pool_id; } void LoadJob::wait() const @@ -148,22 +149,35 @@ void LoadTask::remove() { loader.remove(jobs); jobs.clear(); + goal_jobs.clear(); } } void LoadTask::detach() { jobs.clear(); + goal_jobs.clear(); } -AsyncLoader::AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_, bool log_failures_, bool log_progress_) + +AsyncLoader::AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_) : log_failures(log_failures_) , log_progress(log_progress_) , log(&Poco::Logger::get("AsyncLoader")) - , max_threads(max_threads_) - , pool(metric_threads, metric_active_threads, max_threads) { - + pools.reserve(pool_initializers.size()); + for (auto && init : pool_initializers) + pools.push_back({ + .name = init.name, + .priority = init.priority, + .thread_pool = std::make_unique( + init.metric_threads, + init.metric_active_threads, + init.max_threads, + /* max_free_threads = */ 0, + init.max_threads), + .max_threads = init.max_threads + }); } AsyncLoader::~AsyncLoader() @@ -175,13 +189,20 @@ void AsyncLoader::start() { std::unique_lock lock{mutex}; is_running = true; - for (size_t i = 0; workers < max_threads && i < ready_queue.size(); i++) - spawn(lock); + updateCurrentPriorityAndSpawn(lock); } void AsyncLoader::wait() { - pool.wait(); + // Because job can create new jobs in other pools we have to recheck in cycle + std::unique_lock lock{mutex}; + while (!scheduled_jobs.empty()) + { + lock.unlock(); + for (auto & p : pools) + p.thread_pool->wait(); + lock.lock(); + } } void AsyncLoader::stop() @@ -191,7 +212,7 @@ void AsyncLoader::stop() is_running = false; // NOTE: there is no need to notify because workers never wait } - pool.wait(); + wait(); } void AsyncLoader::schedule(LoadTask & task) @@ -229,9 +250,9 @@ void AsyncLoader::scheduleImpl(const LoadJobSet & input_jobs) old_jobs = finished_jobs.size(); } - // Make set of jobs to schedule: + // Pass 1. Make set of jobs to schedule: // 1) exclude already scheduled or finished jobs - // 2) include pending dependencies, that are not yet scheduled + // 2) include assigned job dependencies (that are not yet scheduled) LoadJobSet jobs; for (const auto & job : input_jobs) gatherNotScheduled(job, jobs, lock); @@ -242,17 +263,17 @@ void AsyncLoader::scheduleImpl(const LoadJobSet & input_jobs) // We do not want any exception to be throws after this point, because the following code is not exception-safe DENY_ALLOCATIONS_IN_SCOPE; - // Schedule all incoming jobs + // Pass 2. Schedule all incoming jobs for (const auto & job : jobs) { NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - scheduled_jobs.emplace(job, Info{.initial_priority = job->load_priority, .priority = job->load_priority}); + scheduled_jobs.try_emplace(job); job->scheduled(); }); } - // Process dependencies on scheduled pending jobs + // Pass 3. Process dependencies on scheduled jobs, priority inheritance for (const auto & job : jobs) { Info & info = scheduled_jobs.find(job)->second; @@ -267,17 +288,18 @@ void AsyncLoader::scheduleImpl(const LoadJobSet & input_jobs) }); info.dependencies_left++; - // Priority inheritance: prioritize deps to have at least given `priority` to avoid priority inversion - prioritize(dep, info.priority, lock); + // Priority inheritance: prioritize deps to have at least given `pool.priority` to avoid priority inversion + prioritize(dep, job->pool_id, lock); } } // Enqueue non-blocked jobs (w/o dependencies) to ready queue - if (!info.is_blocked()) + if (!info.isBlocked()) enqueue(info, job, lock); } - // Process dependencies on other jobs. It is done in a separate pass to facilitate propagation of cancel signals (if any). + // Pass 4: Process dependencies on other jobs. + // It is done in a separate pass to facilitate cancelling due to already failed dependencies. for (const auto & job : jobs) { if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) @@ -285,12 +307,12 @@ void AsyncLoader::scheduleImpl(const LoadJobSet & input_jobs) for (const auto & dep : job->dependencies) { if (scheduled_jobs.contains(dep)) - continue; // Skip dependencies on scheduled pending jobs (already processed) + continue; // Skip dependencies on scheduled jobs (already processed in pass 3) LoadStatus dep_status = dep->status(); if (dep_status == LoadStatus::OK) continue; // Dependency on already successfully finished job -- it's okay. - // Dependency on not scheduled pending job -- it's bad. + // Dependency on assigned job -- it's bad. // Probably, there is an error in `jobs` set, `gatherNotScheduled()` should have fixed it. chassert(dep_status != LoadStatus::PENDING); @@ -305,7 +327,7 @@ void AsyncLoader::scheduleImpl(const LoadJobSet & input_jobs) job->name, getExceptionMessage(dep->exception(), /* with_stacktrace = */ false))); }); - finish(lock, job, LoadStatus::CANCELED, e); + finish(job, LoadStatus::CANCELED, e, lock); break; // This job is now finished, stop its dependencies processing } } @@ -327,13 +349,13 @@ void AsyncLoader::gatherNotScheduled(const LoadJobPtr & job, LoadJobSet & jobs, } } -void AsyncLoader::prioritize(const LoadJobPtr & job, ssize_t new_priority) +void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool) { if (!job) return; DENY_ALLOCATIONS_IN_SCOPE; std::unique_lock lock{mutex}; - prioritize(job, new_priority, lock); + prioritize(job, new_pool, lock); } void AsyncLoader::remove(const LoadJobSet & jobs) @@ -347,14 +369,14 @@ void AsyncLoader::remove(const LoadJobSet & jobs) { if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) { - if (info->second.is_executing()) + if (info->second.isExecuting()) continue; // Skip executing jobs on the first pass std::exception_ptr e; NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; e = std::make_exception_ptr(Exception(ErrorCodes::ASYNC_LOAD_CANCELED, "Load job '{}' canceled", job->name)); }); - finish(lock, job, LoadStatus::CANCELED, e); + finish(job, LoadStatus::CANCELED, e, lock); } } // On the second pass wait for executing jobs to finish @@ -363,7 +385,7 @@ void AsyncLoader::remove(const LoadJobSet & jobs) if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) { // Job is currently executing - chassert(info->second.is_executing()); + chassert(info->second.isExecuting()); lock.unlock(); job->waitNoThrow(); // Wait for job to finish lock.lock(); @@ -379,25 +401,36 @@ void AsyncLoader::remove(const LoadJobSet & jobs) } } -void AsyncLoader::setMaxThreads(size_t value) +void AsyncLoader::setMaxThreads(size_t pool, size_t value) { std::unique_lock lock{mutex}; - pool.setMaxThreads(value); - pool.setMaxFreeThreads(value); - pool.setQueueSize(value); - max_threads = value; + auto & p = pools[pool]; + p.thread_pool->setMaxThreads(value); + p.thread_pool->setQueueSize(value); // Keep queue size equal max threads count to avoid blocking during spawning + p.max_threads = value; if (!is_running) return; - for (size_t i = 0; workers < max_threads && i < ready_queue.size(); i++) - spawn(lock); + for (size_t i = 0; canSpawnWorker(p, lock) && i < p.ready_queue.size(); i++) + spawn(p, lock); } -size_t AsyncLoader::getMaxThreads() const +size_t AsyncLoader::getMaxThreads(size_t pool) const { std::unique_lock lock{mutex}; - return max_threads; + return pools[pool].max_threads; } +const String & AsyncLoader::getPoolName(size_t pool) const +{ + return pools[pool].name; // NOTE: lock is not needed because `name` is const and `pools` are immutable +} + +ssize_t AsyncLoader::getPoolPriority(size_t pool) const +{ + return pools[pool].priority; // NOTE: lock is not needed because `priority` is const and `pools` are immutable +} + + size_t AsyncLoader::getScheduledJobCount() const { std::unique_lock lock{mutex}; @@ -412,11 +445,10 @@ std::vector AsyncLoader::getJobStates() const states.emplace(job->name, JobState{ .job = job, .dependencies_left = info.dependencies_left, - .is_executing = info.is_executing(), - .is_blocked = info.is_blocked(), - .is_ready = info.is_ready(), - .initial_priority = info.initial_priority, - .ready_seqno = last_ready_seqno + .ready_seqno = info.ready_seqno, + .is_blocked = info.isBlocked(), + .is_ready = info.isReady(), + .is_executing = info.isExecuting() }); for (const auto & job : finished_jobs) states.emplace(job->name, JobState{.job = job}); @@ -462,21 +494,21 @@ String AsyncLoader::checkCycleImpl(const LoadJobPtr & job, LoadJobSet & left, Lo return {}; } -void AsyncLoader::finish(std::unique_lock & lock, const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job) +void AsyncLoader::finish(const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job, std::unique_lock & lock) { + chassert(scheduled_jobs.contains(job)); // Job was pending if (status == LoadStatus::OK) { // Notify waiters job->ok(); // Update dependent jobs and enqueue if ready - chassert(scheduled_jobs.contains(job)); // Job was pending for (const auto & dep : scheduled_jobs[job].dependent_jobs) { chassert(scheduled_jobs.contains(dep)); // All depended jobs must be pending Info & dep_info = scheduled_jobs[dep]; dep_info.dependencies_left--; - if (!dep_info.is_blocked()) + if (!dep_info.isBlocked()) enqueue(dep_info, dep, lock); } } @@ -488,11 +520,10 @@ void AsyncLoader::finish(std::unique_lock & lock, const LoadJobPtr & else if (status == LoadStatus::CANCELED) job->canceled(exception_from_job); - chassert(scheduled_jobs.contains(job)); // Job was pending Info & info = scheduled_jobs[job]; - if (info.is_ready()) + if (info.isReady()) { - ready_queue.erase(info.key()); + pools[job->pool_id].ready_queue.erase(info.ready_seqno); info.ready_seqno = 0; } @@ -512,7 +543,7 @@ void AsyncLoader::finish(std::unique_lock & lock, const LoadJobPtr & dep->name, getExceptionMessage(exception_from_job, /* with_stacktrace = */ false))); }); - finish(lock, dep, LoadStatus::CANCELED, e); + finish(dep, LoadStatus::CANCELED, e, lock); } // Clean dependency graph edges pointing to canceled jobs @@ -531,58 +562,98 @@ void AsyncLoader::finish(std::unique_lock & lock, const LoadJobPtr & }); } -void AsyncLoader::prioritize(const LoadJobPtr & job, ssize_t new_priority, std::unique_lock & lock) +void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::unique_lock & lock) { if (auto info = scheduled_jobs.find(job); info != scheduled_jobs.end()) { - if (info->second.priority >= new_priority) - return; // Never lower priority + Pool & old_pool = pools[job->pool_id]; + Pool & new_pool = pools[new_pool_id]; + if (old_pool.priority >= new_pool.priority) + return; // Never lower priority or change pool leaving the same priority // Update priority and push job forward through ready queue if needed - if (info->second.ready_seqno) - ready_queue.erase(info->second.key()); - info->second.priority = new_priority; - job->load_priority.store(new_priority); // Set user-facing priority (may affect executing jobs) - if (info->second.ready_seqno) + UInt64 ready_seqno = info->second.ready_seqno; + if (ready_seqno) + old_pool.ready_queue.erase(ready_seqno); + job->pool_id.store(new_pool_id); // Set user-facing pool and priority (may affect executing jobs) + if (ready_seqno) { NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - ready_queue.emplace(info->second.key(), job); + new_pool.ready_queue.emplace(ready_seqno, job); }); } // Recurse into dependencies for (const auto & dep : job->dependencies) - prioritize(dep, new_priority, lock); + prioritize(dep, new_pool_id, lock); } } void AsyncLoader::enqueue(Info & info, const LoadJobPtr & job, std::unique_lock & lock) { - chassert(!info.is_blocked()); + chassert(!info.isBlocked()); chassert(info.ready_seqno == 0); info.ready_seqno = ++last_ready_seqno; + Pool & pool = pools[job->pool_id]; NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - ready_queue.emplace(info.key(), job); + pool.ready_queue.emplace(info.ready_seqno, job); }); job->enqueued(); - if (is_running && workers < max_threads) - spawn(lock); + if (canSpawnWorker(pool, lock)) + spawn(pool, lock); } -void AsyncLoader::spawn(std::unique_lock &) +bool AsyncLoader::canSpawnWorker(Pool & pool, std::unique_lock &) { - workers++; + return is_running + && !pool.ready_queue.empty() + && pool.workers < pool.max_threads + && (!current_priority || *current_priority <= pool.priority); +} + +bool AsyncLoader::canWorkerLive(Pool & pool, std::unique_lock &) +{ + return is_running + && !pool.ready_queue.empty() + && pool.workers <= pool.max_threads + && (!current_priority || *current_priority <= pool.priority); +} + +void AsyncLoader::updateCurrentPriorityAndSpawn(std::unique_lock & lock) +{ + // Find current priority. + // NOTE: We assume low number of pools, so O(N) scans are fine. + std::optional priority; + for (Pool & pool : pools) + { + if (pool.isActive() && (!priority || *priority < pool.priority)) + priority = pool.priority; + } + current_priority = priority; + + // Spawn workers in all pools with current priority + for (Pool & pool : pools) + { + for (size_t i = 0; canSpawnWorker(pool, lock) && i < pool.ready_queue.size(); i++) + spawn(pool, lock); + } +} + +void AsyncLoader::spawn(Pool & pool, std::unique_lock &) +{ + pool.workers++; + current_priority = pool.priority; // canSpawnWorker() ensures this would not decrease current_priority NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; - pool.scheduleOrThrowOnError([this] { worker(); }); + pool.thread_pool->scheduleOrThrowOnError([this, &pool] { worker(pool); }); }); } -void AsyncLoader::worker() +void AsyncLoader::worker(Pool & pool) { DENY_ALLOCATIONS_IN_SCOPE; @@ -591,27 +662,28 @@ void AsyncLoader::worker() while (true) { // This is inside the loop to also reset previous thread names set inside the jobs - setThreadName("AsyncLoader"); + setThreadName(pool.name.c_str()); { std::unique_lock lock{mutex}; // Handle just executed job if (exception_from_job) - finish(lock, job, LoadStatus::FAILED, exception_from_job); + finish(job, LoadStatus::FAILED, exception_from_job, lock); else if (job) - finish(lock, job, LoadStatus::OK); + finish(job, LoadStatus::OK, {}, lock); - if (!is_running || ready_queue.empty() || workers > max_threads) + if (!canWorkerLive(pool, lock)) { - workers--; + if (--pool.workers == 0) + updateCurrentPriorityAndSpawn(lock); // It will spawn lower priority workers if needed return; } // Take next job to be executed from the ready queue - auto it = ready_queue.begin(); + auto it = pool.ready_queue.begin(); job = it->second; - ready_queue.erase(it); + pool.ready_queue.erase(it); scheduled_jobs.find(job)->second.ready_seqno = 0; // This job is no longer in the ready queue } diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index d4a3218a541..0c52c549a44 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -12,7 +12,7 @@ #include #include #include -#include +#include namespace Poco { class Logger; } @@ -46,22 +46,22 @@ class LoadJob : private boost::noncopyable { public: template - LoadJob(LoadJobSetType && dependencies_, String name_, Func && func_, ssize_t priority_ = 0) + LoadJob(LoadJobSetType && dependencies_, String name_, size_t pool_id_, Func && func_) : dependencies(std::forward(dependencies_)) , name(std::move(name_)) + , pool_id(pool_id_) , func(std::forward(func_)) - , load_priority(priority_) {} // Current job status. LoadStatus status() const; std::exception_ptr exception() const; - // Returns current value of a priority of the job. May differ from initial priority. - ssize_t priority() const; + // Returns current pool of the job. May differ from initial pool. + size_t pool() const; // Sync wait for a pending job to be finished: OK, FAILED or CANCELED status. - // Throws if job is FAILED or CANCELED. Returns or throws immediately on non-pending job. + // Throws if job is FAILED or CANCELED. Returns or throws immediately if called on non-pending job. void wait() const; // Wait for a job to reach any non PENDING status. @@ -92,8 +92,8 @@ private: void enqueued(); void execute(const LoadJobPtr & self); + std::atomic pool_id; std::function func; - std::atomic load_priority; mutable std::mutex mutex; mutable std::condition_variable finished; @@ -115,25 +115,25 @@ struct EmptyJobFunc template LoadJobPtr makeLoadJob(LoadJobSet && dependencies, String name, Func && func = EmptyJobFunc()) { - return std::make_shared(std::move(dependencies), std::move(name), std::forward(func)); + return std::make_shared(std::move(dependencies), std::move(name), 0, std::forward(func)); } template LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, String name, Func && func = EmptyJobFunc()) { - return std::make_shared(dependencies, std::move(name), std::forward(func)); + return std::make_shared(dependencies, std::move(name), 0, std::forward(func)); } template -LoadJobPtr makeLoadJob(LoadJobSet && dependencies, ssize_t priority, String name, Func && func = EmptyJobFunc()) +LoadJobPtr makeLoadJob(LoadJobSet && dependencies, size_t pool_id, String name, Func && func = EmptyJobFunc()) { - return std::make_shared(std::move(dependencies), std::move(name), std::forward(func), priority); + return std::make_shared(std::move(dependencies), std::move(name), pool_id, std::forward(func)); } template -LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, ssize_t priority, String name, Func && func = EmptyJobFunc()) +LoadJobPtr makeLoadJob(const LoadJobSet & dependencies, size_t pool_id, String name, Func && func = EmptyJobFunc()) { - return std::make_shared(dependencies, std::move(name), std::forward(func), priority); + return std::make_shared(dependencies, std::move(name), pool_id, std::forward(func)); } // Represents a logically connected set of LoadJobs required to achieve some goals (final LoadJob in the set). @@ -251,96 +251,112 @@ inline LoadTaskPtrs joinTasks(const LoadTaskPtrs & tasks1, const LoadTaskPtrs & return result; } -// `AsyncLoader` is a scheduler for DAG of `LoadJob`s. It tracks dependencies and priorities of jobs. +// `AsyncLoader` is a scheduler for DAG of `LoadJob`s. It tracks job dependencies and priorities. // Basic usage example: +// // Start async_loader with two thread pools (0=bg, 1=fg): +// AsyncLoader async_loader({ +// {"BgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 1, .priority = 0} +// {"FgPool", CurrentMetrics::AsyncLoaderThreads, CurrentMetrics::AsyncLoaderThreadsActive, .max_threads = 2, .priority = 1} +// }); +// +// // Create and schedule a task consisting of three jobs. Job1 has no dependencies and is run first. +// // Job2 and job3 depend on job1 and are run only after job1 completion. // auto job_func = [&] (const LoadJobPtr & self) { -// LOG_TRACE(log, "Executing load job '{}' with priority '{}'", self->name, self->priority()); +// LOG_TRACE(log, "Executing load job '{}' in pool '{}'", self->name, async_loader->getPoolName(self->pool())); // }; -// auto job1 = makeLoadJob({}, "job1", job_func); -// auto job2 = makeLoadJob({ job1 }, "job2", job_func); -// auto job3 = makeLoadJob({ job1 }, "job3", job_func); +// auto job1 = makeLoadJob({}, "job1", /* pool_id = */ 0, job_func); +// auto job2 = makeLoadJob({ job1 }, "job2", /* pool_id = */ 0, job_func); +// auto job3 = makeLoadJob({ job1 }, "job3", /* pool_id = */ 0, job_func); // auto task = makeLoadTask(async_loader, { job1, job2, job3 }); // task.schedule(); -// Here we have created and scheduled a task consisting of three jobs. Job1 has no dependencies and is run first. -// Job2 and job3 depend on job1 and are run only after job1 completion. Another thread may prioritize a job and wait for it: -// async_loader->prioritize(job3, /* priority = */ 1); // higher priority jobs are run first, default priority is zero. +// +// // Another thread may prioritize a job by changing its pool and wait for it: +// async_loader->prioritize(job3, /* pool_id = */ 1); // higher priority jobs are run first, default priority is zero. // job3->wait(); // blocks until job completion or cancellation and rethrow an exception (if any) // -// AsyncLoader tracks state of all scheduled jobs. Job lifecycle is the following: -// 1) Job is constructed with PENDING status and initial priority. The job is placed into a task. -// 2) The task is scheduled with all its jobs and their dependencies. A scheduled job may be ready (i.e. have all its dependencies finished) or blocked. -// 3a) When all dependencies are successfully executed, the job became ready. A ready job is enqueued into the ready queue. +// Every job has a pool associated with it. AsyncLoader starts every job in its thread pool. +// Each pool has a constant priority and a mutable maximum number of threads. +// Higher priority (greater `pool.priority` value) jobs are run first. +// No job with lower priority is started while there is at least one higher priority job ready or running. +// +// Job priority can be elevated (but cannot be lowered) +// (a) if either it has a dependent job with higher priority: +// in this case the priority and the pool of a dependent job is inherited during `schedule()` call; +// (b) or job was explicitly prioritized by `prioritize(job, higher_priority_pool)` call: +// this also leads to a priority inheritance for all the dependencies. +// Value stored in load job `pool_id` field is atomic and can be changed even during job execution. +// Job is, of course, not moved from its initial thread pool, but it should use `self->pool()` for +// all new jobs it create to avoid priority inversion. +// +// === IMPLEMENTATION DETAILS === +// All possible states and statuses of a job: +// .---------- scheduled ----------. +// ctor --> assigned --> blocked --> ready --> executing --> finished ------> removed --> dtor +// STATUS: '------------------ PENDING -----------------' '-- OK|FAILED|CANCELED --' +// +// AsyncLoader tracks state of all scheduled and finished jobs. Job lifecycle is the following: +// 1) A job is constructed with PENDING status and assigned to a pool. The job is placed into a task. +// 2) The task is scheduled with all its jobs and their dependencies. A scheduled job may be ready, blocked (and later executing). +// 3a) When all dependencies are successfully finished, the job became ready. A ready job is enqueued into the ready queue of its pool. // 3b) If at least one of the job dependencies is failed or canceled, then this job is canceled (with all it's dependent jobs as well). // On cancellation an ASYNC_LOAD_CANCELED exception is generated and saved inside LoadJob object. The job status is changed to CANCELED. // Exception is rethrown by any existing or new `wait()` call. The job is moved to the set of the finished jobs. -// 4) The scheduled pending ready job starts execution by a worker. The job is dequeued. Callback `job_func` is called. -// Status of an executing job is PENDING. And it is still considered as a scheduled job by AsyncLoader. -// Note that `job_func` of a CANCELED job is never executed. +// 4) The ready job starts execution by a worker. The job is dequeued. Callback `job_func` is called. +// Status of an executing job is PENDING. Note that `job_func` of a CANCELED job is never executed. // 5a) On successful execution the job status is changed to OK and all existing and new `wait()` calls finish w/o exceptions. // 5b) Any exception thrown out of `job_func` is wrapped into an ASYNC_LOAD_FAILED exception and saved inside LoadJob. // The job status is changed to FAILED. All the dependent jobs are canceled. The exception is rethrown from all existing and new `wait()` calls. // 6) The job is no longer considered as scheduled and is instead moved to the finished jobs set. This is just for introspection of the finished jobs. // 7) The task containing this job is destructed or `remove()` is explicitly called. The job is removed from the finished job set. // 8) The job is destructed. -// -// Every job has a priority associated with it. AsyncLoader runs higher priority (greater `priority` value) jobs first. Job priority can be elevated -// (a) if either it has a dependent job with higher priority (in this case priority of a dependent job is inherited); -// (b) or job was explicitly prioritized by `prioritize(job, higher_priority)` call (this also leads to a priority inheritance for all the dependencies). -// Note that to avoid priority inversion `job_func` should use `self->priority()` to schedule new jobs in AsyncLoader or any other pool. -// Value stored in load job priority field is atomic and can be increased even during job execution. -// -// When a task is scheduled it can contain dependencies on previously scheduled jobs. These jobs can have any status. If job A being scheduled depends on -// another job B that is not yet scheduled, then job B will also be scheduled (even if the task does not contain it). class AsyncLoader : private boost::noncopyable { private: - // Key of a pending job in the ready queue. - struct ReadyKey - { - ssize_t priority; // Ascending order - ssize_t initial_priority; // Ascending order - UInt64 ready_seqno; // Descending order + // Thread pool for job execution. + // Pools control the following aspects of job execution: + // 1) Concurrency: Amount of concurrently executing jobs in a pool is `max_threads`. + // 2) Priority: As long as there is executing worker with higher priority, workers with lower priorities are not started + // (although, they can finish last job started before higher priority jobs appeared) + struct Pool { + const String name; + const ssize_t priority; + std::unique_ptr thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools. + std::map ready_queue; // FIFO queue of jobs to be executed in this pool. Map is used for faster erasing. Key is `ready_seqno` + size_t max_threads; // Max number of workers to be spawn + size_t workers = 0; // Number of currently execution workers - bool operator<(const ReadyKey & rhs) const - { - if (priority > rhs.priority) - return true; - if (priority < rhs.priority) - return false; - if (initial_priority > rhs.initial_priority) - return true; - if (initial_priority < rhs.initial_priority) - return false; - return ready_seqno < rhs.ready_seqno; - } + bool isActive() const { return workers > 0 || !ready_queue.empty(); } }; // Scheduling information for a pending job. struct Info { - ssize_t initial_priority = 0; // Initial priority passed into schedule(). - ssize_t priority = 0; // Elevated priority, due to priority inheritance or prioritize(). size_t dependencies_left = 0; // Current number of dependencies on pending jobs. UInt64 ready_seqno = 0; // Zero means that job is not in ready queue. LoadJobSet dependent_jobs; // Set of jobs dependent on this job. - // Three independent states of a non-finished job. - bool is_blocked() const { return dependencies_left > 0; } - bool is_ready() const { return dependencies_left == 0 && ready_seqno > 0; } - bool is_executing() const { return dependencies_left == 0 && ready_seqno == 0; } - - // Get key of a ready job - ReadyKey key() const - { - return {.priority = priority, .initial_priority = initial_priority, .ready_seqno = ready_seqno}; - } + // Three independent states of a scheduled job. + bool isBlocked() const { return dependencies_left > 0; } + bool isReady() const { return dependencies_left == 0 && ready_seqno > 0; } + bool isExecuting() const { return dependencies_left == 0 && ready_seqno == 0; } }; public: using Metric = CurrentMetrics::Metric; - AsyncLoader(Metric metric_threads, Metric metric_active_threads, size_t max_threads_, bool log_failures_, bool log_progress_); + // Helper struct for AsyncLoader construction + struct PoolInitializer + { + String name; + Metric metric_threads; + Metric metric_active_threads; + size_t max_threads; + ssize_t priority; + }; + AsyncLoader(std::vector pool_initializers, bool log_failures_, bool log_progress_); + + // Stops AsyncLoader before destruction // WARNING: all tasks instances should be destructed before associated AsyncLoader. ~AsyncLoader(); @@ -356,10 +372,10 @@ public: // - or canceled using ~Task() or remove() later. void stop(); - // Schedule all jobs of given `task` and their dependencies (if any, not scheduled yet). - // Higher priority jobs (with greater `job->priority()` value) are executed earlier. - // All dependencies of a scheduled job inherit its priority if it is higher. This way higher priority job - // never wait for (blocked by) lower priority jobs. No priority inversion is possible. + // Schedule all jobs of given `task` and their dependencies (even if they are not in task). + // All dependencies of a scheduled job inherit its pool if it has higher priority. This way higher priority job + // never waits for (blocked by) lower priority jobs. No priority inversion is possible. + // Idempotent: multiple schedule() calls for the same job are no-op. // Note that `task` destructor ensures that all its jobs are finished (OK, FAILED or CANCELED) // and are removed from AsyncLoader, so it is thread-safe to destroy them. void schedule(LoadTask & task); @@ -369,15 +385,19 @@ public: void schedule(const std::vector & tasks); // Increase priority of a job and all its dependencies recursively. - void prioritize(const LoadJobPtr & job, ssize_t new_priority); + // Jobs from higher (than `new_pool`) priority pools are not changed. + void prioritize(const LoadJobPtr & job, size_t new_pool); // Remove finished jobs, cancel scheduled jobs, wait for executing jobs to finish and remove them. void remove(const LoadJobSet & jobs); - // Increase or decrease maximum number of simultaneously executing jobs. - void setMaxThreads(size_t value); + // Increase or decrease maximum number of simultaneously executing jobs in `pool`. + void setMaxThreads(size_t pool, size_t value); + + size_t getMaxThreads(size_t pool) const; + const String & getPoolName(size_t pool) const; + ssize_t getPoolPriority(size_t pool) const; - size_t getMaxThreads() const; size_t getScheduledJobCount() const; // Helper class for introspection @@ -385,11 +405,10 @@ public: { LoadJobPtr job; size_t dependencies_left = 0; - bool is_executing = false; + UInt64 ready_seqno = 0; bool is_blocked = false; bool is_ready = false; - std::optional initial_priority; - std::optional ready_seqno; + bool is_executing = false; }; // For introspection and debug only, see `system.async_loader` table @@ -398,42 +417,32 @@ public: private: void checkCycle(const LoadJobSet & jobs, std::unique_lock & lock); String checkCycleImpl(const LoadJobPtr & job, LoadJobSet & left, LoadJobSet & visited, std::unique_lock & lock); - void finish(std::unique_lock & lock, const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job = {}); + void finish(const LoadJobPtr & job, LoadStatus status, std::exception_ptr exception_from_job, std::unique_lock & lock); void scheduleImpl(const LoadJobSet & input_jobs); void gatherNotScheduled(const LoadJobPtr & job, LoadJobSet & jobs, std::unique_lock & lock); - void prioritize(const LoadJobPtr & job, ssize_t new_priority, std::unique_lock & lock); + void prioritize(const LoadJobPtr & job, size_t new_pool_id, std::unique_lock & lock); void enqueue(Info & info, const LoadJobPtr & job, std::unique_lock & lock); - void spawn(std::unique_lock &); - void worker(); + bool canSpawnWorker(Pool & pool, std::unique_lock &); + bool canWorkerLive(Pool & pool, std::unique_lock &); + void updateCurrentPriorityAndSpawn(std::unique_lock &); + void spawn(Pool & pool, std::unique_lock &); + void worker(Pool & pool); // Logging const bool log_failures; // Worker should log all exceptions caught from job functions. const bool log_progress; // Periodically log total progress Poco::Logger * log; - std::chrono::system_clock::time_point busy_period_start_time; - AtomicStopwatch stopwatch; - size_t old_jobs = 0; // Number of jobs that were finished in previous busy period (for correct progress indication) mutable std::mutex mutex; // Guards all the fields below. - bool is_running = false; - - // Full set of scheduled pending jobs along with scheduling info. - std::unordered_map scheduled_jobs; - - // Subset of scheduled pending non-blocked jobs (waiting for a worker to be executed). - // Represent a queue of jobs in order of decreasing priority and FIFO for jobs with equal priorities. - std::map ready_queue; - - // Set of finished jobs (for introspection only, until jobs are removed). - LoadJobSet finished_jobs; - - // Increasing counter for `ReadyKey` assignment (to preserve FIFO order of the jobs with equal priorities). - UInt64 last_ready_seqno = 0; - - // For executing jobs. Note that we avoid using an internal queue of the pool to be able to prioritize jobs. - size_t max_threads; - size_t workers = 0; - ThreadPool pool; + bool is_running = true; + std::optional current_priority; // highest priority among active pools + UInt64 last_ready_seqno = 0; // Increasing counter for ready queue keys. + std::unordered_map scheduled_jobs; // Full set of scheduled pending jobs along with scheduling info. + std::vector pools; // Thread pools for job execution and ready queues + LoadJobSet finished_jobs; // Set of finished jobs (for introspection only, until jobs are removed). + AtomicStopwatch stopwatch; // For progress indication + size_t old_jobs = 0; // Number of jobs that were finished in previous busy period (for correct progress indication) + std::chrono::system_clock::time_point busy_period_start_time; }; } From f6d7145a723510572de153165f7a008176073ac6 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 15 May 2023 23:05:13 +0000 Subject: [PATCH 02/46] fix tests --- src/Common/tests/gtest_async_loader.cpp | 72 +++++++++++++++++++++---- 1 file changed, 62 insertions(+), 10 deletions(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 5666c4b923e..763f7d7972c 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -30,6 +30,11 @@ namespace DB::ErrorCodes extern const int ASYNC_LOAD_CANCELED; } +struct Initializer { + size_t max_threads = 1; + ssize_t priority = 0; +}; + struct AsyncLoaderTest { AsyncLoader loader; @@ -37,10 +42,32 @@ struct AsyncLoaderTest std::mutex rng_mutex; pcg64 rng{randomSeed()}; - explicit AsyncLoaderTest(size_t max_threads = 1) - : loader(CurrentMetrics::TablesLoaderThreads, CurrentMetrics::TablesLoaderThreadsActive, max_threads, /* log_failures = */ false, /* log_progress = */ false) + explicit AsyncLoaderTest(std::vector initializers) + : loader(getPoolInitializers(initializers), /* log_failures = */ false, /* log_progress = */ false) {} + explicit AsyncLoaderTest(size_t max_threads = 1) + : AsyncLoaderTest({{.max_threads = max_threads}}) + {} + + std::vector getPoolInitializers(std::vector initializers) + { + std::vector result; + size_t pool_id = 0; + for (auto & desc : initializers) + { + result.push_back({ + .name = fmt::format("Pool{}", pool_id), + .metric_threads = CurrentMetrics::TablesLoaderThreads, + .metric_active_threads = CurrentMetrics::TablesLoaderThreadsActive, + .max_threads = desc.max_threads, + .priority = desc.priority + }); + pool_id++; + } + return result; + } + template T randomInt(T from, T to) { @@ -114,16 +141,19 @@ struct AsyncLoaderTest TEST(AsyncLoader, Smoke) { - AsyncLoaderTest t(2); + AsyncLoaderTest t({ + {.max_threads = 2, .priority = 0}, + {.max_threads = 2, .priority = -1}, + }); - static constexpr ssize_t low_priority = -1; + static constexpr ssize_t low_priority_pool = 1; std::atomic jobs_done{0}; std::atomic low_priority_jobs_done{0}; auto job_func = [&] (const LoadJobPtr & self) { jobs_done++; - if (self->priority() == low_priority) + if (self->pool() == low_priority_pool) low_priority_jobs_done++; }; @@ -135,7 +165,7 @@ TEST(AsyncLoader, Smoke) auto job3 = makeLoadJob({ job2 }, "job3", job_func); auto job4 = makeLoadJob({ job2 }, "job4", job_func); auto task2 = t.schedule({ job3, job4 }); - auto job5 = makeLoadJob({ job3, job4 }, low_priority, "job5", job_func); + auto job5 = makeLoadJob({ job3, job4 }, low_priority_pool, "job5", job_func); task2->merge(t.schedule({ job5 })); std::thread waiter_thread([=] { job5->wait(); }); @@ -562,13 +592,24 @@ TEST(AsyncLoader, TestOverload) TEST(AsyncLoader, StaticPriorities) { - AsyncLoaderTest t(1); + AsyncLoaderTest t({ + {.max_threads = 1, .priority = 0}, + {.max_threads = 1, .priority = 1}, + {.max_threads = 1, .priority = 2}, + {.max_threads = 1, .priority = 3}, + {.max_threads = 1, .priority = 4}, + {.max_threads = 1, .priority = 5}, + {.max_threads = 1, .priority = 6}, + {.max_threads = 1, .priority = 7}, + {.max_threads = 1, .priority = 8}, + {.max_threads = 1, .priority = 9}, + }); std::string schedule; auto job_func = [&] (const LoadJobPtr & self) { - schedule += fmt::format("{}{}", self->name, self->priority()); + schedule += fmt::format("{}{}", self->name, self->pool()); }; std::vector jobs; @@ -590,7 +631,18 @@ TEST(AsyncLoader, StaticPriorities) TEST(AsyncLoader, DynamicPriorities) { - AsyncLoaderTest t(1); + AsyncLoaderTest t({ + {.max_threads = 1, .priority = 0}, + {.max_threads = 1, .priority = 1}, + {.max_threads = 1, .priority = 2}, + {.max_threads = 1, .priority = 3}, + {.max_threads = 1, .priority = 4}, + {.max_threads = 1, .priority = 5}, + {.max_threads = 1, .priority = 6}, + {.max_threads = 1, .priority = 7}, + {.max_threads = 1, .priority = 8}, + {.max_threads = 1, .priority = 9}, + }); for (bool prioritize : {false, true}) { @@ -602,7 +654,7 @@ TEST(AsyncLoader, DynamicPriorities) { if (prioritize && self->name == "C") t.loader.prioritize(job_to_prioritize, 9); // dynamic prioritization - schedule += fmt::format("{}{}", self->name, self->priority()); + schedule += fmt::format("{}{}", self->name, self->pool()); }; // Job DAG with initial priorities. During execution of C4, job G0 priority is increased to G9, postponing B3 job executing. From e1509ca59d879920d51ef7a08970a5d5f2e2202b Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 16 May 2023 08:23:06 +0000 Subject: [PATCH 03/46] fix style --- src/Common/AsyncLoader.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 0c52c549a44..d52cd03ef32 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -317,7 +317,8 @@ private: // 1) Concurrency: Amount of concurrently executing jobs in a pool is `max_threads`. // 2) Priority: As long as there is executing worker with higher priority, workers with lower priorities are not started // (although, they can finish last job started before higher priority jobs appeared) - struct Pool { + struct Pool + { const String name; const ssize_t priority; std::unique_ptr thread_pool; // NOTE: we avoid using a `ThreadPool` queue to be able to move jobs between pools. From 128b8e5889691b3a55129dbb3adbec22fa509a4e Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 21 May 2023 14:28:16 +0000 Subject: [PATCH 04/46] fix tests + add test for dynamic pools --- src/Common/AsyncLoader.cpp | 11 ++++- src/Common/AsyncLoader.h | 21 ++++++--- src/Common/tests/gtest_async_loader.cpp | 60 ++++++++++++++++++++++++- 3 files changed, 81 insertions(+), 11 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index ccb4944b4c1..a359c5b4196 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -42,6 +42,11 @@ std::exception_ptr LoadJob::exception() const return load_exception; } +size_t LoadJob::execution_pool() const +{ + return execution_pool_id; +} + size_t LoadJob::pool() const { return pool_id; @@ -113,8 +118,9 @@ void LoadJob::enqueued() enqueue_time = std::chrono::system_clock::now(); } -void LoadJob::execute(const LoadJobPtr & self) +void LoadJob::execute(size_t pool, const LoadJobPtr & self) { + execution_pool_id = pool; start_time = std::chrono::system_clock::now(); func(self); } @@ -657,6 +663,7 @@ void AsyncLoader::worker(Pool & pool) { DENY_ALLOCATIONS_IN_SCOPE; + size_t pool_id = &pool - &*pools.begin(); LoadJobPtr job; std::exception_ptr exception_from_job; while (true) @@ -691,7 +698,7 @@ void AsyncLoader::worker(Pool & pool) try { - job->execute(job); + job->execute(pool_id, job); exception_from_job = {}; } catch (...) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index d52cd03ef32..7c6c934d03e 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -57,7 +57,13 @@ public: LoadStatus status() const; std::exception_ptr exception() const; - // Returns current pool of the job. May differ from initial pool. + // Returns pool in which the job is executing (was executed). May differ from initial pool and from current pool. + // Value is only valid (and constant) after execution started. + size_t execution_pool() const; + + // Returns current pool of the job. May differ from initial and execution pool. + // This value is intended for creating new jobs during this job execution. + // Value may change during job execution by `prioritize()`. size_t pool() const; // Sync wait for a pending job to be finished: OK, FAILED or CANCELED status. @@ -90,8 +96,9 @@ private: void scheduled(); void enqueued(); - void execute(const LoadJobPtr & self); + void execute(size_t pool, const LoadJobPtr & self); + std::atomic execution_pool_id; std::atomic pool_id; std::function func; @@ -185,7 +192,7 @@ inline void scheduleLoad(const LoadTaskPtrs & tasks) } template -inline void scheduleLoad(Args && ... args) +inline void scheduleLoadAll(Args && ... args) { (scheduleLoad(std::forward(args)), ...); } @@ -208,16 +215,16 @@ inline void waitLoad(const LoadTaskPtrs & tasks) } template -inline void waitLoad(Args && ... args) +inline void waitLoadAll(Args && ... args) { (waitLoad(std::forward(args)), ...); } template -inline void scheduleAndWaitLoad(Args && ... args) +inline void scheduleAndWaitLoadAll(Args && ... args) { - scheduleLoad(std::forward(args)...); - waitLoad(std::forward(args)...); + scheduleLoadAll(std::forward(args)...); + waitLoadAll(std::forward(args)...); } inline LoadJobSet getGoals(const LoadTaskPtrs & tasks) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 763f7d7972c..eabddfbebeb 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -566,7 +566,7 @@ TEST(AsyncLoader, TestOverload) AsyncLoaderTest t(3); t.loader.start(); - size_t max_threads = t.loader.getMaxThreads(); + size_t max_threads = t.loader.getMaxThreads(/* pool = */ 0); std::atomic executing{0}; for (int concurrency = 4; concurrency <= 8; concurrency++) @@ -794,8 +794,64 @@ TEST(AsyncLoader, SetMaxThreads) syncs[idx]->arrive_and_wait(); // (A) sync_index++; if (sync_index < syncs.size()) - t.loader.setMaxThreads(max_threads_values[sync_index]); + t.loader.setMaxThreads(/* pool = */ 0, max_threads_values[sync_index]); syncs[idx]->arrive_and_wait(); // (B) this sync point is required to allow `executing` value to go back down to zero after we change number of workers } t.loader.wait(); } + +TEST(AsyncLoader, DynamicPools) +{ + const size_t max_threads[] { 2, 10 }; + const int jobs_in_chain = 16; + AsyncLoaderTest t({ + {.max_threads = max_threads[0], .priority = 0}, + {.max_threads = max_threads[1], .priority = 1}, + }); + + t.loader.start(); + + std::atomic executing[2] { 0, 0 }; // Number of currently executing jobs per pool + + for (int concurrency = 1; concurrency <= 12; concurrency++) + { + std::atomic boosted{false}; // Visible concurrency was increased + std::atomic left{concurrency * jobs_in_chain / 2}; // Number of jobs to start before `prioritize()` call + + LoadJobSet jobs_to_prioritize; + + auto job_func = [&] (const LoadJobPtr & self) + { + auto pool_id = self->execution_pool(); + executing[pool_id]++; + if (executing[pool_id] > max_threads[0]) + boosted = true; + ASSERT_LE(executing[pool_id], max_threads[pool_id]); + + // Dynamic prioritization + if (--left == 0) + { + for (const auto & job : jobs_to_prioritize) + t.loader.prioritize(job, 1); + } + + t.randomSleepUs(100, 200, 100); + + ASSERT_LE(executing[pool_id], max_threads[pool_id]); + executing[pool_id]--; + }; + + std::vector tasks; + tasks.reserve(concurrency); + for (int i = 0; i < concurrency; i++) + tasks.push_back(makeLoadTask(t.loader, t.chainJobSet(jobs_in_chain, job_func))); + jobs_to_prioritize = getGoals(tasks); // All jobs + scheduleAndWaitLoadAll(tasks); + + ASSERT_EQ(executing[0], 0); + ASSERT_EQ(executing[1], 0); + ASSERT_EQ(boosted, concurrency > 2); + boosted = false; + } + +} From c5765e71f921bb156543973833220a4ae7395cdb Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 21 May 2023 17:02:56 +0000 Subject: [PATCH 05/46] requeue jobs w/o allocations and spawn workers during prioritization --- src/Common/AsyncLoader.cpp | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index a359c5b4196..0f75f17465b 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -579,17 +579,18 @@ void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool_id, std::un // Update priority and push job forward through ready queue if needed UInt64 ready_seqno = info->second.ready_seqno; - if (ready_seqno) - old_pool.ready_queue.erase(ready_seqno); - job->pool_id.store(new_pool_id); // Set user-facing pool and priority (may affect executing jobs) + + // Requeue job into the new pool queue without allocations if (ready_seqno) { - NOEXCEPT_SCOPE({ - ALLOW_ALLOCATIONS_IN_SCOPE; - new_pool.ready_queue.emplace(ready_seqno, job); - }); + new_pool.ready_queue.insert(old_pool.ready_queue.extract(ready_seqno)); + if (canSpawnWorker(new_pool, lock)) + spawn(new_pool, lock); } + // Set user-facing pool and priority (may affect executing jobs) + job->pool_id.store(new_pool_id); + // Recurse into dependencies for (const auto & dep : job->dependencies) prioritize(dep, new_pool_id, lock); From cd11c258646d509c12242c38c0c246cf0650b57a Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 21 May 2023 17:04:00 +0000 Subject: [PATCH 06/46] fix test + more testing for dynamic prioritization --- src/Common/tests/gtest_async_loader.cpp | 91 +++++++++++++++++++++++-- 1 file changed, 87 insertions(+), 4 deletions(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index eabddfbebeb..975cb6a4573 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -629,6 +629,46 @@ TEST(AsyncLoader, StaticPriorities) ASSERT_EQ(schedule, "A9E9D9F9G9H9C4B3"); } +TEST(AsyncLoader, SimplePrioritization) +{ + AsyncLoaderTest t({ + {.max_threads = 1, .priority = 0}, + {.max_threads = 1, .priority = 1}, + {.max_threads = 1, .priority = 2}, + }); + + t.loader.start(); + + std::atomic executed{0}; // Number of previously executed jobs (to test execution order) + LoadJobPtr job_to_prioritize; + + auto job_func_A_booster = [&] (const LoadJobPtr &) + { + ASSERT_EQ(executed++, 0); + t.loader.prioritize(job_to_prioritize, 2); + }; + + auto job_func_B_tester = [&] (const LoadJobPtr &) + { + ASSERT_EQ(executed++, 2); + }; + + auto job_func_C_boosted = [&] (const LoadJobPtr &) + { + ASSERT_EQ(executed++, 1); + }; + + std::vector jobs; + jobs.push_back(makeLoadJob({}, 1, "A", job_func_A_booster)); // 0 + jobs.push_back(makeLoadJob({jobs[0]}, 1, "tester", job_func_B_tester)); // 1 + jobs.push_back(makeLoadJob({}, 0, "C", job_func_C_boosted)); // 2 + auto task = makeLoadTask(t.loader, { jobs.begin(), jobs.end() }); + + job_to_prioritize = jobs[2]; // C + + scheduleAndWaitLoadAll(task); +} + TEST(AsyncLoader, DynamicPriorities) { AsyncLoaderTest t({ @@ -646,15 +686,53 @@ TEST(AsyncLoader, DynamicPriorities) for (bool prioritize : {false, true}) { + // Although all pools have max_threads=1, workers from different pools can run simultaneously just after `prioritize()` call + std::barrier sync(2); + bool wait_sync = prioritize; + std::mutex schedule_mutex; std::string schedule; LoadJobPtr job_to_prioritize; + // Order of execution of jobs D and E after prioritization is undefined, because it depend on `ready_seqno` + // (Which depends on initial `schedule()` order, which in turn depend on `std::unordered_map` order) + // So we have to obtain `ready_seqno` to be sure. + UInt64 ready_seqno_D = 0; + UInt64 ready_seqno_E = 0; + auto job_func = [&] (const LoadJobPtr & self) { + { + std::unique_lock lock{schedule_mutex}; + schedule += fmt::format("{}{}", self->name, self->execution_pool()); + } + if (prioritize && self->name == "C") - t.loader.prioritize(job_to_prioritize, 9); // dynamic prioritization - schedule += fmt::format("{}{}", self->name, self->pool()); + { + for (const auto & state : t.loader.getJobStates()) + { + if (state.job->name == "D") + ready_seqno_D = state.ready_seqno; + if (state.job->name == "E") + ready_seqno_E = state.ready_seqno; + } + + // Jobs D and E should be enqueued at the moment + ASSERT_LT(0, ready_seqno_D); + ASSERT_LT(0, ready_seqno_E); + + // Dynamic prioritization G0 -> G9 + // Note that it will spawn concurrent worker in higher priority pool + t.loader.prioritize(job_to_prioritize, 9); + + sync.arrive_and_wait(); // (A) wait for higher priority worker (B) to test they can be concurrent + } + + if (wait_sync && (self->name == "D" || self->name == "E")) + { + wait_sync = false; + sync.arrive_and_wait(); // (B) + } }; // Job DAG with initial priorities. During execution of C4, job G0 priority is increased to G9, postponing B3 job executing. @@ -676,14 +754,19 @@ TEST(AsyncLoader, DynamicPriorities) jobs.push_back(makeLoadJob({ jobs[6] }, 0, "H", job_func)); // 7 auto task = t.schedule({ jobs.begin(), jobs.end() }); - job_to_prioritize = jobs[6]; + job_to_prioritize = jobs[6]; // G t.loader.start(); t.loader.wait(); t.loader.stop(); if (prioritize) - ASSERT_EQ(schedule, "A4C4E9D9F9G9B3H0"); + { + if (ready_seqno_D < ready_seqno_E) + ASSERT_EQ(schedule, "A4C4D9E9F9G9B3H0"); + else + ASSERT_EQ(schedule, "A4C4E9D9F9G9B3H0"); + } else ASSERT_EQ(schedule, "A4C4B3E2D1F0G0H0"); } From 372601d6dfd9fb07a97882c1209f49e5c305adf4 Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 21 May 2023 17:04:49 +0000 Subject: [PATCH 07/46] fix --- src/Common/tests/gtest_async_loader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 975cb6a4573..2e4c42d6826 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -660,7 +660,7 @@ TEST(AsyncLoader, SimplePrioritization) std::vector jobs; jobs.push_back(makeLoadJob({}, 1, "A", job_func_A_booster)); // 0 - jobs.push_back(makeLoadJob({jobs[0]}, 1, "tester", job_func_B_tester)); // 1 + jobs.push_back(makeLoadJob({jobs[0]}, 1, "B", job_func_B_tester)); // 1 jobs.push_back(makeLoadJob({}, 0, "C", job_func_C_boosted)); // 2 auto task = makeLoadTask(t.loader, { jobs.begin(), jobs.end() }); From df436b2cd4a7f93bffc8e0295f5cbb3b1ac7b3c8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 May 2023 06:59:08 +0000 Subject: [PATCH 08/46] Spark compatibility: Add new function space() --- .../functions/string-functions.md | 42 ++++- src/Functions/repeat.cpp | 41 ++--- src/Functions/space.cpp | 166 ++++++++++++++++++ .../02752_space_function.reference | 51 ++++++ .../0_stateless/02752_space_function.sql | 50 ++++++ 5 files changed, 328 insertions(+), 22 deletions(-) create mode 100644 src/Functions/space.cpp create mode 100644 tests/queries/0_stateless/02752_space_function.reference create mode 100644 tests/queries/0_stateless/02752_space_function.sql diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 920a1f972d8..8662d08431c 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -323,11 +323,11 @@ Alias: `REPEAT` **Arguments** - `s` — The string to repeat. [String](../../sql-reference/data-types/string.md). -- `n` — The number of times to repeat the string. [UInt or Int](../../sql-reference/data-types/int-uint.md). +- `n` — The number of times to repeat the string. [UInt* or Int*](../../sql-reference/data-types/int-uint.md). **Returned value** -The single string containing string `s` repeated `n` times. If `n` \< 1, the function returns empty string. +A string containing string `s` repeated `n` times. If `n` <= 0, the function returns the empty string. Type: `String`. @@ -345,6 +345,44 @@ Result: └────────────────────────────────┘ ``` +## space + +Concatenates a space (` `) as many times with itself as specified. + +**Syntax** + +``` sql +space(n) +``` + +Alias: `SPACE`. + +**Arguments** + +- `n` — The number of times to repeat the space. [UInt* or Int*](../../sql-reference/data-types/int-uint.md). + +**Returned value** + +The string containing string ` ` repeated `n` times. If `n` <= 0, the function returns the empty string. + +Type: `String`. + +**Example** + +Query: + +``` sql +SELECT space(3); +``` + +Result: + +``` text +┌─space(3) ────┐ +│ │ +└──────────────┘ +``` + ## reverse Reverses the sequence of bytes in a string. diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index 0c323c39969..88eaebf4f03 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -186,36 +186,37 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isString(arguments[0])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", - arguments[0]->getName(), getName()); - if (!isInteger(arguments[1])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}", - arguments[1]->getName(), getName()); - return arguments[0]; + FunctionArgumentDescriptors args{ + {"s", &isString, nullptr, "String"}, + {"n", &isInteger, nullptr, "Integer"}, + }; + + validateFunctionArgumentTypes(*this, arguments, args); + + return std::make_shared(); } bool useDefaultImplementationForConstants() const override { return true; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override { - const auto & strcolumn = arguments[0].column; - const auto & numcolumn = arguments[1].column; + const auto & col_str = arguments[0].column; + const auto & col_num = arguments[1].column; ColumnPtr res; - if (const ColumnString * col = checkAndGetColumn(strcolumn.get())) + if (const ColumnString * col = checkAndGetColumn(col_str.get())) { - if (const ColumnConst * scale_column_num = checkAndGetColumn(numcolumn.get())) + if (const ColumnConst * col_num_const = checkAndGetColumn(col_num.get())) { auto col_res = ColumnString::create(); castType(arguments[1].type.get(), [&](const auto & type) { using DataType = std::decay_t; using T = typename DataType::FieldType; - T repeat_time = scale_column_num->getValue(); - RepeatImpl::vectorStrConstRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), repeat_time); + T times = col_num_const->getValue(); + RepeatImpl::vectorStrConstRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), times); return true; }); return col_res; @@ -224,9 +225,9 @@ public: { using DataType = std::decay_t; using T = typename DataType::FieldType; - const ColumnVector * colnum = checkAndGetColumn>(numcolumn.get()); + const ColumnVector * column = checkAndGetColumn>(col_num.get()); auto col_res = ColumnString::create(); - RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), colnum->getData()); + RepeatImpl::vectorStrVectorRepeat(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets(), column->getData()); res = std::move(col_res); return true; })) @@ -234,7 +235,7 @@ public: return res; } } - else if (const ColumnConst * col_const = checkAndGetColumn(strcolumn.get())) + else if (const ColumnConst * col_const = checkAndGetColumn(col_str.get())) { /// Note that const-const case is handled by useDefaultImplementationForConstants. @@ -244,9 +245,9 @@ public: { using DataType = std::decay_t; using T = typename DataType::FieldType; - const ColumnVector * colnum = checkAndGetColumn>(numcolumn.get()); + const ColumnVector * column = checkAndGetColumn>(col_num.get()); auto col_res = ColumnString::create(); - RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), colnum->getData()); + RepeatImpl::constStrVectorRepeat(copy_str, col_res->getChars(), col_res->getOffsets(), column->getData()); res = std::move(col_res); return true; })) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp new file mode 100644 index 00000000000..afdc01b717d --- /dev/null +++ b/src/Functions/space.cpp @@ -0,0 +1,166 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + +namespace +{ + +/// Prints whitespace n-times. Actually, space() could also be pushed down to repeat(). Chose a standalone-implementation because +/// we can do memset() whereas repeat() does memcpy(). +class FunctionSpace : public IFunction +{ +private: + static constexpr auto space = ' '; + +public: + static constexpr auto name = "space"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {"n", &isInteger, nullptr, "Integer"} + }; + + validateFunctionArgumentTypes(*this, arguments, args); + + return std::make_shared(); + } + + + template + bool executeConstant(ColumnPtr col_times, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const + { + const ColumnConst * col_times_const = checkAndGetColumn(col_times.get()); + + const ColumnPtr & col_times_const_internal = col_times_const->getDataColumnPtr(); + if (!checkAndGetColumn(col_times_const_internal.get())) + return false; + + using T = typename DataType::FieldType; + T times = col_times_const->getValue(); + + if (times < 1) + times = 0; + + res_offsets.resize(col_times->size()); + res_chars.resize(col_times->size() * (times + 1)); + + size_t pos = 0; + + for (size_t i = 0; i < col_times->size(); ++i) + { + memset(res_chars.begin() + pos, space, times); + pos += times; + + *(res_chars.begin() + pos) = '\n'; + pos += 1; + + res_offsets[i] = pos; + } + + return true; + } + + + template + bool executeVector(ColumnPtr col_times_, ColumnString::Offsets & res_offsets, ColumnString::Chars & res_chars) const + { + auto * col_times = checkAndGetColumn(col_times_.get()); + if (!col_times) + return false; + + res_offsets.resize(col_times->size()); + res_chars.reserve(col_times->size() * 10); /// heuristic + + const PaddedPODArray & times_data = col_times->getData(); + + size_t pos = 0; + + for (size_t i = 0; i < col_times->size(); ++i) + { + typename DataType::FieldType times = times_data[i]; + + if (times < 1) + times = 0; + + if (res_chars.size() + times + 1 >= res_chars.capacity()) + res_chars.reserve(2 * res_chars.capacity()); + + memset(res_chars.begin() + pos, space, times); + pos += times; + + *(res_chars.begin() + pos) = '\n'; + pos += 1; + + res_offsets[i] = pos; + } + + res_chars.resize(pos); + + return true; + } + + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + const auto & col_num = arguments[0].column; + + auto col_res = ColumnString::create(); + + ColumnString::Offsets & res_offsets = col_res->getOffsets(); + ColumnString::Chars & res_chars = col_res->getChars(); + + if (const ColumnConst * col_num_const = checkAndGetColumn(col_num.get())) + { + if ((executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars)) + || (executeConstant(col_num, res_offsets, res_chars))) + return col_res; + } + else + { + if ((executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars)) + || (executeVector(col_num, res_offsets, res_chars))) + return col_res; + } + + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", arguments[0].column->getName(), getName()); + } +}; +} + +REGISTER_FUNCTION(Space) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} diff --git a/tests/queries/0_stateless/02752_space_function.reference b/tests/queries/0_stateless/02752_space_function.reference new file mode 100644 index 00000000000..b0adccb415b --- /dev/null +++ b/tests/queries/0_stateless/02752_space_function.reference @@ -0,0 +1,51 @@ +const times, uint + + + + +const times, int + + + + +negative const times + + + + +negative tests +\N +non-const times, uint + 3 + 2 + 1 + 0 + 12 + 10 + 4 + 5 + 4 + 21 + 9 + 7 + 56 + 20 + 5 + 7 +non-const times, int + 3 + 2 + 1 + 0 + 12 + 10 + 4 + 5 + 0 + 0 + 0 + 0 + 56 + 20 + 5 + 7 diff --git a/tests/queries/0_stateless/02752_space_function.sql b/tests/queries/0_stateless/02752_space_function.sql new file mode 100644 index 00000000000..8f3605e6e6f --- /dev/null +++ b/tests/queries/0_stateless/02752_space_function.sql @@ -0,0 +1,50 @@ +SELECT 'const times, uint'; +SELECT space(3::UInt8); +SELECT space(3::UInt16); +SELECT space(3::UInt32); +SELECT space(3::UInt64); +SELECT 'const times, int'; +SELECT space(3::Int8); +SELECT space(3::Int16); +SELECT space(3::Int32); +SELECT space(3::Int64); + +SELECT 'negative const times'; +SELECT space(-3::Int8); +SELECT space(-3::Int16); +SELECT space(-3::Int32); +SELECT space(-3::Int64); + +SELECT 'negative tests'; +SELECT space('abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT space(['abc']); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT space(('abc')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT space(NULL); + +DROP TABLE IF EXISTS defaults; +CREATE TABLE defaults +( + u8 UInt8, + u16 UInt16, + u32 UInt32, + u64 UInt64, + i8 Int8, + i16 Int16, + i32 Int32, + i64 Int64 +) ENGINE = Memory(); + +INSERT INTO defaults values (3, 12, 4, 56, 3, 12, -4, 56) (2, 10, 21, 20, 2, 10, -21, 20) (1, 4, 9, 5, 1, 4, -9, 5) (0, 5, 7, 7, 0, 5, -7, 7); + +SELECT 'non-const times, uint'; +SELECT space(u8), length(space(u8)) FROM defaults; +SELECT space(u16), length(space(u16)) FROM defaults; +SELECT space(u32), length(space(u32)) from defaults; +SELECT space(u64), length(space(u64)) FROM defaults; +SELECT 'non-const times, int'; +SELECT space(i8), length(space(i8)) FROM defaults; +SELECT space(i16), length(space(i16)) FROM defaults; +SELECT space(i32), length(space(i32)) from defaults; +SELECT space(i64), length(space(i64)) FROM defaults; + +DROP TABLE defaults; From d5cfcdfae10c3706ad2e33d0e3a1f52e6c59ece0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 May 2023 19:09:11 +0000 Subject: [PATCH 09/46] String terminator: \n --> \0 --- src/Functions/space.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index afdc01b717d..dd30e548f30 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -70,7 +70,7 @@ public: memset(res_chars.begin() + pos, space, times); pos += times; - *(res_chars.begin() + pos) = '\n'; + *(res_chars.begin() + pos) = '\0'; pos += 1; res_offsets[i] = pos; @@ -107,7 +107,7 @@ public: memset(res_chars.begin() + pos, space, times); pos += times; - *(res_chars.begin() + pos) = '\n'; + *(res_chars.begin() + pos) = '\0'; pos += 1; res_offsets[i] = pos; From d76498dca06962873d9fa654d9cda55937135a82 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 May 2023 19:19:08 +0000 Subject: [PATCH 10/46] reserve() --> resize() --- src/Functions/space.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index dd30e548f30..70e4d3e6794 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -88,7 +88,7 @@ public: return false; res_offsets.resize(col_times->size()); - res_chars.reserve(col_times->size() * 10); /// heuristic + res_chars.resize(col_times->size() * 10); /// heuristic const PaddedPODArray & times_data = col_times->getData(); @@ -102,7 +102,7 @@ public: times = 0; if (res_chars.size() + times + 1 >= res_chars.capacity()) - res_chars.reserve(2 * res_chars.capacity()); + res_chars.resize(2 * res_chars.capacity()); memset(res_chars.begin() + pos, space, times); pos += times; From 786bbee62cec5ec55750eaace43bebae683d6197 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 22 May 2023 20:03:29 +0000 Subject: [PATCH 11/46] Extend tests --- .../02752_space_function.reference | 69 ++++++++++++++----- .../0_stateless/02752_space_function.sql | 47 ++++++++----- 2 files changed, 82 insertions(+), 34 deletions(-) diff --git a/tests/queries/0_stateless/02752_space_function.reference b/tests/queries/0_stateless/02752_space_function.reference index b0adccb415b..d265a843ba9 100644 --- a/tests/queries/0_stateless/02752_space_function.reference +++ b/tests/queries/0_stateless/02752_space_function.reference @@ -1,21 +1,56 @@ -const times, uint - - - - -const times, int - - - - -negative const times - - - - +const, uint + 3 + 3 + 3 + 3 +const, int + 3 + 3 + 3 + 3 +const, int, negative + 0 + 0 + 0 + 0 negative tests +null \N -non-const times, uint +const, uint, multiple + + + + + + + + + + + + + + + + +const int, multiple + + + + + + + + + + + + + + + + +non-const, uint 3 2 1 @@ -32,7 +67,7 @@ non-const times, uint 20 5 7 -non-const times, int +non-const, int 3 2 1 diff --git a/tests/queries/0_stateless/02752_space_function.sql b/tests/queries/0_stateless/02752_space_function.sql index 8f3605e6e6f..d621af13096 100644 --- a/tests/queries/0_stateless/02752_space_function.sql +++ b/tests/queries/0_stateless/02752_space_function.sql @@ -1,24 +1,26 @@ -SELECT 'const times, uint'; -SELECT space(3::UInt8); -SELECT space(3::UInt16); -SELECT space(3::UInt32); -SELECT space(3::UInt64); -SELECT 'const times, int'; -SELECT space(3::Int8); -SELECT space(3::Int16); -SELECT space(3::Int32); -SELECT space(3::Int64); +SELECT 'const, uint'; +SELECT space(3::UInt8), length(space(3::UInt8)); +SELECT space(3::UInt16), length(space(3::UInt16)); +SELECT space(3::UInt32), length(space(3::UInt32)); +SELECT space(3::UInt64), length(space(3::UInt64)); +SELECT 'const, int'; +SELECT space(3::Int8), length(space(3::Int8)); +SELECT space(3::Int16), length(space(3::Int16)); +SELECT space(3::Int32), length(space(3::Int32)); +SELECT space(3::Int64), length(space(3::Int64)); -SELECT 'negative const times'; -SELECT space(-3::Int8); -SELECT space(-3::Int16); -SELECT space(-3::Int32); -SELECT space(-3::Int64); +SELECT 'const, int, negative'; +SELECT space(-3::Int8), length(space(-3::Int8)); +SELECT space(-3::Int16), length(space(-3::Int16)); +SELECT space(-3::Int32), length(space(-3::Int32)); +SELECT space(-3::Int64), length(space(-3::Int64)); SELECT 'negative tests'; SELECT space('abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT space(['abc']); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT space(('abc')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +SELECT 'null'; SELECT space(NULL); DROP TABLE IF EXISTS defaults; @@ -36,12 +38,23 @@ CREATE TABLE defaults INSERT INTO defaults values (3, 12, 4, 56, 3, 12, -4, 56) (2, 10, 21, 20, 2, 10, -21, 20) (1, 4, 9, 5, 1, 4, -9, 5) (0, 5, 7, 7, 0, 5, -7, 7); -SELECT 'non-const times, uint'; +SELECT 'const, uint, multiple'; +SELECT space(30::UInt8) FROM defaults; +SELECT space(30::UInt16) FROM defaults; +SELECT space(30::UInt32) FROM defaults; +SELECT space(30::UInt64) FROM defaults; +SELECT 'const int, multiple'; +SELECT space(30::Int8) FROM defaults; +SELECT space(30::Int16) FROM defaults; +SELECT space(30::Int32) FROM defaults; +SELECT space(30::Int64) FROM defaults; + +SELECT 'non-const, uint'; SELECT space(u8), length(space(u8)) FROM defaults; SELECT space(u16), length(space(u16)) FROM defaults; SELECT space(u32), length(space(u32)) from defaults; SELECT space(u64), length(space(u64)) FROM defaults; -SELECT 'non-const times, int'; +SELECT 'non-const, int'; SELECT space(i8), length(space(i8)) FROM defaults; SELECT space(i16), length(space(i16)) FROM defaults; SELECT space(i32), length(space(i32)) from defaults; From d9a7227cf414caad0b45f921e1805f7eeaed3520 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 May 2023 06:49:19 +0000 Subject: [PATCH 12/46] Fix style check --- src/Functions/repeat.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index 88eaebf4f03..93f569d3eea 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -13,7 +13,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int TOO_LARGE_STRING_SIZE; } From b106757a3cb7417b32438706b290fbd1013c8fd7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 May 2023 06:51:59 +0000 Subject: [PATCH 13/46] Fix 02415_all_new_functions_must_be_documented --- .../02415_all_new_functions_must_be_documented.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 79a6ad1fa2d..7ab26982402 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -655,6 +655,7 @@ sleep sleepEachRow snowflakeToDateTime snowflakeToDateTime64 +space splitByChar splitByNonAlpha splitByRegexp From 285e8f4ae1f8bc3dc8a8b1d7e12bc152debc0650 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 23 May 2023 12:16:49 +0000 Subject: [PATCH 14/46] Protect against DOS --- src/Functions/repeat.cpp | 8 +++----- src/Functions/space.cpp | 13 +++++++++++++ tests/queries/0_stateless/02752_space_function.sql | 1 + 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/src/Functions/repeat.cpp b/src/Functions/repeat.cpp index 93f569d3eea..c1b553ac6b3 100644 --- a/src/Functions/repeat.cpp +++ b/src/Functions/repeat.cpp @@ -24,18 +24,16 @@ struct RepeatImpl /// Safety threshold against DoS. static inline void checkRepeatTime(UInt64 repeat_time) { - static constexpr UInt64 max_repeat_times = 1000000; + static constexpr UInt64 max_repeat_times = 1'000'000; if (repeat_time > max_repeat_times) - throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", - std::to_string(repeat_time), std::to_string(max_repeat_times)); + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", repeat_time, max_repeat_times); } static inline void checkStringSize(UInt64 size) { static constexpr UInt64 max_string_size = 1 << 30; if (size > max_string_size) - throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size ({}) in function repeat, maximum is: {}", - size, max_string_size); + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too large string size ({}) in function repeat, maximum is: {}", size, max_string_size); } template diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index 70e4d3e6794..9d825a8b294 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -13,6 +13,7 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; + extern const int TOO_LARGE_STRING_SIZE; } namespace @@ -25,6 +26,14 @@ class FunctionSpace : public IFunction private: static constexpr auto space = ' '; + /// Safety threshold against DoS. + static inline void checkRepeatTime(UInt64 repeat_time) + { + static constexpr UInt64 max_repeat_times = 1'000'000; + if (repeat_time > max_repeat_times) + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", repeat_time, max_repeat_times); + } + public: static constexpr auto name = "space"; static FunctionPtr create(ContextPtr) { return std::make_shared(); } @@ -60,6 +69,8 @@ public: if (times < 1) times = 0; + checkRepeatTime(times); + res_offsets.resize(col_times->size()); res_chars.resize(col_times->size() * (times + 1)); @@ -101,6 +112,8 @@ public: if (times < 1) times = 0; + checkRepeatTime(times); + if (res_chars.size() + times + 1 >= res_chars.capacity()) res_chars.resize(2 * res_chars.capacity()); diff --git a/tests/queries/0_stateless/02752_space_function.sql b/tests/queries/0_stateless/02752_space_function.sql index d621af13096..b12906927df 100644 --- a/tests/queries/0_stateless/02752_space_function.sql +++ b/tests/queries/0_stateless/02752_space_function.sql @@ -19,6 +19,7 @@ SELECT 'negative tests'; SELECT space('abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT space(['abc']); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT space(('abc')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT space(30303030303030303030303030303030::UInt64); -- { serverError TOO_LARGE_STRING_SIZE } SELECT 'null'; SELECT space(NULL); From f76f989b53645136a5e83a9a1a9ab1335e9a2cbf Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Wed, 24 May 2023 13:33:05 +0000 Subject: [PATCH 15/46] Implement a uniform way to query processor core IDs --- src/Common/AsynchronousMetrics.cpp | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index ac2180103c5..3753aaca405 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -1041,18 +1041,16 @@ void AsynchronousMetrics::update(TimePoint update_time) // It doesn't read the EOL itself. ++cpuinfo->position(); - if (s.rfind("processor", 0) == 0) + static constexpr std::string_view PROCESSOR = "processor"; + if (s.starts_with(PROCESSOR)) { /// s390x example: processor 0: version = FF, identification = 039C88, machine = 3906 /// non s390x example: processor : 0 - if (auto colon = s.find_first_of(':')) - { -#ifdef __s390x__ - core_id = std::stoi(s.substr(10)); /// 10: length of "processor" plus 1 -#else - core_id = std::stoi(s.substr(colon + 2)); -#endif - } + auto core_id_start = std::ssize(PROCESSOR); + while (core_id_start < std::ssize(s) && !std::isdigit(s[core_id_start])) + ++core_id_start; + + core_id = std::stoi(s.substr(core_id_start)); } else if (s.rfind("cpu MHz", 0) == 0) { From 2a3362e0c8d4bc7a49a8031d8bc4860e30bfaa8f Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 24 May 2023 17:27:47 +0200 Subject: [PATCH 16/46] Implement encrypted disk transaction and fix shared merge tree with encrypted disk --- src/Disks/DiskEncrypted.cpp | 9 +- src/Disks/DiskEncrypted.h | 29 +-- src/Disks/DiskEncryptedTransaction.cpp | 123 +++++++++++++ src/Disks/DiskEncryptedTransaction.h | 242 +++++++++++++++++++++++++ 4 files changed, 385 insertions(+), 18 deletions(-) create mode 100644 src/Disks/DiskEncryptedTransaction.cpp create mode 100644 src/Disks/DiskEncryptedTransaction.h diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index 72f668db00b..a94036ef1a6 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -203,18 +203,19 @@ private: }; DiskEncrypted::DiskEncrypted( - const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_) - : DiskEncrypted(name_, parseDiskEncryptedSettings(name_, config_, config_prefix_, map_)) + const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_, bool use_fake_transaction_) + : DiskEncrypted(name_, parseDiskEncryptedSettings(name_, config_, config_prefix_, map_), use_fake_transaction_) { } -DiskEncrypted::DiskEncrypted(const String & name_, std::unique_ptr settings_) +DiskEncrypted::DiskEncrypted(const String & name_, std::unique_ptr settings_, bool use_fake_transaction_) : IDisk(name_) , delegate(settings_->wrapped_disk) , encrypted_name(name_) , disk_path(settings_->disk_path) , disk_absolute_path(settings_->wrapped_disk->getPath() + settings_->disk_path) , current_settings(std::move(settings_)) + , use_fake_transaction(use_fake_transaction_) { delegate->createDirectories(disk_path); } @@ -416,7 +417,7 @@ void registerDiskEncrypted(DiskFactory & factory, bool global_skip_access_check) const DisksMap & map) -> DiskPtr { bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false); - DiskPtr disk = std::make_shared(name, config, config_prefix, map); + DiskPtr disk = std::make_shared(name, config, config_prefix, map, config.getBool(config_prefix + ".use_fake_transaction", true)); disk->startup(context, skip_access_check); return disk; }; diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 530d9b2dc02..57de8177a2b 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -6,22 +6,14 @@ #include #include #include +#include namespace DB { + class ReadBufferFromFileBase; class WriteBufferFromFileBase; -namespace FileEncryption { enum class Algorithm; } - -struct DiskEncryptedSettings -{ - DiskPtr wrapped_disk; - String disk_path; - std::unordered_map keys; - UInt64 current_key_id; - FileEncryption::Algorithm current_algorithm; -}; /// Encrypted disk ciphers all written files on the fly and writes the encrypted files to an underlying (normal) disk. /// And when we read files from an encrypted disk it deciphers them automatically, @@ -29,8 +21,8 @@ struct DiskEncryptedSettings class DiskEncrypted : public IDisk { public: - DiskEncrypted(const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_); - DiskEncrypted(const String & name_, std::unique_ptr settings_); + DiskEncrypted(const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_, bool use_fake_transaction_); + DiskEncrypted(const String & name_, std::unique_ptr settings_, bool use_fake_transaction_); const String & getName() const override { return encrypted_name; } const String & getPath() const override { return disk_absolute_path; } @@ -69,7 +61,6 @@ public: delegate->createDirectories(wrapped_path); } - void clearDirectory(const String & path) override { auto wrapped_path = wrappedPath(path); @@ -293,7 +284,16 @@ public: { /// Need to overwrite explicetly because this disk change /// a lot of "delegate" methods. - return std::make_shared(*this); + + if (use_fake_transaction) + { + return std::make_shared(*this); + } + else + { + auto delegate_transaction = delegate->createTransaction(); + return std::make_shared(delegate_transaction, disk_path, *current_settings.get(), delegate.get()); + } } UInt64 getTotalSpace() const override @@ -342,6 +342,7 @@ private: const String disk_path; const String disk_absolute_path; MultiVersion current_settings; + bool use_fake_transaction; }; } diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp new file mode 100644 index 00000000000..7174e98f256 --- /dev/null +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -0,0 +1,123 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int DATA_ENCRYPTION_ERROR; + extern const int NOT_IMPLEMENTED; +} + + +namespace +{ + +FileEncryption::Header readHeader(ReadBufferFromFileBase & read_buffer) +{ + try + { + FileEncryption::Header header; + header.read(read_buffer); + return header; + } + catch (Exception & e) + { + e.addMessage("While reading the header of encrypted file " + quoteString(read_buffer.getFileName())); + throw; + } +} + +String getCurrentKey(const String & path, const DiskEncryptedSettings & settings) +{ + auto it = settings.keys.find(settings.current_key_id); + if (it == settings.keys.end()) + throw Exception( + ErrorCodes::DATA_ENCRYPTION_ERROR, + "Not found a key with the current ID {} required to cipher file {}", + settings.current_key_id, + quoteString(path)); + + return it->second; +} + +String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings) +{ + auto it = settings.keys.find(header.key_id); + if (it == settings.keys.end()) + throw Exception( + ErrorCodes::DATA_ENCRYPTION_ERROR, + "Not found a key with ID {} required to decipher file {}", + header.key_id, + quoteString(path)); + + String key = it->second; + if (FileEncryption::calculateKeyHash(key) != header.key_hash) + throw Exception( + ErrorCodes::DATA_ENCRYPTION_ERROR, "Wrong key with ID {}, could not decipher file {}", header.key_id, quoteString(path)); + + return key; +} + +} + +void DiskEncryptedTransaction::copyFile(const std::string & from_file_path, const std::string & to_file_path) +{ + auto wrapped_from_path = wrappedPath(from_file_path); + auto wrapped_to_path = wrappedPath(to_file_path); + delegate_transaction->copyFile(wrapped_from_path, wrapped_to_path); +} + +std::unique_ptr DiskEncryptedTransaction::writeFile( + const std::string & path, + size_t buf_size, + WriteMode mode, + const WriteSettings & settings, + bool autocommit) +{ + auto wrapped_path = wrappedPath(path); + FileEncryption::Header header; + String key; + UInt64 old_file_size = 0; + if (mode == WriteMode::Append && delegate_disk->exists(path)) + { + old_file_size = delegate_disk->getFileSize(path); + if (old_file_size) + { + /// Append mode: we continue to use the same header. + auto read_buffer = delegate_disk->readFile(wrapped_path, ReadSettings().adjustBufferSize(FileEncryption::Header::kSize)); + header = readHeader(*read_buffer); + key = getKey(path, header, current_settings); + } + } + if (!old_file_size) + { + /// Rewrite mode: we generate a new header. + key = getCurrentKey(path, current_settings); + header.algorithm = current_settings.current_algorithm; + header.key_id = current_settings.current_key_id; + header.key_hash = FileEncryption::calculateKeyHash(key); + header.init_vector = FileEncryption::InitVector::random(); + } + auto buffer = delegate_transaction->writeFile(wrapped_path, buf_size, mode, settings, autocommit); + return std::make_unique(buf_size, std::move(buffer), key, header, old_file_size); + +} +void DiskEncryptedTransaction::writeFileUsingCustomWriteObject( + const String &, + WriteMode, + std::function & object_attributes)>) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `writeFileUsingCustomWriteObject()` is not implemented"); +} + + + +} diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h new file mode 100644 index 00000000000..d68e0c42807 --- /dev/null +++ b/src/Disks/DiskEncryptedTransaction.h @@ -0,0 +1,242 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace FileEncryption { enum class Algorithm; } + +struct DiskEncryptedSettings +{ + DiskPtr wrapped_disk; + String disk_path; + std::unordered_map keys; + UInt64 current_key_id; + FileEncryption::Algorithm current_algorithm; +}; + +class DiskEncryptedTransaction : public IDiskTransaction +{ +public: + explicit DiskEncryptedTransaction(DiskTransactionPtr delegate_transaction_, const std::string & disk_path_, DiskEncryptedSettings current_settings_, IDisk * delegate_disk_) + : delegate_transaction(delegate_transaction_) + , disk_path(disk_path_) + , current_settings(current_settings_) + , delegate_disk(delegate_disk_) + {} + + /// Tries to commit all accumulated operations simultaneously. + /// If something fails rollback and throw exception. + void commit(const TransactionCommitOptionsVariant & options = NoCommitOptions{}) override // NOLINT + { + delegate_transaction->commit(options); + } + + void undo() override + { + delegate_transaction->undo(); + } + + TransactionCommitOutcomeVariant tryCommit(const TransactionCommitOptionsVariant & options) override + { + return delegate_transaction->tryCommit(options); + } + + ~DiskEncryptedTransaction() override = default; + + /// Create directory. + void createDirectory(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->createDirectory(wrapped_path); + } + + /// Create directory and all parent directories if necessary. + void createDirectories(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->createDirectories(wrapped_path); + } + + /// Remove all files from the directory. Directories are not removed. + void clearDirectory(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->clearDirectory(wrapped_path); + } + + /// Move directory from `from_path` to `to_path`. + void moveDirectory(const std::string & from_path, const std::string & to_path) override + { + auto wrapped_from_path = wrappedPath(from_path); + auto wrapped_to_path = wrappedPath(to_path); + delegate_transaction->moveDirectory(wrapped_from_path, wrapped_to_path); + } + + void moveFile(const std::string & from_path, const std::string & to_path) override + { + auto wrapped_from_path = wrappedPath(from_path); + auto wrapped_to_path = wrappedPath(to_path); + delegate_transaction->moveFile(wrapped_from_path, wrapped_to_path); + + } + + void createFile(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->createFile(wrapped_path); + } + + /// Move the file from `from_path` to `to_path`. + /// If a file with `to_path` path already exists, it will be replaced. + void replaceFile(const std::string & from_path, const std::string & to_path) override + { + auto wrapped_from_path = wrappedPath(from_path); + auto wrapped_to_path = wrappedPath(to_path); + delegate_transaction->replaceFile(wrapped_from_path, wrapped_to_path); + } + + /// Only copy of several files supported now. Disk interface support copy to another disk + /// but it's impossible to implement correctly in transactions because other disk can + /// use different metadata storage. + /// TODO: maybe remove it at all, we don't want copies + void copyFile(const std::string & from_file_path, const std::string & to_file_path) override; + + /// Open the file for write and return WriteBufferFromFileBase object. + std::unique_ptr writeFile( /// NOLINT + const std::string & path, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, + WriteMode mode = WriteMode::Rewrite, + const WriteSettings & settings = {}, + bool autocommit = true) override; + + /// Write a file using a custom function to write an object to the disk's object storage. + void writeFileUsingCustomWriteObject( + const String & path, + WriteMode mode, + std::function & object_attributes)> + custom_write_object_function) override; + + /// Remove file. Throws exception if file doesn't exists or it's a directory. + void removeFile(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeFile(wrapped_path); + } + + /// Remove file if it exists. + void removeFileIfExists(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeFileIfExists(wrapped_path); + } + + /// Remove directory. Throws exception if it's not a directory or if directory is not empty. + void removeDirectory(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeDirectory(wrapped_path); + } + + /// Remove file or directory with all children. Use with extra caution. Throws exception if file doesn't exists. + void removeRecursive(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeRecursive(wrapped_path); + } + + /// Remove file. Throws exception if file doesn't exists or if directory is not empty. + /// Differs from removeFile for S3/HDFS disks + /// Second bool param is a flag to remove (true) or keep (false) shared data on S3 + void removeSharedFile(const std::string & path, bool keep_shared_data) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeSharedFile(wrapped_path, keep_shared_data); + } + + /// Remove file or directory with all children. Use with extra caution. Throws exception if file doesn't exists. + /// Differs from removeRecursive for S3/HDFS disks + /// Second bool param is a flag to remove (false) or keep (true) shared data on S3. + /// Third param determines which files cannot be removed even if second is true. + void removeSharedRecursive(const std::string & path, bool keep_all_shared_data, const NameSet & file_names_remove_metadata_only) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeSharedRecursive(wrapped_path, keep_all_shared_data, file_names_remove_metadata_only); + } + + /// Remove file or directory if it exists. + /// Differs from removeFileIfExists for S3/HDFS disks + /// Second bool param is a flag to remove (true) or keep (false) shared data on S3 + void removeSharedFileIfExists(const std::string & path, bool keep_shared_data) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->removeSharedFileIfExists(wrapped_path, keep_shared_data); + } + + /// Batch request to remove multiple files. + /// May be much faster for blob storage. + /// Second bool param is a flag to remove (true) or keep (false) shared data on S3. + /// Third param determines which files cannot be removed even if second is true. + void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override + { + for (const auto & file : files) + { + auto wrapped_path = wrappedPath(file.path); + bool keep = keep_all_batch_data || file_names_remove_metadata_only.contains(fs::path(file.path).filename()); + if (file.if_exists) + delegate_transaction->removeSharedFileIfExists(wrapped_path, keep); + else + delegate_transaction->removeSharedFile(wrapped_path, keep); + } + } + + /// Set last modified time to file or directory at `path`. + void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->setLastModified(wrapped_path, timestamp); + } + + /// Just chmod. + void chmod(const String & path, mode_t mode) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->chmod(wrapped_path, mode); + } + + /// Set file at `path` as read-only. + void setReadOnly(const std::string & path) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->setReadOnly(wrapped_path); + } + + /// Create hardlink from `src_path` to `dst_path`. + void createHardLink(const std::string & src_path, const std::string & dst_path) override + { + auto wrapped_src_path = wrappedPath(src_path); + auto wrapped_dst_path = wrappedPath(dst_path); + delegate_transaction->createHardLink(wrapped_src_path, wrapped_dst_path); + } + +private: + String wrappedPath(const String & path) const + { + // if path starts_with disk_path -> got already wrapped path + if (!disk_path.empty() && path.starts_with(disk_path)) + return path; + return disk_path + path; + } + + DiskTransactionPtr delegate_transaction; + std::string disk_path; + DiskEncryptedSettings current_settings; + IDisk * delegate_disk; +}; + +} From 5676a2c8805c9775017e0c40964fbd5fe1235731 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 24 May 2023 19:34:51 +0200 Subject: [PATCH 17/46] Small refactoring of encrypted disk --- src/Disks/DiskEncrypted.cpp | 45 --------- src/Disks/DiskEncrypted.h | 130 ++++++++++++++----------- src/Disks/DiskEncryptedTransaction.cpp | 9 -- src/Disks/DiskEncryptedTransaction.h | 36 ++++--- 4 files changed, 93 insertions(+), 127 deletions(-) diff --git a/src/Disks/DiskEncrypted.cpp b/src/Disks/DiskEncrypted.cpp index a94036ef1a6..2415b432e01 100644 --- a/src/Disks/DiskEncrypted.cpp +++ b/src/Disks/DiskEncrypted.cpp @@ -138,19 +138,6 @@ namespace } } - String getCurrentKey(const String & path, const DiskEncryptedSettings & settings) - { - auto it = settings.keys.find(settings.current_key_id); - if (it == settings.keys.end()) - throw Exception( - ErrorCodes::DATA_ENCRYPTION_ERROR, - "Not found a key with the current ID {} required to cipher file {}", - settings.current_key_id, - quoteString(path)); - - return it->second; - } - String getKey(const String & path, const FileEncryption::Header & header, const DiskEncryptedSettings & settings) { auto it = settings.keys.find(header.key_id); @@ -310,38 +297,6 @@ std::unique_ptr DiskEncrypted::readFile( return std::make_unique(settings.local_fs_buffer_size, std::move(buffer), key, header); } -std::unique_ptr DiskEncrypted::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &) -{ - auto wrapped_path = wrappedPath(path); - FileEncryption::Header header; - String key; - UInt64 old_file_size = 0; - auto settings = current_settings.get(); - if (mode == WriteMode::Append && exists(path)) - { - old_file_size = getFileSize(path); - if (old_file_size) - { - /// Append mode: we continue to use the same header. - auto read_buffer = delegate->readFile(wrapped_path, ReadSettings().adjustBufferSize(FileEncryption::Header::kSize)); - header = readHeader(*read_buffer); - key = getKey(path, header, *settings); - } - } - if (!old_file_size) - { - /// Rewrite mode: we generate a new header. - key = getCurrentKey(path, *settings); - header.algorithm = settings->current_algorithm; - header.key_id = settings->current_key_id; - header.key_hash = calculateKeyHash(key); - header.init_vector = InitVector::random(); - } - auto buffer = delegate->writeFile(wrapped_path, buf_size, mode); - return std::make_unique(buf_size, std::move(buffer), key, header, old_file_size); -} - - size_t DiskEncrypted::getFileSize(const String & path) const { auto wrapped_path = wrappedPath(path); diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 57de8177a2b..73bbe3fd74a 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -51,27 +51,30 @@ public: void createDirectory(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->createDirectory(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->createDirectory(path); + tx->commit(); } void createDirectories(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->createDirectories(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->createDirectories(path); + tx->commit(); } void clearDirectory(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->clearDirectory(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->clearDirectory(path); + tx->commit(); } void moveDirectory(const String & from_path, const String & to_path) override { - auto wrapped_from_path = wrappedPath(from_path); - auto wrapped_to_path = wrappedPath(to_path); - delegate->moveDirectory(wrapped_from_path, wrapped_to_path); + auto tx = createEncryptedTransaction(); + tx->moveDirectory(from_path, to_path); + tx->commit(); } DirectoryIteratorPtr iterateDirectory(const String & path) const override @@ -82,22 +85,23 @@ public: void createFile(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->createFile(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->createFile(path); + tx->commit(); } void moveFile(const String & from_path, const String & to_path) override { - auto wrapped_from_path = wrappedPath(from_path); - auto wrapped_to_path = wrappedPath(to_path); - delegate->moveFile(wrapped_from_path, wrapped_to_path); + auto tx = createEncryptedTransaction(); + tx->moveFile(from_path, to_path); + tx->commit(); } void replaceFile(const String & from_path, const String & to_path) override { - auto wrapped_from_path = wrappedPath(from_path); - auto wrapped_to_path = wrappedPath(to_path); - delegate->replaceFile(wrapped_from_path, wrapped_to_path); + auto tx = createEncryptedTransaction(); + tx->replaceFile(from_path, to_path); + tx->commit(); } void listFiles(const String & path, std::vector & file_names) const override @@ -120,61 +124,67 @@ public: const String & path, size_t buf_size, WriteMode mode, - const WriteSettings & settings) override; + const WriteSettings & settings) override + { + auto tx = createEncryptedTransaction(); + auto result = tx->writeFile(path, buf_size, mode, settings); + return result; + } void removeFile(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->removeFile(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->removeFile(path); + tx->commit(); } void removeFileIfExists(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->removeFileIfExists(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->removeFileIfExists(path); + tx->commit(); } void removeDirectory(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->removeDirectory(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->removeDirectory(path); + tx->commit(); } void removeRecursive(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->removeRecursive(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->removeRecursive(path); + tx->commit(); } void removeSharedFile(const String & path, bool flag) override { - auto wrapped_path = wrappedPath(path); - delegate->removeSharedFile(wrapped_path, flag); + auto tx = createEncryptedTransaction(); + tx->removeSharedFile(path, flag); + tx->commit(); } void removeSharedRecursive(const String & path, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override { - auto wrapped_path = wrappedPath(path); - delegate->removeSharedRecursive(wrapped_path, keep_all_batch_data, file_names_remove_metadata_only); + auto tx = createEncryptedTransaction(); + tx->removeSharedRecursive(path, keep_all_batch_data, file_names_remove_metadata_only); + tx->commit(); } void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override { - for (const auto & file : files) - { - auto wrapped_path = wrappedPath(file.path); - bool keep = keep_all_batch_data || file_names_remove_metadata_only.contains(fs::path(file.path).filename()); - if (file.if_exists) - delegate->removeSharedFileIfExists(wrapped_path, keep); - else - delegate->removeSharedFile(wrapped_path, keep); - } + auto tx = createEncryptedTransaction(); + tx->removeSharedFiles(files, keep_all_batch_data, file_names_remove_metadata_only); + tx->commit(); } void removeSharedFileIfExists(const String & path, bool flag) override { - auto wrapped_path = wrappedPath(path); - delegate->removeSharedFileIfExists(wrapped_path, flag); + auto tx = createEncryptedTransaction(); + tx->removeSharedFileIfExists(path, flag); + tx->commit(); } Strings getBlobPath(const String & path) const override @@ -185,8 +195,9 @@ public: void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override { - auto wrapped_path = wrappedPath(path); - delegate->writeFileUsingBlobWritingFunction(wrapped_path, mode, std::move(write_blob_function)); + auto tx = createEncryptedTransaction(); + tx->writeFileUsingBlobWritingFunction(path, mode, std::move(write_blob_function)); + tx->commit(); } std::unique_ptr readEncryptedFile(const String & path, const ReadSettings & settings) const override @@ -201,8 +212,9 @@ public: WriteMode mode, const WriteSettings & settings) const override { - auto wrapped_path = wrappedPath(path); - return delegate->writeFile(wrapped_path, buf_size, mode, settings); + auto tx = createEncryptedTransaction(); + auto buf = tx->writeEncryptedFile(path, buf_size, mode, settings); + return buf; } size_t getEncryptedFileSize(const String & path) const override @@ -219,8 +231,9 @@ public: void setLastModified(const String & path, const Poco::Timestamp & timestamp) override { - auto wrapped_path = wrappedPath(path); - delegate->setLastModified(wrapped_path, timestamp); + auto tx = createEncryptedTransaction(); + tx->setLastModified(path, timestamp); + tx->commit(); } Poco::Timestamp getLastModified(const String & path) const override @@ -237,15 +250,16 @@ public: void setReadOnly(const String & path) override { - auto wrapped_path = wrappedPath(path); - delegate->setReadOnly(wrapped_path); + auto tx = createEncryptedTransaction(); + tx->setReadOnly(path); + tx->commit(); } void createHardLink(const String & src_path, const String & dst_path) override { - auto wrapped_src_path = wrappedPath(src_path); - auto wrapped_dst_path = wrappedPath(dst_path); - delegate->createHardLink(wrapped_src_path, wrapped_dst_path); + auto tx = createEncryptedTransaction(); + tx->createHardLink(src_path, dst_path); + tx->commit(); } void truncateFile(const String & path, size_t size) override; @@ -280,19 +294,21 @@ public: SyncGuardPtr getDirectorySyncGuard(const String & path) const override; + std::shared_ptr createEncryptedTransaction() const + { + auto delegate_transaction = delegate->createTransaction(); + return std::make_shared(delegate_transaction, disk_path, *current_settings.get(), delegate.get()); + } + DiskTransactionPtr createTransaction() override { - /// Need to overwrite explicetly because this disk change - /// a lot of "delegate" methods. - if (use_fake_transaction) { return std::make_shared(*this); } else { - auto delegate_transaction = delegate->createTransaction(); - return std::make_shared(delegate_transaction, disk_path, *current_settings.get(), delegate.get()); + return createEncryptedTransaction(); } } diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index 7174e98f256..0c927a027cd 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -110,14 +110,5 @@ std::unique_ptr DiskEncryptedTransaction::writeFile( return std::make_unique(buf_size, std::move(buffer), key, header, old_file_size); } -void DiskEncryptedTransaction::writeFileUsingCustomWriteObject( - const String &, - WriteMode, - std::function & object_attributes)>) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `writeFileUsingCustomWriteObject()` is not implemented"); -} - - } diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h index d68e0c42807..0ef8d946f7f 100644 --- a/src/Disks/DiskEncryptedTransaction.h +++ b/src/Disks/DiskEncryptedTransaction.h @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -23,7 +22,7 @@ struct DiskEncryptedSettings class DiskEncryptedTransaction : public IDiskTransaction { public: - explicit DiskEncryptedTransaction(DiskTransactionPtr delegate_transaction_, const std::string & disk_path_, DiskEncryptedSettings current_settings_, IDisk * delegate_disk_) + DiskEncryptedTransaction(DiskTransactionPtr delegate_transaction_, const std::string & disk_path_, DiskEncryptedSettings current_settings_, IDisk * delegate_disk_) : delegate_transaction(delegate_transaction_) , disk_path(disk_path_) , current_settings(current_settings_) @@ -32,9 +31,9 @@ public: /// Tries to commit all accumulated operations simultaneously. /// If something fails rollback and throw exception. - void commit(const TransactionCommitOptionsVariant & options = NoCommitOptions{}) override // NOLINT + void commit() override // NOLINT { - delegate_transaction->commit(options); + delegate_transaction->commit(); } void undo() override @@ -42,11 +41,6 @@ public: delegate_transaction->undo(); } - TransactionCommitOutcomeVariant tryCommit(const TransactionCommitOptionsVariant & options) override - { - return delegate_transaction->tryCommit(options); - } - ~DiskEncryptedTransaction() override = default; /// Create directory. @@ -115,13 +109,6 @@ public: const WriteSettings & settings = {}, bool autocommit = true) override; - /// Write a file using a custom function to write an object to the disk's object storage. - void writeFileUsingCustomWriteObject( - const String & path, - WriteMode mode, - std::function & object_attributes)> - custom_write_object_function) override; - /// Remove file. Throws exception if file doesn't exists or it's a directory. void removeFile(const std::string & path) override { @@ -224,6 +211,23 @@ public: delegate_transaction->createHardLink(wrapped_src_path, wrapped_dst_path); } + void writeFileUsingBlobWritingFunction(const String & path, WriteMode mode, WriteBlobFunction && write_blob_function) override + { + auto wrapped_path = wrappedPath(path); + delegate_transaction->writeFileUsingBlobWritingFunction(wrapped_path, mode, std::move(write_blob_function)); + } + + std::unique_ptr writeEncryptedFile( + const String & path, + size_t buf_size, + WriteMode mode, + const WriteSettings & settings) const + { + auto wrapped_path = wrappedPath(path); + return delegate_transaction->writeFile(wrapped_path, buf_size, mode, settings); + } + + private: String wrappedPath(const String & path) const { From 4a4246a8cc87c29e88bceb02baa5ffcfe15864c8 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 24 May 2023 19:39:53 +0200 Subject: [PATCH 18/46] Dedup --- src/Disks/DiskEncrypted.h | 5 +---- src/Disks/DiskEncryptedTransaction.h | 16 ++++++++++++---- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Disks/DiskEncrypted.h b/src/Disks/DiskEncrypted.h index 73bbe3fd74a..69d051a9537 100644 --- a/src/Disks/DiskEncrypted.h +++ b/src/Disks/DiskEncrypted.h @@ -347,10 +347,7 @@ public: private: String wrappedPath(const String & path) const { - // if path starts_with disk_path -> got already wrapped path - if (!disk_path.empty() && path.starts_with(disk_path)) - return path; - return disk_path + path; + return DiskEncryptedTransaction::wrappedPath(disk_path, path); } DiskPtr delegate; diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h index 0ef8d946f7f..9f60ed36c4e 100644 --- a/src/Disks/DiskEncryptedTransaction.h +++ b/src/Disks/DiskEncryptedTransaction.h @@ -19,9 +19,19 @@ struct DiskEncryptedSettings FileEncryption::Algorithm current_algorithm; }; + + class DiskEncryptedTransaction : public IDiskTransaction { public: + static String wrappedPath(const String disk_path, const String & path) + { + // if path starts_with disk_path -> got already wrapped path + if (!disk_path.empty() && path.starts_with(disk_path)) + return path; + return disk_path + path; + } + DiskEncryptedTransaction(DiskTransactionPtr delegate_transaction_, const std::string & disk_path_, DiskEncryptedSettings current_settings_, IDisk * delegate_disk_) : delegate_transaction(delegate_transaction_) , disk_path(disk_path_) @@ -229,12 +239,10 @@ public: private: + String wrappedPath(const String & path) const { - // if path starts_with disk_path -> got already wrapped path - if (!disk_path.empty() && path.starts_with(disk_path)) - return path; - return disk_path + path; + return wrappedPath(disk_path, path); } DiskTransactionPtr delegate_transaction; From 1c627fbcab71e652975c3c2338db67d63ecec149 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 24 May 2023 20:01:36 +0200 Subject: [PATCH 19/46] Fxi --- src/Disks/DiskEncryptedTransaction.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index 0c927a027cd..deafe88e43d 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -13,7 +13,6 @@ namespace DB namespace ErrorCodes { extern const int DATA_ENCRYPTION_ERROR; - extern const int NOT_IMPLEMENTED; } From 3e69648268957cd9f2e31183757bc9584ee23b33 Mon Sep 17 00:00:00 2001 From: Alexander Sapin Date: Wed, 24 May 2023 20:02:03 +0200 Subject: [PATCH 20/46] Fxi moar --- src/Disks/DiskEncryptedTransaction.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h index 9f60ed36c4e..197df56c2a2 100644 --- a/src/Disks/DiskEncryptedTransaction.h +++ b/src/Disks/DiskEncryptedTransaction.h @@ -20,7 +20,6 @@ struct DiskEncryptedSettings }; - class DiskEncryptedTransaction : public IDiskTransaction { public: From 9582d9e892d5bfd7b0c6b46b11f758c15f5ab0cc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 09:57:32 +0200 Subject: [PATCH 21/46] Fix UB for INTO OUTFILE extensions (APPEND / AND STDOUT) MSAn report: ==38627==WARNING: MemorySanitizer: use-of-uninitialized-value 0 0x555599f5e114 in std::__1::__unique_if::__unique_single std::__1::make_unique[abi:v15000]<> build_docker/./contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:714:32 1 0x555599f5e114 in DB::ClientBase::initOutputFormat() build_docker/./src/Client/ClientBase.cpp:604:21 2 0x555599f590a8 in DB::ClientBase::onData() build_docker/./src/Client/ClientBase.cpp:446:5 3 0x555599f6f36e in DB::ClientBase::receiveAndProcessPacket() build_docker/./src/Client/ClientBase.cpp:1019:17 4 0x555599f6e863 in DB::ClientBase::receiveResult() build_docker/./src/Client/ClientBase.cpp:987:18 5 0x555599f6c05b in DB::ClientBase::processOrdinaryQuery() build_docker/./src/Client/ClientBase.cpp:905:13 6 0x555599f67e05 in DB::ClientBase::processParsedSingleQuery() build_docker/./src/Client/ClientBase.cpp:1711:13 7 0x555599f86fb6 in DB::ClientBase::executeMultiQuery(std::__1::basic_string, std::__1::allocator > const&) build_docker/./src/Client/ClientBase.cpp:1975:21 Uninitialized value was created by a heap allocation 8 0x55559bd3e038 in DB::ParserExplainQuery::parseImpl(DB::IParser::Pos&, std::__1::shared_ptr&, DB::Expected&) build_docker/./src/Parsers/ParserExplainQuery.cpp:53:26 9 0x55559bce31f4 in DB::IParserBase::parse(DB::IParser::Pos&, std::__1::shared_ptr&, DB::Expected&)::$_0::operator()() const build_docker/./src/Parsers/IParserBase.cpp:13:20 .. 21 0x55559be13b5c in DB::parseQueryAndMovePosition(DB::IParser&, char const*&, char const*, std::__1::basic_string, std::__1::allocator > const&, bool, unsigned long, unsigned long) build_docker/./src/Parsers/parseQuery.cpp:357:18 22 0x555599f5673a in DB::ClientBase::parseQuery(char const*&, char const*, bool) const build_docker/./src/Client/ClientBase.cpp:362:15 23 0x555599f84a4f in DB::ClientBase::analyzeMultiQueryText() build_docker/./src/Client/ClientBase.cpp:1821:24 24 0x555599f867b3 in DB::ClientBase::executeMultiQuery(std::__1::basic_string, std::__1::allocator > const&) build_docker/./src/Client/ClientBase.cpp:1910:22 25 0x555599f8a2fd in DB::ClientBase::processQueryText(std::__1::basic_string, std::__1::allocator > const&) build_docker/./src/Client/ClientBase.cpp:2120:12 26 0x555599f94aee in DB::ClientBase::runNonInteractive() build_docker/./src/Client/ClientBase.cpp:2403:9 Signed-off-by: Azat Khuzhin --- src/Parsers/ASTQueryWithOutput.h | 4 ++-- .../02767_into_outfile_extensions_msan.reference | 2 ++ .../0_stateless/02767_into_outfile_extensions_msan.sh | 11 +++++++++++ 3 files changed, 15 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02767_into_outfile_extensions_msan.reference create mode 100755 tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh diff --git a/src/Parsers/ASTQueryWithOutput.h b/src/Parsers/ASTQueryWithOutput.h index 09f08772468..7db021405e7 100644 --- a/src/Parsers/ASTQueryWithOutput.h +++ b/src/Parsers/ASTQueryWithOutput.h @@ -15,8 +15,8 @@ class ASTQueryWithOutput : public IAST { public: ASTPtr out_file; - bool is_into_outfile_with_stdout; - bool is_outfile_append; + bool is_into_outfile_with_stdout = false; + bool is_outfile_append = false; ASTPtr format; ASTPtr settings_ast; ASTPtr compression; diff --git a/tests/queries/0_stateless/02767_into_outfile_extensions_msan.reference b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.reference new file mode 100644 index 00000000000..0c8b489c2fd --- /dev/null +++ b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.reference @@ -0,0 +1,2 @@ +Expression ((Projection + Before ORDER BY)) + ReadFromStorage (SystemNumbers) diff --git a/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh new file mode 100755 index 00000000000..d1e7312b150 --- /dev/null +++ b/tests/queries/0_stateless/02767_into_outfile_extensions_msan.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +out="explain1.$CLICKHOUSE_TEST_UNIQUE_NAME.out" +# only EXPLAIN triggers the problem under MSan +$CLICKHOUSE_CLIENT -q "explain select * from numbers(1) into outfile '$out'" +cat "$out" +rm -f "$out" From c053d75741aead3764e3be9d955f496dc47749d5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 10:09:33 +0200 Subject: [PATCH 22/46] Fix formatting of INTO OUTFILE extensions (APPEND / AND STDOUT) Signed-off-by: Azat Khuzhin --- src/Parsers/ASTQueryWithOutput.cpp | 7 +++++++ ...8_into_outfile_extensions_format.reference | 20 +++++++++++++++++++ .../02768_into_outfile_extensions_format.sh | 12 +++++++++++ 3 files changed, 39 insertions(+) create mode 100644 tests/queries/0_stateless/02768_into_outfile_extensions_format.reference create mode 100755 tests/queries/0_stateless/02768_into_outfile_extensions_format.sh diff --git a/src/Parsers/ASTQueryWithOutput.cpp b/src/Parsers/ASTQueryWithOutput.cpp index 9f771ab92e3..5f717715a69 100644 --- a/src/Parsers/ASTQueryWithOutput.cpp +++ b/src/Parsers/ASTQueryWithOutput.cpp @@ -35,6 +35,13 @@ void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & stat { s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "INTO OUTFILE " << (s.hilite ? hilite_none : ""); out_file->formatImpl(s, state, frame); + + s.ostr << (s.hilite ? hilite_keyword : ""); + if (is_outfile_append) + s.ostr << " APPEND"; + if (is_into_outfile_with_stdout) + s.ostr << " AND STDOUT"; + s.ostr << (s.hilite ? hilite_none : ""); } if (format) diff --git a/tests/queries/0_stateless/02768_into_outfile_extensions_format.reference b/tests/queries/0_stateless/02768_into_outfile_extensions_format.reference new file mode 100644 index 00000000000..4ebc1da8865 --- /dev/null +++ b/tests/queries/0_stateless/02768_into_outfile_extensions_format.reference @@ -0,0 +1,20 @@ +SELECT * +FROM numbers(1) +INTO OUTFILE '/dev/null' +; + +SELECT * +FROM numbers(1) +INTO OUTFILE '/dev/null' AND STDOUT +; + +SELECT * +FROM numbers(1) +INTO OUTFILE '/dev/null' APPEND +; + +SELECT * +FROM numbers(1) +INTO OUTFILE '/dev/null' APPEND AND STDOUT +; + diff --git a/tests/queries/0_stateless/02768_into_outfile_extensions_format.sh b/tests/queries/0_stateless/02768_into_outfile_extensions_format.sh new file mode 100755 index 00000000000..756488076f9 --- /dev/null +++ b/tests/queries/0_stateless/02768_into_outfile_extensions_format.sh @@ -0,0 +1,12 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo " +select * from numbers(1) into outfile '/dev/null'; +select * from numbers(1) into outfile '/dev/null' and stdout; +select * from numbers(1) into outfile '/dev/null' append; +select * from numbers(1) into outfile '/dev/null' append and stdout; +" | clickhouse-format -n From b30cfe55030fd819d0dfa1da62ed74ab29ea9e63 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 10:14:47 +0200 Subject: [PATCH 23/46] Fix UB in ASTWatchQuery for is_watch_events Signed-off-by: Azat Khuzhin --- src/Parsers/ASTWatchQuery.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTWatchQuery.h b/src/Parsers/ASTWatchQuery.h index 29dadd71675..156fe6828bc 100644 --- a/src/Parsers/ASTWatchQuery.h +++ b/src/Parsers/ASTWatchQuery.h @@ -23,7 +23,7 @@ class ASTWatchQuery : public ASTQueryWithTableAndOutput public: ASTPtr limit_length; - bool is_watch_events; + bool is_watch_events = false; ASTWatchQuery() = default; String getID(char) const override { return "WatchQuery_" + getDatabase() + "_" + getTable(); } From b680697cced2d9a5dabe87219db8b5e75d4867e2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 May 2023 10:16:54 +0200 Subject: [PATCH 24/46] Initialize POD members of ASTs to make it less error-prone The cost of initializing members is insignificant in compare to parsing, while the cost of the error is high. Signed-off-by: Azat Khuzhin --- src/Parsers/ASTColumnDeclaration.h | 2 +- src/Parsers/ASTDictionaryAttributeDeclaration.h | 8 ++++---- src/Parsers/ASTOrderByElement.h | 8 ++++---- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Parsers/ASTColumnDeclaration.h b/src/Parsers/ASTColumnDeclaration.h index 2008e4f99d1..45814551db8 100644 --- a/src/Parsers/ASTColumnDeclaration.h +++ b/src/Parsers/ASTColumnDeclaration.h @@ -16,7 +16,7 @@ public: std::optional null_modifier; String default_specifier; ASTPtr default_expression; - bool ephemeral_default; + bool ephemeral_default = false; ASTPtr comment; ASTPtr codec; ASTPtr ttl; diff --git a/src/Parsers/ASTDictionaryAttributeDeclaration.h b/src/Parsers/ASTDictionaryAttributeDeclaration.h index b6572e89d16..52103650684 100644 --- a/src/Parsers/ASTDictionaryAttributeDeclaration.h +++ b/src/Parsers/ASTDictionaryAttributeDeclaration.h @@ -19,13 +19,13 @@ public: /// Attribute expression ASTPtr expression; /// Is attribute mirrored to the parent identifier - bool hierarchical; + bool hierarchical = false; /// Is hierarchical attribute bidirectional - bool bidirectional; + bool bidirectional = false; /// Flag that shows whether the id->attribute image is injective - bool injective; + bool injective = false; /// MongoDB object ID - bool is_object_id; + bool is_object_id = false; String getID(char delim) const override { return "DictionaryAttributeDeclaration" + (delim + name); } diff --git a/src/Parsers/ASTOrderByElement.h b/src/Parsers/ASTOrderByElement.h index 4d07405c17a..468d2161dff 100644 --- a/src/Parsers/ASTOrderByElement.h +++ b/src/Parsers/ASTOrderByElement.h @@ -11,14 +11,14 @@ namespace DB class ASTOrderByElement : public IAST { public: - int direction; /// 1 for ASC, -1 for DESC - int nulls_direction; /// Same as direction for NULLS LAST, opposite for NULLS FIRST. - bool nulls_direction_was_explicitly_specified; + int direction = 0; /// 1 for ASC, -1 for DESC + int nulls_direction = 0; /// Same as direction for NULLS LAST, opposite for NULLS FIRST. + bool nulls_direction_was_explicitly_specified = false; /** Collation for locale-specific string comparison. If empty, then sorting done by bytes. */ ASTPtr collation; - bool with_fill; + bool with_fill = false; ASTPtr fill_from; ASTPtr fill_to; ASTPtr fill_step; From b2c9611da66254fc10ad4bc8fe30d7929b3c30a3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 May 2023 12:01:24 +0200 Subject: [PATCH 25/46] Fix build --- src/Disks/DiskEncryptedTransaction.cpp | 5 +++++ src/Disks/DiskEncryptedTransaction.h | 6 ++++++ src/Disks/tests/gtest_disk_encrypted.cpp | 2 +- 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index deafe88e43d..d7448b986bb 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -1,4 +1,7 @@ #include + + +#if USE_SSL #include #include #include @@ -111,3 +114,5 @@ std::unique_ptr DiskEncryptedTransaction::writeFile( } } + +#endif diff --git a/src/Disks/DiskEncryptedTransaction.h b/src/Disks/DiskEncryptedTransaction.h index 197df56c2a2..bae3f2c728c 100644 --- a/src/Disks/DiskEncryptedTransaction.h +++ b/src/Disks/DiskEncryptedTransaction.h @@ -1,5 +1,9 @@ #pragma once +#include "config.h" + +#if USE_SSL + #include #include #include @@ -251,3 +255,5 @@ private: }; } + +#endif diff --git a/src/Disks/tests/gtest_disk_encrypted.cpp b/src/Disks/tests/gtest_disk_encrypted.cpp index 32db5f5c1d9..80a10e8680b 100644 --- a/src/Disks/tests/gtest_disk_encrypted.cpp +++ b/src/Disks/tests/gtest_disk_encrypted.cpp @@ -40,7 +40,7 @@ protected: settings->keys[0] = key; settings->current_key_id = 0; settings->disk_path = path; - encrypted_disk = std::make_shared("encrypted_disk", std::move(settings)); + encrypted_disk = std::make_shared("encrypted_disk", std::move(settings), true); } String getFileNames() From 5b76ab4e038c1b9a9fb395d787205b298ecac9d1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 May 2023 13:02:40 +0200 Subject: [PATCH 26/46] Fix build --- src/Disks/DiskEncryptedTransaction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index d7448b986bb..7534abe59a8 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -77,7 +77,7 @@ void DiskEncryptedTransaction::copyFile(const std::string & from_file_path, cons delegate_transaction->copyFile(wrapped_from_path, wrapped_to_path); } -std::unique_ptr DiskEncryptedTransaction::writeFile( +std::unique_ptr DiskEncryptedTransaction::writeFile( // NOLINT const std::string & path, size_t buf_size, WriteMode mode, From 32cb9931b61df452ade3885fab8006582375640b Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 25 May 2023 13:47:56 +0200 Subject: [PATCH 27/46] Update src/Common/AsyncLoader.h Co-authored-by: Antonio Andelic --- src/Common/AsyncLoader.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index 7c6c934d03e..c3d11b26336 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -59,7 +59,7 @@ public: // Returns pool in which the job is executing (was executed). May differ from initial pool and from current pool. // Value is only valid (and constant) after execution started. - size_t execution_pool() const; + size_t executionPool() const; // Returns current pool of the job. May differ from initial and execution pool. // This value is intended for creating new jobs during this job execution. From 78c89da8bbc2cc75882a7883cd2e32315a4dfc32 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 25 May 2023 13:48:03 +0200 Subject: [PATCH 28/46] Update src/Common/AsyncLoader.cpp Co-authored-by: Antonio Andelic --- src/Common/AsyncLoader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 0f75f17465b..6e558f36b7d 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -42,7 +42,7 @@ std::exception_ptr LoadJob::exception() const return load_exception; } -size_t LoadJob::execution_pool() const +size_t LoadJob::executionPool() const { return execution_pool_id; } From 8804dfd4b052bf768d2747fec814c1956f7f40f4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 11:55:23 +0000 Subject: [PATCH 29/46] Fix resizing --- src/Functions/space.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index 9d825a8b294..7d55d704038 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -27,9 +27,9 @@ private: static constexpr auto space = ' '; /// Safety threshold against DoS. - static inline void checkRepeatTime(UInt64 repeat_time) + static inline void checkRepeatTime(size_t repeat_time) { - static constexpr UInt64 max_repeat_times = 1'000'000; + static constexpr auto max_repeat_times = 1'000'000uz; if (repeat_time > max_repeat_times) throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "Too many times to repeat ({}), maximum is: {}", repeat_time, max_repeat_times); } @@ -114,8 +114,8 @@ public: checkRepeatTime(times); - if (res_chars.size() + times + 1 >= res_chars.capacity()) - res_chars.resize(2 * res_chars.capacity()); + if (pos + times + 1 > res_chars.size()) + res_chars.resize(std::max(2 * res_chars.size(), pos + times + 1)); memset(res_chars.begin() + pos, space, times); pos += times; From e3ce2f834acd13e844b49c3bb7cea7aafe8b63dc Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 25 May 2023 12:35:00 +0000 Subject: [PATCH 30/46] fix style --- src/Common/tests/gtest_async_loader.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 2e4c42d6826..6a7d2d727a9 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -704,7 +704,7 @@ TEST(AsyncLoader, DynamicPriorities) { { std::unique_lock lock{schedule_mutex}; - schedule += fmt::format("{}{}", self->name, self->execution_pool()); + schedule += fmt::format("{}{}", self->name, self->executionPool()); } if (prioritize && self->name == "C") @@ -905,7 +905,7 @@ TEST(AsyncLoader, DynamicPools) auto job_func = [&] (const LoadJobPtr & self) { - auto pool_id = self->execution_pool(); + auto pool_id = self->executionPool(); executing[pool_id]++; if (executing[pool_id] > max_threads[0]) boosted = true; From b8d3e495e51849588ceca6a6976c3931bdd3ef2d Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 25 May 2023 12:42:19 +0000 Subject: [PATCH 31/46] add `pool_id` out-of-bound checks --- src/Common/AsyncLoader.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/AsyncLoader.cpp b/src/Common/AsyncLoader.cpp index 6e558f36b7d..b5612517cd6 100644 --- a/src/Common/AsyncLoader.cpp +++ b/src/Common/AsyncLoader.cpp @@ -272,6 +272,7 @@ void AsyncLoader::scheduleImpl(const LoadJobSet & input_jobs) // Pass 2. Schedule all incoming jobs for (const auto & job : jobs) { + chassert(job->pool() < pools.size()); NOEXCEPT_SCOPE({ ALLOW_ALLOCATIONS_IN_SCOPE; scheduled_jobs.try_emplace(job); @@ -359,6 +360,7 @@ void AsyncLoader::prioritize(const LoadJobPtr & job, size_t new_pool) { if (!job) return; + chassert(new_pool < pools.size()); DENY_ALLOCATIONS_IN_SCOPE; std::unique_lock lock{mutex}; prioritize(job, new_pool, lock); From e94b0c8e5e30451d2ab29fd3365bc8285bbec40e Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 May 2023 16:38:19 +0200 Subject: [PATCH 32/46] Fix bug --- src/Disks/DiskEncryptedTransaction.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index 7534abe59a8..7797e11bf39 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -88,9 +88,10 @@ std::unique_ptr DiskEncryptedTransaction::writeFile( // FileEncryption::Header header; String key; UInt64 old_file_size = 0; - if (mode == WriteMode::Append && delegate_disk->exists(path)) + if (mode == WriteMode::Append && delegate_disk->exists(wrapped_path)) { - old_file_size = delegate_disk->getFileSize(path); + size_t size = delegate_disk->getFileSize(wrapped_path); + old_file_size = size > FileEncryption::Header::kSize ? (size - FileEncryption::Header::kSize) : 0; if (old_file_size) { /// Append mode: we continue to use the same header. From e4c1e2f2326c6f52cb42c1b1b7eaa1a333784a61 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 May 2023 17:37:09 +0200 Subject: [PATCH 33/46] Fix build while it's not failing locally --- src/Disks/DiskEncryptedTransaction.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Disks/DiskEncryptedTransaction.cpp b/src/Disks/DiskEncryptedTransaction.cpp index 7797e11bf39..4a613374ccf 100644 --- a/src/Disks/DiskEncryptedTransaction.cpp +++ b/src/Disks/DiskEncryptedTransaction.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB { From eca08438f42ed43b1e393b2f1dd4b4b1aefefd5e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 17:05:18 +0000 Subject: [PATCH 34/46] Fix macos build --- src/Functions/space.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/space.cpp b/src/Functions/space.cpp index 7d55d704038..009bc20e065 100644 --- a/src/Functions/space.cpp +++ b/src/Functions/space.cpp @@ -115,7 +115,7 @@ public: checkRepeatTime(times); if (pos + times + 1 > res_chars.size()) - res_chars.resize(std::max(2 * res_chars.size(), pos + times + 1)); + res_chars.resize(std::max(2 * res_chars.size(), static_cast(pos + times + 1))); memset(res_chars.begin() + pos, space, times); pos += times; From 231d52d0e6d3fc16511522228e4241d95005d15f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 25 May 2023 18:27:31 +0000 Subject: [PATCH 35/46] Document system.build_options --- .../operations/system-tables/build_options.md | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) create mode 100644 docs/en/operations/system-tables/build_options.md diff --git a/docs/en/operations/system-tables/build_options.md b/docs/en/operations/system-tables/build_options.md new file mode 100644 index 00000000000..5225d0ff99d --- /dev/null +++ b/docs/en/operations/system-tables/build_options.md @@ -0,0 +1,27 @@ +--- +slug: /en/operations/system-tables/build_options +--- +# build_options + +Contains information about the ClickHouse server's build options. + +Columns: + +- `name` (String) — Name of the build option, e.g. `USE_ODBC` +- `value` (String) — Value of the build option, e.g. `1` + +**Example** + +``` sql +SELECT * FROM system.build_options LIMIT 5 +``` + +``` text +┌─name─────────────┬─value─┐ +│ USE_BROTLI │ 1 │ +│ USE_BZIP2 │ 1 │ +│ USE_CAPNP │ 1 │ +│ USE_CASSANDRA │ 1 │ +│ USE_DATASKETCHES │ 1 │ +└──────────────────┴───────┘ +``` From 6b98e028576ac8564336fab1a9021482a5129739 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 25 May 2023 19:21:32 -0300 Subject: [PATCH 36/46] Update settings.md --- .../settings.md | 26 +++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index e3ca04f5b9b..99630043095 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -917,9 +917,9 @@ We recommend using this option in macOS since the `getrlimit()` function returns Restriction on deleting tables. -If the size of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `max_table_size_to_drop` (in bytes), you can’t delete it using a DROP query. +If the size of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `max_table_size_to_drop` (in bytes), you can’t delete it using a [DROP](sql-reference/statements/drop.md) query or [TRUNCATE](../../sql-reference/statements/truncate.md) query. -If you still need to delete the table without restarting the ClickHouse server, create the `/flags/force_drop_table` file and run the DROP query. +This setting does not require a restart of the Clickhouse server to apply. Another way to disable the restriction is to create the `/flags/force_drop_table` file. Default value: 50 GB. @@ -931,6 +931,28 @@ The value 0 means that you can delete all tables without any restrictions. 0 ``` +## max_partition_size_to_drop {#max-partition-size-to-drop} + +Restriction on dropping partitions. + +If the size of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `max_partition_size_to_drop` (in bytes), you can’t drop a partition using a [DROP PARTITION](../../sql-reference/statements/alter/partition.md#drop-partitionpart) query. + +This setting does not require a restart of the Clickhouse server to apply. Another way to disable the restriction is to create the `/flags/force_drop_table` file. + +Default value: 50 GB. + +The value 0 means that you can drop partitions without any restrictions. + +:::note +This limitation does not restrict drop table and truncate table, see [max_table_size_to_drop](#max_table_size_to_drop) +::: + +**Example** + +``` xml +0 +``` + ## max_thread_pool_size {#max-thread-pool-size} ClickHouse uses threads from the Global Thread pool to process queries. If there is no idle thread to process a query, then a new thread is created in the pool. `max_thread_pool_size` limits the maximum number of threads in the pool. From b3a96de533c27283540d0ecad2054a35f86c9357 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 26 May 2023 00:04:24 +0100 Subject: [PATCH 37/46] Pure parallel replicas: JOIN support (#49544) --- src/Interpreters/ExpressionAnalyzer.cpp | 9 + src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 14 +- src/Interpreters/InterpreterSelectQuery.cpp | 31 +-- src/Interpreters/interpretSubquery.cpp | 2 - .../02535_max_parallel_replicas_custom_key.sh | 2 +- ...708_parallel_replicas_not_found_column.sql | 1 + ..._parallel_replicas_join_subquery.reference | 44 +++++ .../02731_parallel_replicas_join_subquery.sql | 182 ++++++++++++++++++ ...l_replicas_bug_chunkinfo_not_set.reference | 0 ...arallel_replicas_bug_chunkinfo_not_set.sql | 43 +++++ ...764_parallel_replicas_plain_merge_tree.sql | 2 +- 12 files changed, 312 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference create mode 100644 tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql create mode 100644 tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.reference create mode 100644 tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 96a86df7ffd..c7c66f6f414 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -969,6 +969,15 @@ const ASTSelectQuery * ExpressionAnalyzer::getSelectQuery() const return select_query; } +bool ExpressionAnalyzer::isRemoteStorage() const +{ + const Settings & csettings = getContext()->getSettingsRef(); + // Consider any storage used in parallel replicas as remote, so the query is executed in multiple servers + const bool enable_parallel_processing_of_joins + = csettings.max_parallel_replicas > 1 && csettings.allow_experimental_parallel_reading_from_replicas > 0; + return syntax->is_remote_storage || enable_parallel_processing_of_joins; +} + const ASTSelectQuery * SelectQueryExpressionAnalyzer::getAggregatingQuery() const { if (!has_aggregation) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 1b6e8e24091..00cd353aa66 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -201,7 +201,7 @@ protected: const ASTSelectQuery * getSelectQuery() const; - bool isRemoteStorage() const { return syntax->is_remote_storage; } + bool isRemoteStorage() const; NamesAndTypesList getColumnsAfterArrayJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns); NamesAndTypesList analyzeJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns); diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index b105cae31c6..08862032007 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -205,10 +205,19 @@ public: } private: + static bool shouldBeExecutedGlobally(const Data & data) + { + const Settings & settings = data.getContext()->getSettingsRef(); + /// For parallel replicas we reinterpret JOIN as GLOBAL JOIN as a way to broadcast data + const bool enable_parallel_processing_of_joins = data.getContext()->canUseParallelReplicasOnInitiator(); + return settings.prefer_global_in_and_join || enable_parallel_processing_of_joins; + } + + /// GLOBAL IN static void visit(ASTFunction & func, ASTPtr &, Data & data) { - if ((data.getContext()->getSettingsRef().prefer_global_in_and_join + if ((shouldBeExecutedGlobally(data) && (func.name == "in" || func.name == "notIn" || func.name == "nullIn" || func.name == "notNullIn")) || func.name == "globalIn" || func.name == "globalNotIn" || func.name == "globalNullIn" || func.name == "globalNotNullIn") { @@ -238,8 +247,7 @@ private: static void visit(ASTTablesInSelectQueryElement & table_elem, ASTPtr &, Data & data) { if (table_elem.table_join - && (table_elem.table_join->as().locality == JoinLocality::Global - || data.getContext()->getSettingsRef().prefer_global_in_and_join)) + && (table_elem.table_join->as().locality == JoinLocality::Global || shouldBeExecutedGlobally(data))) { data.addExternalStorage(table_elem.table_expression, true); data.has_global_subqueries = true; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ebaf88ea5d5..d4ca2e405e6 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -458,19 +458,11 @@ InterpreterSelectQuery::InterpreterSelectQuery( } } - /// Check support for JOINs for parallel replicas - if (joined_tables.tablesCount() > 1 && (!settings.parallel_replicas_custom_key.value.empty() || settings.allow_experimental_parallel_reading_from_replicas > 0)) + /// Check support for JOIN for parallel replicas with custom key + if (joined_tables.tablesCount() > 1 && !settings.parallel_replicas_custom_key.value.empty()) { - if (settings.allow_experimental_parallel_reading_from_replicas == 1) - { - LOG_WARNING(log, "JOINs are not supported with parallel replicas. Query will be executed without using them."); - context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); - context->setSetting("parallel_replicas_custom_key", String{""}); - } - else if (settings.allow_experimental_parallel_reading_from_replicas == 2) - { - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "JOINs are not supported with parallel replicas"); - } + LOG_WARNING(log, "JOINs are not supported with parallel_replicas_custom_key. Query will be executed without using them."); + context->setSetting("parallel_replicas_custom_key", String{""}); } /// Check support for FINAL for parallel replicas @@ -489,6 +481,21 @@ InterpreterSelectQuery::InterpreterSelectQuery( } } + /// Check support for parallel replicas for non-replicated storage (plain MergeTree) + bool is_plain_merge_tree = storage && storage->isMergeTree() && !storage->supportsReplication(); + if (is_plain_merge_tree && settings.allow_experimental_parallel_reading_from_replicas > 0 && !settings.parallel_replicas_for_non_replicated_merge_tree) + { + if (settings.allow_experimental_parallel_reading_from_replicas == 1) + { + LOG_WARNING(log, "To use parallel replicas with plain MergeTree tables please enable setting `parallel_replicas_for_non_replicated_merge_tree`. For now query will be executed without using them."); + context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + } + else if (settings.allow_experimental_parallel_reading_from_replicas == 2) + { + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "To use parallel replicas with plain MergeTree tables please enable setting `parallel_replicas_for_non_replicated_merge_tree`"); + } + } + /// Rewrite JOINs if (!has_input && joined_tables.tablesCount() > 1) { diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index 550fa2912ba..5f00be07fa5 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -112,8 +112,6 @@ std::shared_ptr interpretSubquery( subquery_options.removeDuplicates(); } - /// We don't want to execute reading for subqueries in parallel - subquery_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); return std::make_shared(query, subquery_context, subquery_options, required_source_columns); } diff --git a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh index 50e89cca4c9..9850406eb3a 100755 --- a/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh +++ b/tests/queries/0_stateless/02535_max_parallel_replicas_custom_key.sh @@ -41,6 +41,6 @@ run_count_with_custom_key "y" run_count_with_custom_key "cityHash64(y)" run_count_with_custom_key "cityHash64(y) + 1" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with parallel replicas" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM cluster(test_cluster_one_shard_three_replicas_localhost, currentDatabase(), 02535_custom_key) as t1 JOIN 02535_custom_key USING y" --parallel_replicas_custom_key="y" --send_logs_level="trace" 2>&1 | grep -Fac "JOINs are not supported with" $CLICKHOUSE_CLIENT --query="DROP TABLE 02535_custom_key" diff --git a/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql b/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql index 8900025502c..ff7b53ce01f 100644 --- a/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql +++ b/tests/queries/0_stateless/02708_parallel_replicas_not_found_column.sql @@ -1,3 +1,4 @@ CREATE TABLE IF NOT EXISTS t_02708(x DateTime) ENGINE = MergeTree ORDER BY tuple(); +SET send_logs_level='error'; SELECT count() FROM t_02708 SETTINGS allow_experimental_parallel_reading_from_replicas=1; DROP TABLE t_02708; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference new file mode 100644 index 00000000000..df606679523 --- /dev/null +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference @@ -0,0 +1,44 @@ +=============== INNER QUERY (NO PARALLEL) =============== +0 PJFiUe#J2O _s\' 14427935816175499794 +1 >T%O ,z< 17537932797009027240 +12 D[6,P #}Lmb[ ZzU 6394957109822140795 +18 $_N- 24422838680427462 +2 bX?}ix [ Ny]2 G 16242612901291874718 +20 VE] Y 15120036904703536841 +22 Ti~3)N)< A!( 3 18361093572663329113 +23 Sx>b:^UG XpedE)Q: 7433019734386307503 +29 2j&S)ba?XG QuQj 17163829389637435056 +3 UlI+1 14144472852965836438 +=============== INNER QUERY (PARALLEL) =============== +0 PJFiUe#J2O _s\' 14427935816175499794 +1 >T%O ,z< 17537932797009027240 +12 D[6,P #}Lmb[ ZzU 6394957109822140795 +18 $_N- 24422838680427462 +2 bX?}ix [ Ny]2 G 16242612901291874718 +20 VE] Y 15120036904703536841 +22 Ti~3)N)< A!( 3 18361093572663329113 +23 Sx>b:^UG XpedE)Q: 7433019734386307503 +29 2j&S)ba?XG QuQj 17163829389637435056 +3 UlI+1 14144472852965836438 +=============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE =============== +0 3 SELECT `key`, `value1`, `value2`, toUInt64(min(`time`)) AS `start_ts` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` ORDER BY `key` ASC, `value1` ASC, `value2` ASC LIMIT 10 +1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1; +=============== OUTER QUERY (NO PARALLEL) =============== +>T%O ,z< 10 +NQTpY# W\\Xx4 10 +PJFiUe#J2O _s\' 10 +U c 10 +UlI+1 10 +bX?}ix [ Ny]2 G 10 +tT%O ,z< 10 +NQTpY# W\\Xx4 10 +PJFiUe#J2O _s\' 10 +U c 10 +UlI+1 10 +bX?}ix [ Ny]2 G 10 +t toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` +0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_11888098645495698704_17868075224240210014` USING (`key`) GROUP BY `key`, `value1`, `value2` +1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1; diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql new file mode 100644 index 00000000000..29c20980c14 --- /dev/null +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql @@ -0,0 +1,182 @@ +-- Tags: zookeeper + +CREATE TABLE join_inner_table +( + id UUID, + key String, + number Int64, + value1 String, + value2 String, + time Int64 +) +ENGINE=ReplicatedMergeTree('/clickhouse/tables/{database}/join_inner_table', 'r1') +ORDER BY (id, number, key); + +INSERT INTO join_inner_table +SELECT + '833c9e22-c245-4eb5-8745-117a9a1f26b1'::UUID as id, + rowNumberInAllBlocks()::String as key, + * FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) +LIMIT 100; + +SET allow_experimental_analyzer = 0; +SET max_parallel_replicas = 3; +SET prefer_localhost_replica = 1; +SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; +SET use_hedged_requests = 0; +SET joined_subquery_requires_alias = 0; + +SELECT '=============== INNER QUERY (NO PARALLEL) ==============='; + +SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts +FROM join_inner_table + PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) +GROUP BY key, value1, value2 +ORDER BY key, value1, value2 +LIMIT 10; + +SELECT '=============== INNER QUERY (PARALLEL) ==============='; + +-- Parallel inner query alone +SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts +FROM join_inner_table +PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) +GROUP BY key, value1, value2 +ORDER BY key, value1, value2 +LIMIT 10 +SETTINGS allow_experimental_parallel_reading_from_replicas = 1; + +SELECT '=============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE ==============='; + +SYSTEM FLUSH LOGS; +-- There should be 4 queries. The main query as received by the initiator and the 3 equal queries sent to each replica +SELECT is_initial_query, count() as c, query, +FROM system.query_log +WHERE + event_date >= yesterday() + AND type = 'QueryFinish' + AND initial_query_id = + ( + SELECT query_id + FROM system.query_log + WHERE + current_database = currentDatabase() + AND event_date >= yesterday() + AND type = 'QueryFinish' + AND query LIKE '-- Parallel inner query alone%' + ) +GROUP BY is_initial_query, query +ORDER BY is_initial_query, c, query; + +---- Query with JOIN + +CREATE TABLE join_outer_table +( + id UUID, + key String, + otherValue1 String, + otherValue2 String, + time Int64 +) +ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/join_outer_table', 'r1') +ORDER BY (id, time, key); + +INSERT INTO join_outer_table +SELECT + '833c9e22-c245-4eb5-8745-117a9a1f26b1'::UUID as id, + (rowNumberInAllBlocks() % 10)::String as key, + * FROM generateRandom('otherValue1 String, otherValue2 String, time Int64', 1, 10, 2) +LIMIT 100; + + +SELECT '=============== OUTER QUERY (NO PARALLEL) ==============='; + +SELECT + value1, + value2, + avg(count) AS avg +FROM +( + SELECT + key, + value1, + value2, + count() AS count + FROM join_outer_table + INNER JOIN + ( + SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts + FROM join_inner_table + PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) + GROUP BY key, value1, value2 + ) USING (key) + GROUP BY key, value1, value2 +) +GROUP BY value1, value2 +ORDER BY value1, value2; + +SELECT '=============== OUTER QUERY (PARALLEL) ==============='; + +-- Parallel full query +SELECT + value1, + value2, + avg(count) AS avg +FROM + ( + SELECT + key, + value1, + value2, + count() AS count + FROM join_outer_table + INNER JOIN + ( + SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts + FROM join_inner_table + PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) + GROUP BY key, value1, value2 + ) USING (key) + GROUP BY key, value1, value2 + ) +GROUP BY value1, value2 +ORDER BY value1, value2 +SETTINGS allow_experimental_parallel_reading_from_replicas = 1; + +SYSTEM FLUSH LOGS; + +-- There should be 7 queries. The main query as received by the initiator, the 3 equal queries to execute the subquery +-- in the inner join and the 3 queries executing the whole query (but replacing the subquery with a temp table) +SELECT is_initial_query, count() as c, query, +FROM system.query_log +WHERE + event_date >= yesterday() + AND type = 'QueryFinish' + AND initial_query_id = + ( + SELECT query_id + FROM system.query_log + WHERE + current_database = currentDatabase() + AND event_date >= yesterday() + AND type = 'QueryFinish' + AND query LIKE '-- Parallel full query%' + ) +GROUP BY is_initial_query, query +ORDER BY is_initial_query, c, query; diff --git a/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.reference b/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql b/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql new file mode 100644 index 00000000000..2ea2cecc7b5 --- /dev/null +++ b/tests/queries/0_stateless/02751_parallel_replicas_bug_chunkinfo_not_set.sql @@ -0,0 +1,43 @@ +CREATE TABLE join_inner_table__fuzz_1 +( + `id` UUID, + `key` Nullable(Date), + `number` Int64, + `value1` LowCardinality(String), + `value2` LowCardinality(String), + `time` Int128 +) +ENGINE = MergeTree +ORDER BY (id, number, key) +SETTINGS allow_nullable_key = 1; + +INSERT INTO join_inner_table__fuzz_1 SELECT + CAST('833c9e22-c245-4eb5-8745-117a9a1f26b1', 'UUID') AS id, + CAST(rowNumberInAllBlocks(), 'String') AS key, + * +FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) +LIMIT 100; + +SET max_parallel_replicas = 3, prefer_localhost_replica = 1, use_hedged_requests = 0, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 1; + +-- SELECT query will write a Warning to the logs +SET send_logs_level='error'; + +SELECT + key, + value1, + value2, + toUInt64(min(time)) AS start_ts +FROM join_inner_table__fuzz_1 +PREWHERE (id = '833c9e22-c245-4eb5-8745-117a9a1f26b1') AND (number > toUInt64('1610517366120')) +GROUP BY + key, + value1, + value2 + WITH ROLLUP +ORDER BY + key ASC, + value1 ASC, + value2 ASC NULLS LAST +LIMIT 10 +FORMAT Null; diff --git a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql index ea8eb04bd07..aaf68dfd300 100644 --- a/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql +++ b/tests/queries/0_stateless/02764_parallel_replicas_plain_merge_tree.sql @@ -2,7 +2,7 @@ CREATE TABLE IF NOT EXISTS parallel_replicas_plain (x String) ENGINE=MergeTree() INSERT INTO parallel_replicas_plain SELECT toString(number) FROM numbers(10); SET max_parallel_replicas=3, allow_experimental_parallel_reading_from_replicas=1, use_hedged_requests=0, cluster_for_parallel_replicas='parallel_replicas'; - +SET send_logs_level='error'; SET parallel_replicas_for_non_replicated_merge_tree = 0; SELECT x FROM parallel_replicas_plain LIMIT 1 FORMAT Null; From 08051917c773da3ca10e51f70fb798bf2836eabb Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 25 May 2023 21:35:51 -0300 Subject: [PATCH 38/46] Update named-collections.md --- docs/en/operations/named-collections.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index 7db8f2b642b..a521a369721 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -167,9 +167,9 @@ user = 'myuser', password = 'mypass', host = '127.0.0.1', port = 3306, -database = 'test' -connection_pool_size = 8 -on_duplicate_clause = 1 +database = 'test', +connection_pool_size = 8, +on_duplicate_clause = 1, replace_query = 1 ``` From c6c4ded5e176e79bbc8aff81eebc87d6f754f986 Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Fri, 26 May 2023 10:12:34 +0200 Subject: [PATCH 39/46] Fixes for WITH FILL grouped by sorting prefix --- docs/en/sql-reference/statements/select/order-by.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index f1efd6c4718..712395a0357 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -544,10 +544,10 @@ Result: └─────┴──────────┴───────┘ ``` -##Filling grouped by sorting prefix +## Filling grouped by sorting prefix It can be useful to fill rows which have the same values in particular columns independently, - a good example is filling missing values in time series. -Assume there is the following time series table +Assume there is the following time series table: ``` sql CREATE TABLE timeseries ( @@ -567,7 +567,7 @@ SELECT * FROM timeseries; └───────────┴─────────────────────────┴───────┘ ``` And we'd like to fill missing values for each sensor independently with 1 second interval. -The way to achieve it is to use `sensor_id` column as sorting prefix for filling column `timestamp` +The way to achieve it is to use `sensor_id` column as sorting prefix for filling column `timestamp`: ``` SELECT * FROM timeseries @@ -589,7 +589,7 @@ INTERPOLATE ( value AS 9999 ) │ 432 │ 2021-12-01 00:00:05.000 │ 5 │ └───────────┴─────────────────────────┴───────┘ ``` -Here, the `value` column was interpolated with `9999` just to make filled rows more noticeable +Here, the `value` column was interpolated with `9999` just to make filled rows more noticeable. This behavior is controlled by setting `use_with_fill_by_sorting_prefix` (enabled by default) ## Related content From 0d0e53ecc0874530a791fc454bb58684b79244d5 Mon Sep 17 00:00:00 2001 From: Mohammad Arab Anvari Date: Fri, 26 May 2023 13:07:37 +0330 Subject: [PATCH 40/46] Update distributed.md Fix broken link in `**See Also**` section. --- docs/en/engines/table-engines/special/distributed.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index 43b8e387ba2..d1a0b13b363 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -258,4 +258,4 @@ Since [remote](../../../sql-reference/table-functions/remote.md) and [cluster](. - [Virtual columns](../../../engines/table-engines/index.md#table_engines-virtual_columns) description - [background_distributed_schedule_pool_size](../../../operations/settings/settings.md#background_distributed_schedule_pool_size) setting -- [shardNum()](../../../sql-reference/functions/other-functions.md#shard-num) and [shardCount()](../../../sql-reference/functions/other-functions.md#shard-count) functions +- [shardNum()](../../../sql-reference/functions/other-functions.md#shardnum) and [shardCount()](../../../sql-reference/functions/other-functions.md#shardcount) functions From 0d1f2e297b4de987877f79ded7f1aa7c6798565d Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 26 May 2023 15:55:30 +0200 Subject: [PATCH 41/46] Unify priorities: lower value means higher priority (#50205) --- src/Common/Priority.h | 11 ++++++++++ src/Common/ThreadPool.cpp | 8 +++---- src/Common/ThreadPool.h | 21 ++++++++++--------- .../CachedCompressedReadBuffer.cpp | 2 +- src/Compression/CachedCompressedReadBuffer.h | 2 +- .../CompressedReadBufferFromFile.cpp | 2 +- .../CompressedReadBufferFromFile.h | 2 +- .../IO/AsynchronousBoundedReadBuffer.cpp | 6 +++--- src/Disks/IO/AsynchronousBoundedReadBuffer.h | 6 +++--- src/IO/AsynchronousReadBufferFromFile.cpp | 4 ++-- src/IO/AsynchronousReadBufferFromFile.h | 6 +++--- ...ynchronousReadBufferFromFileDescriptor.cpp | 8 +++---- ...AsynchronousReadBufferFromFileDescriptor.h | 9 ++++---- src/IO/AsynchronousReader.h | 3 ++- src/IO/CompressedReadBufferWrapper.h | 2 +- src/IO/ParallelReadBuffer.cpp | 2 +- src/IO/PeekableReadBuffer.h | 2 +- src/IO/ReadBuffer.h | 9 ++++---- src/IO/ReadBufferFromFileDescriptor.cpp | 2 +- src/IO/ReadBufferFromFileDescriptor.h | 2 +- src/IO/ReadSettings.h | 5 +++-- src/IO/S3/copyS3File.cpp | 2 +- src/IO/SeekAvoidingReadBuffer.h | 2 +- src/IO/WriteBufferFromS3TaskTracker.cpp | 2 +- src/Interpreters/Context.cpp | 2 +- .../FilesystemReadPrefetchesLog.cpp | 4 ++-- .../FilesystemReadPrefetchesLog.h | 3 ++- src/Interpreters/threadPoolCallbackRunner.h | 9 ++++---- .../HDFS/AsynchronousReadBufferFromHDFS.cpp | 6 +++--- .../HDFS/AsynchronousReadBufferFromHDFS.h | 6 +++--- src/Storages/MergeTree/IMergeTreeReader.h | 2 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 2 +- .../MergeTree/MergeTreeBlockReadUtils.h | 8 ++----- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 14 ++++++------- .../MergeTree/MergeTreePrefetchedReadPool.h | 3 +-- .../MergeTree/MergeTreeReaderCompact.cpp | 2 +- .../MergeTree/MergeTreeReaderCompact.h | 2 +- .../MergeTree/MergeTreeReaderWide.cpp | 8 +++---- src/Storages/MergeTree/MergeTreeReaderWide.h | 6 +++--- src/Storages/MergeTree/MergeTreeSource.cpp | 2 +- src/Storages/StorageS3.cpp | 4 ++-- 42 files changed, 108 insertions(+), 97 deletions(-) create mode 100644 src/Common/Priority.h diff --git a/src/Common/Priority.h b/src/Common/Priority.h new file mode 100644 index 00000000000..8952fe4dd5a --- /dev/null +++ b/src/Common/Priority.h @@ -0,0 +1,11 @@ +#pragma once + +#include + +/// Common type for priority values. +/// Separate type (rather than `Int64` is used just to avoid implicit conversion errors and to default-initialize +struct Priority +{ + Int64 value = 0; /// Note that lower value means higher priority. + constexpr operator Int64() const { return value; } /// NOLINT +}; diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 63eef72dffb..f91849ead66 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -123,7 +123,7 @@ void ThreadPoolImpl::setQueueSize(size_t value) template template -ReturnType ThreadPoolImpl::scheduleImpl(Job job, ssize_t priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context) +ReturnType ThreadPoolImpl::scheduleImpl(Job job, Priority priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context) { auto on_error = [&](const std::string & reason) { @@ -231,19 +231,19 @@ void ThreadPoolImpl::startNewThreadsNoLock() } template -void ThreadPoolImpl::scheduleOrThrowOnError(Job job, ssize_t priority) +void ThreadPoolImpl::scheduleOrThrowOnError(Job job, Priority priority) { scheduleImpl(std::move(job), priority, std::nullopt); } template -bool ThreadPoolImpl::trySchedule(Job job, ssize_t priority, uint64_t wait_microseconds) noexcept +bool ThreadPoolImpl::trySchedule(Job job, Priority priority, uint64_t wait_microseconds) noexcept { return scheduleImpl(std::move(job), priority, wait_microseconds); } template -void ThreadPoolImpl::scheduleOrThrow(Job job, ssize_t priority, uint64_t wait_microseconds, bool propagate_opentelemetry_tracing_context) +void ThreadPoolImpl::scheduleOrThrow(Job job, Priority priority, uint64_t wait_microseconds, bool propagate_opentelemetry_tracing_context) { scheduleImpl(std::move(job), priority, wait_microseconds, propagate_opentelemetry_tracing_context); } diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index b92cb256b13..70053ff7558 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -18,6 +18,7 @@ #include #include #include +#include #include /** Very simple thread pool similar to boost::threadpool. @@ -59,17 +60,17 @@ public: /// If any thread was throw an exception, first exception will be rethrown from this method, /// and exception will be cleared. /// Also throws an exception if cannot create thread. - /// Priority: greater is higher. + /// Priority: lower is higher. /// NOTE: Probably you should call wait() if exception was thrown. If some previously scheduled jobs are using some objects, /// located on stack of current thread, the stack must not be unwinded until all jobs finished. However, /// if ThreadPool is a local object, it will wait for all scheduled jobs in own destructor. - void scheduleOrThrowOnError(Job job, ssize_t priority = 0); + void scheduleOrThrowOnError(Job job, Priority priority = {}); /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or return false. - bool trySchedule(Job job, ssize_t priority = 0, uint64_t wait_microseconds = 0) noexcept; + bool trySchedule(Job job, Priority priority = {}, uint64_t wait_microseconds = 0) noexcept; /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or throw an exception. - void scheduleOrThrow(Job job, ssize_t priority = 0, uint64_t wait_microseconds = 0, bool propagate_opentelemetry_tracing_context = true); + void scheduleOrThrow(Job job, Priority priority = {}, uint64_t wait_microseconds = 0, bool propagate_opentelemetry_tracing_context = true); /// Wait for all currently active jobs to be done. /// You may call schedule and wait many times in arbitrary order. @@ -123,15 +124,15 @@ private: struct JobWithPriority { Job job; - ssize_t priority; + Priority priority; DB::OpenTelemetry::TracingContextOnThread thread_trace_context; - JobWithPriority(Job job_, ssize_t priority_, const DB::OpenTelemetry::TracingContextOnThread& thread_trace_context_) + JobWithPriority(Job job_, Priority priority_, const DB::OpenTelemetry::TracingContextOnThread & thread_trace_context_) : job(job_), priority(priority_), thread_trace_context(thread_trace_context_) {} - bool operator< (const JobWithPriority & rhs) const + bool operator<(const JobWithPriority & rhs) const { - return priority < rhs.priority; + return priority > rhs.priority; // Reversed for `priority_queue` max-heap to yield minimum value (i.e. highest priority) first } }; @@ -141,7 +142,7 @@ private: std::stack on_destroy_callbacks; template - ReturnType scheduleImpl(Job job, ssize_t priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context = true); + ReturnType scheduleImpl(Job job, Priority priority, std::optional wait_microseconds, bool propagate_opentelemetry_tracing_context = true); void worker(typename std::list::iterator thread_it); @@ -227,7 +228,7 @@ public: DB::ThreadStatus thread_status; std::apply(function, arguments); }, - 0, // default priority + {}, // default priority 0, // default wait_microseconds propagate_opentelemetry_context ); diff --git a/src/Compression/CachedCompressedReadBuffer.cpp b/src/Compression/CachedCompressedReadBuffer.cpp index fdb2132d134..0febfca75cc 100644 --- a/src/Compression/CachedCompressedReadBuffer.cpp +++ b/src/Compression/CachedCompressedReadBuffer.cpp @@ -28,7 +28,7 @@ void CachedCompressedReadBuffer::initInput() } -void CachedCompressedReadBuffer::prefetch(int64_t priority) +void CachedCompressedReadBuffer::prefetch(Priority priority) { initInput(); file_in->prefetch(priority); diff --git a/src/Compression/CachedCompressedReadBuffer.h b/src/Compression/CachedCompressedReadBuffer.h index ee2728752e9..cce9a8e671c 100644 --- a/src/Compression/CachedCompressedReadBuffer.h +++ b/src/Compression/CachedCompressedReadBuffer.h @@ -36,7 +36,7 @@ private: bool nextImpl() override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; /// Passed into file_in. ReadBufferFromFileBase::ProfileCallback profile_callback; diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index ca697e2f51e..9dc40b8217c 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -51,7 +51,7 @@ CompressedReadBufferFromFile::CompressedReadBufferFromFile(std::unique_ptr buf, bool allow_different_codecs_ = false); diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp index 611fd00a023..aed4c2f82f2 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.cpp @@ -83,19 +83,19 @@ bool AsynchronousBoundedReadBuffer::hasPendingDataToRead() } std::future -AsynchronousBoundedReadBuffer::asyncReadInto(char * data, size_t size, int64_t priority) +AsynchronousBoundedReadBuffer::asyncReadInto(char * data, size_t size, Priority priority) { IAsynchronousReader::Request request; request.descriptor = std::make_shared(*impl, async_read_counters); request.buf = data; request.size = size; request.offset = file_offset_of_buffer_end; - request.priority = read_settings.priority + priority; + request.priority = Priority{read_settings.priority.value + priority.value}; request.ignore = bytes_to_ignore; return reader.submit(request); } -void AsynchronousBoundedReadBuffer::prefetch(int64_t priority) +void AsynchronousBoundedReadBuffer::prefetch(Priority priority) { if (prefetch_future.valid()) return; diff --git a/src/Disks/IO/AsynchronousBoundedReadBuffer.h b/src/Disks/IO/AsynchronousBoundedReadBuffer.h index 45256cdfac2..cafc5b6d691 100644 --- a/src/Disks/IO/AsynchronousBoundedReadBuffer.h +++ b/src/Disks/IO/AsynchronousBoundedReadBuffer.h @@ -39,7 +39,7 @@ public: off_t seek(off_t offset_, int whence) override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; void setReadUntilPosition(size_t position) override; /// [..., position). @@ -72,7 +72,7 @@ private: struct LastPrefetchInfo { UInt64 submit_time = 0; - size_t priority = 0; + Priority priority; }; LastPrefetchInfo last_prefetch_info; @@ -87,7 +87,7 @@ private: int64_t size, const std::unique_ptr & execution_watch); - std::future asyncReadInto(char * data, size_t size, int64_t priority); + std::future asyncReadInto(char * data, size_t size, Priority priority); void resetPrefetch(FilesystemPrefetchState state); diff --git a/src/IO/AsynchronousReadBufferFromFile.cpp b/src/IO/AsynchronousReadBufferFromFile.cpp index 7499b684a89..0e6c8090cb5 100644 --- a/src/IO/AsynchronousReadBufferFromFile.cpp +++ b/src/IO/AsynchronousReadBufferFromFile.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, const std::string & file_name_, size_t buf_size, int flags, @@ -60,7 +60,7 @@ AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile( AsynchronousReadBufferFromFile::AsynchronousReadBufferFromFile( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, int & fd_, const std::string & original_file_name, size_t buf_size, diff --git a/src/IO/AsynchronousReadBufferFromFile.h b/src/IO/AsynchronousReadBufferFromFile.h index 0ac0a820e48..d3b7ffbc7dc 100644 --- a/src/IO/AsynchronousReadBufferFromFile.h +++ b/src/IO/AsynchronousReadBufferFromFile.h @@ -17,7 +17,7 @@ protected: public: explicit AsynchronousReadBufferFromFile( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, const std::string & file_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1, @@ -28,7 +28,7 @@ public: /// Use pre-opened file descriptor. explicit AsynchronousReadBufferFromFile( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, int & fd, /// Will be set to -1 if constructor didn't throw and ownership of file descriptor is passed to the object. const std::string & original_file_name = {}, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, @@ -58,7 +58,7 @@ private: public: AsynchronousReadBufferFromFileWithDescriptorsCache( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, const std::string & file_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, int flags = -1, diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp index 743892d24f6..d30773f88f4 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.cpp @@ -40,14 +40,14 @@ std::string AsynchronousReadBufferFromFileDescriptor::getFileName() const } -std::future AsynchronousReadBufferFromFileDescriptor::asyncReadInto(char * data, size_t size, int64_t priority) +std::future AsynchronousReadBufferFromFileDescriptor::asyncReadInto(char * data, size_t size, Priority priority) { IAsynchronousReader::Request request; request.descriptor = std::make_shared(fd); request.buf = data; request.size = size; request.offset = file_offset_of_buffer_end; - request.priority = base_priority + priority; + request.priority = Priority{base_priority.value + priority.value}; request.ignore = bytes_to_ignore; bytes_to_ignore = 0; @@ -61,7 +61,7 @@ std::future AsynchronousReadBufferFromFileDescripto } -void AsynchronousReadBufferFromFileDescriptor::prefetch(int64_t priority) +void AsynchronousReadBufferFromFileDescriptor::prefetch(Priority priority) { if (prefetch_future.valid()) return; @@ -151,7 +151,7 @@ void AsynchronousReadBufferFromFileDescriptor::finalize() AsynchronousReadBufferFromFileDescriptor::AsynchronousReadBufferFromFileDescriptor( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, int fd_, size_t buf_size, char * existing_memory, diff --git a/src/IO/AsynchronousReadBufferFromFileDescriptor.h b/src/IO/AsynchronousReadBufferFromFileDescriptor.h index dff56baef4e..4a4130ebab1 100644 --- a/src/IO/AsynchronousReadBufferFromFileDescriptor.h +++ b/src/IO/AsynchronousReadBufferFromFileDescriptor.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -18,7 +19,7 @@ class AsynchronousReadBufferFromFileDescriptor : public ReadBufferFromFileBase { protected: IAsynchronousReader & reader; - int64_t base_priority; + Priority base_priority; Memory<> prefetch_buffer; std::future prefetch_future; @@ -39,7 +40,7 @@ protected: public: AsynchronousReadBufferFromFileDescriptor( IAsynchronousReader & reader_, - Int32 priority_, + Priority priority_, int fd_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, @@ -49,7 +50,7 @@ public: ~AsynchronousReadBufferFromFileDescriptor() override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; int getFD() const { @@ -70,7 +71,7 @@ public: size_t getFileSize() override; private: - std::future asyncReadInto(char * data, size_t size, int64_t priority); + std::future asyncReadInto(char * data, size_t size, Priority priority); }; } diff --git a/src/IO/AsynchronousReader.h b/src/IO/AsynchronousReader.h index 91a5db9dcdd..6b8b93fcc20 100644 --- a/src/IO/AsynchronousReader.h +++ b/src/IO/AsynchronousReader.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -47,7 +48,7 @@ public: size_t offset = 0; size_t size = 0; char * buf = nullptr; - int64_t priority = 0; + Priority priority; size_t ignore = 0; }; diff --git a/src/IO/CompressedReadBufferWrapper.h b/src/IO/CompressedReadBufferWrapper.h index 8fe87d22dd9..bb58a7bfeb3 100644 --- a/src/IO/CompressedReadBufferWrapper.h +++ b/src/IO/CompressedReadBufferWrapper.h @@ -19,7 +19,7 @@ public: const ReadBuffer & getWrappedReadBuffer() const { return *in; } ReadBuffer & getWrappedReadBuffer() { return *in; } - void prefetch(int64_t priority) override { in->prefetch(priority); } + void prefetch(Priority priority) override { in->prefetch(priority); } protected: std::unique_ptr in; diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index 6188fc210be..fff02db1bd6 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -87,7 +87,7 @@ bool ParallelReadBuffer::addReaderToPool() auto worker = read_workers.emplace_back(std::make_shared(std::move(reader), range_start, size)); ++active_working_reader; - schedule([this, my_worker = std::move(worker)]() mutable { readerThreadFunction(std::move(my_worker)); }, 0); + schedule([this, my_worker = std::move(worker)]() mutable { readerThreadFunction(std::move(my_worker)); }, Priority{}); return true; } diff --git a/src/IO/PeekableReadBuffer.h b/src/IO/PeekableReadBuffer.h index 7af17aff090..78cb319327d 100644 --- a/src/IO/PeekableReadBuffer.h +++ b/src/IO/PeekableReadBuffer.h @@ -20,7 +20,7 @@ public: ~PeekableReadBuffer() override; - void prefetch(int64_t priority) override { sub_buf->prefetch(priority); } + void prefetch(Priority priority) override { sub_buf->prefetch(priority); } /// Sets checkpoint at current position ALWAYS_INLINE inline void setCheckpoint() diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 1a8222001a9..a4ae12f5069 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -20,7 +21,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -static constexpr auto DEFAULT_PREFETCH_PRIORITY = 0; +static constexpr auto DEFAULT_PREFETCH_PRIORITY = Priority{0}; /** A simple abstract class for buffered data reading (char sequences) from somewhere. * Unlike std::istream, it provides access to the internal buffer, @@ -208,10 +209,10 @@ public: /** Do something to allow faster subsequent call to 'nextImpl' if possible. * It's used for asynchronous readers with double-buffering. - * `priority` is the Threadpool priority, with which the prefetch task will be schedules. - * Smaller is more priority. + * `priority` is the `ThreadPool` priority, with which the prefetch task will be scheduled. + * Lower value means higher priority. */ - virtual void prefetch(int64_t /* priority */) {} + virtual void prefetch(Priority) {} /** * Set upper bound for read range [..., position). diff --git a/src/IO/ReadBufferFromFileDescriptor.cpp b/src/IO/ReadBufferFromFileDescriptor.cpp index 1e7ea855364..bf44d9d10da 100644 --- a/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/src/IO/ReadBufferFromFileDescriptor.cpp @@ -124,7 +124,7 @@ bool ReadBufferFromFileDescriptor::nextImpl() } -void ReadBufferFromFileDescriptor::prefetch(int64_t) +void ReadBufferFromFileDescriptor::prefetch(Priority) { #if defined(POSIX_FADV_WILLNEED) /// For direct IO, loading data into page cache is pointless. diff --git a/src/IO/ReadBufferFromFileDescriptor.h b/src/IO/ReadBufferFromFileDescriptor.h index daf8ac56d13..10f140275bb 100644 --- a/src/IO/ReadBufferFromFileDescriptor.h +++ b/src/IO/ReadBufferFromFileDescriptor.h @@ -25,7 +25,7 @@ protected: ThrottlerPtr throttler; bool nextImpl() override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; /// Name or some description of file. std::string getFileName() const override; diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index d1551b16324..e43ecd7f275 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -5,6 +5,7 @@ #include #include #include +#include #include namespace DB @@ -84,8 +85,8 @@ struct ReadSettings size_t mmap_threshold = 0; MMappedFileCache * mmap_cache = nullptr; - /// For 'pread_threadpool'/'io_uring' method. Lower is more priority. - size_t priority = 0; + /// For 'pread_threadpool'/'io_uring' method. Lower value is higher priority. + Priority priority; bool load_marks_asynchronously = true; diff --git a/src/IO/S3/copyS3File.cpp b/src/IO/S3/copyS3File.cpp index 20490ef9a19..7886b84cd00 100644 --- a/src/IO/S3/copyS3File.cpp +++ b/src/IO/S3/copyS3File.cpp @@ -361,7 +361,7 @@ namespace task->exception = std::current_exception(); } task_finish_notify(); - }, 0); + }, Priority{}); } catch (...) { diff --git a/src/IO/SeekAvoidingReadBuffer.h b/src/IO/SeekAvoidingReadBuffer.h index af2a988ab3a..716d7c5046c 100644 --- a/src/IO/SeekAvoidingReadBuffer.h +++ b/src/IO/SeekAvoidingReadBuffer.h @@ -17,7 +17,7 @@ public: off_t seek(off_t off, int whence) override; - void prefetch(int64_t priority) override { impl->prefetch(priority); } + void prefetch(Priority priority) override { impl->prefetch(priority); } private: UInt64 min_bytes_for_seek; /// Minimum positive seek offset which shall be executed using seek operation. diff --git a/src/IO/WriteBufferFromS3TaskTracker.cpp b/src/IO/WriteBufferFromS3TaskTracker.cpp index 7826747c0a4..6c7bb35fc5a 100644 --- a/src/IO/WriteBufferFromS3TaskTracker.cpp +++ b/src/IO/WriteBufferFromS3TaskTracker.cpp @@ -113,7 +113,7 @@ void WriteBufferFromS3::TaskTracker::add(Callback && func) { LOG_TEST(log, "add, in queue {}", futures.size()); - auto future = scheduler(std::move(func), 0); + auto future = scheduler(std::move(func), Priority{}); auto exit_scope = scope_guard( [&future]() { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index c594feb73c2..91f34c8eb97 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4269,7 +4269,7 @@ ReadSettings Context::getReadSettings() const res.prefetch_buffer_size = settings.prefetch_buffer_size; res.direct_io_threshold = settings.min_bytes_to_use_direct_io; res.mmap_threshold = settings.min_bytes_to_use_mmap_io; - res.priority = settings.read_priority; + res.priority = Priority{settings.read_priority}; res.remote_throttler = getRemoteReadThrottler(); res.local_throttler = getLocalReadThrottler(); diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.cpp b/src/Interpreters/FilesystemReadPrefetchesLog.cpp index 9dd0fce84b0..221cb11f09a 100644 --- a/src/Interpreters/FilesystemReadPrefetchesLog.cpp +++ b/src/Interpreters/FilesystemReadPrefetchesLog.cpp @@ -19,7 +19,7 @@ NamesAndTypesList FilesystemReadPrefetchesLogElement::getNamesAndTypes() {"offset", std::make_shared()}, {"size", std::make_shared()}, {"prefetch_submit_time", std::make_shared(6)}, - {"priority", std::make_shared()}, + {"priority", std::make_shared()}, {"prefetch_execution_start_time", std::make_shared(6)}, {"prefetch_execution_end_time", std::make_shared(6)}, {"prefetch_execution_time_us", std::make_shared()}, @@ -40,7 +40,7 @@ void FilesystemReadPrefetchesLogElement::appendToBlock(MutableColumns & columns) columns[i++]->insert(offset); columns[i++]->insert(size); columns[i++]->insert(prefetch_submit_time); - columns[i++]->insert(priority); + columns[i++]->insert(priority.value); if (execution_watch) { columns[i++]->insert(execution_watch->getStart()); diff --git a/src/Interpreters/FilesystemReadPrefetchesLog.h b/src/Interpreters/FilesystemReadPrefetchesLog.h index 7052cf2769d..cf36f513f5a 100644 --- a/src/Interpreters/FilesystemReadPrefetchesLog.h +++ b/src/Interpreters/FilesystemReadPrefetchesLog.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -25,7 +26,7 @@ struct FilesystemReadPrefetchesLogElement Int64 size; /// -1 means unknown Decimal64 prefetch_submit_time{}; std::optional execution_watch; - size_t priority; + Priority priority; FilesystemPrefetchState state; UInt64 thread_id; String reader_id; diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index b8805722346..55c6a848b77 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -11,13 +11,13 @@ namespace DB /// High-order function to run callbacks (functions with 'void()' signature) somewhere asynchronously. template > -using ThreadPoolCallbackRunner = std::function(Callback &&, int64_t priority)>; +using ThreadPoolCallbackRunner = std::function(Callback &&, Priority)>; /// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()'. template > ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name) { - return [my_pool = &pool, thread_group = CurrentThread::getGroup(), thread_name](Callback && callback, int64_t priority) mutable -> std::future + return [my_pool = &pool, thread_group = CurrentThread::getGroup(), thread_name](Callback && callback, Priority priority) mutable -> std::future { auto task = std::make_shared>([thread_group, thread_name, my_callback = std::move(callback)]() mutable -> Result { @@ -44,15 +44,14 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & auto future = task->get_future(); - /// ThreadPool is using "bigger is higher priority" instead of "smaller is more priority". - my_pool->scheduleOrThrow([my_task = std::move(task)]{ (*my_task)(); }, -priority); + my_pool->scheduleOrThrow([my_task = std::move(task)]{ (*my_task)(); }, priority); return future; }; } template -std::future scheduleFromThreadPool(T && task, ThreadPool & pool, const std::string & thread_name, int64_t priority = 0) +std::future scheduleFromThreadPool(T && task, ThreadPool & pool, const std::string & thread_name, Priority priority = {}) { auto schedule = threadPoolCallbackRunner(pool, thread_name); return schedule(std::move(task), priority); diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp index a8502b1bd65..b490c5cac63 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp @@ -64,19 +64,19 @@ bool AsynchronousReadBufferFromHDFS::hasPendingDataToRead() return true; } -std::future AsynchronousReadBufferFromHDFS::asyncReadInto(char * data, size_t size, int64_t priority) +std::future AsynchronousReadBufferFromHDFS::asyncReadInto(char * data, size_t size, Priority priority) { IAsynchronousReader::Request request; request.descriptor = std::make_shared(*impl, nullptr); request.buf = data; request.size = size; request.offset = file_offset_of_buffer_end; - request.priority = base_priority + priority; + request.priority = Priority{base_priority.value + priority.value}; request.ignore = 0; return reader.submit(request); } -void AsynchronousReadBufferFromHDFS::prefetch(int64_t priority) +void AsynchronousReadBufferFromHDFS::prefetch(Priority priority) { interval_watch.restart(); diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h index 07d32194a93..d89aa60ab71 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h +++ b/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h @@ -33,7 +33,7 @@ public: off_t seek(off_t offset_, int whence) override; - void prefetch(int64_t priority) override; + void prefetch(Priority priority) override; size_t getFileSize() override; @@ -50,10 +50,10 @@ private: bool hasPendingDataToRead(); - std::future asyncReadInto(char * data, size_t size, int64_t priority); + std::future asyncReadInto(char * data, size_t size, Priority priority); IAsynchronousReader & reader; - int64_t base_priority; + Priority base_priority; std::shared_ptr impl; std::future prefetch_future; Memory<> prefetch_buffer; diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index c7396e882fa..f4759409c52 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -61,7 +61,7 @@ public: MergeTreeDataPartInfoForReaderPtr data_part_info_for_read; - virtual void prefetchBeginOfRange(int64_t /* priority */) {} + virtual void prefetchBeginOfRange(Priority) {} protected: /// Returns actual column name in part, which can differ from table metadata. diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 4c3d4bc8aa0..9eeb1ec13cf 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -142,7 +142,7 @@ MergeTreeReadTask::MergeTreeReadTask( const NameSet & column_name_set_, const MergeTreeReadTaskColumns & task_columns_, MergeTreeBlockSizePredictorPtr size_predictor_, - int64_t priority_, + Priority priority_, std::future reader_, std::vector> && pre_reader_for_step_) : data_part{data_part_} diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 5fce95e1714..6eefa993e5e 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -71,11 +71,7 @@ struct MergeTreeReadTask std::future reader; std::vector> pre_reader_for_step; - int64_t priority = 0; /// Priority of the task. Bigger value, bigger priority. - bool operator <(const MergeTreeReadTask & rhs) const - { - return priority < rhs.priority; - } + Priority priority; bool isFinished() const { return mark_ranges.empty() && range_reader.isCurrentRangeFinished(); } @@ -86,7 +82,7 @@ struct MergeTreeReadTask const NameSet & column_name_set_, const MergeTreeReadTaskColumns & task_columns_, MergeTreeBlockSizePredictorPtr size_predictor_, - int64_t priority_ = 0, + Priority priority_ = {}, std::future reader_ = {}, std::vector> && pre_reader_for_step_ = {}); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1d5ac21f803..11aa7c5ec76 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1967,7 +1967,7 @@ try res.part->remove(); else preparePartForRemoval(res.part); - }, 0)); + }, Priority{})); } /// Wait for every scheduled task diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 30f31910bee..6b5214622ff 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -90,7 +90,7 @@ std::future MergeTreePrefetchedReadPool::createPrefetchedRea const IMergeTreeDataPart & data_part, const NamesAndTypesList & columns, const MarkRanges & required_ranges, - int64_t priority) const + Priority priority) const { auto reader = data_part.getReader( columns, storage_snapshot->metadata, required_ranges, @@ -142,7 +142,7 @@ bool MergeTreePrefetchedReadPool::TaskHolder::operator <(const TaskHolder & othe { chassert(task->priority >= 0); chassert(other.task->priority >= 0); - return -task->priority < -other.task->priority; /// Less is better. + return task->priority > other.task->priority; /// Less is better. /// With default std::priority_queue, top() returns largest element. /// So closest to 0 will be on top with this comparator. } @@ -153,7 +153,7 @@ void MergeTreePrefetchedReadPool::startPrefetches() const return; [[maybe_unused]] TaskHolder prev(nullptr, 0); - [[maybe_unused]] const int64_t highest_priority = reader_settings.read_settings.priority + 1; + [[maybe_unused]] const Priority highest_priority{reader_settings.read_settings.priority.value + 1}; assert(prefetch_queue.top().task->priority == highest_priority); while (!prefetch_queue.empty()) { @@ -495,11 +495,11 @@ MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThr auto need_marks = min_marks_per_thread; /// Priority is given according to the prefetch number for each thread, - /// e.g. the first task of each thread has the same priority and is bigger - /// than second task of each thread, and so on. + /// e.g. the first task of each thread has the same priority and is greater + /// than the second task of each thread, and so on. /// Add 1 to query read priority because higher priority should be given to /// reads from pool which are from reader. - int64_t priority = reader_settings.read_settings.priority + 1; + Priority priority{reader_settings.read_settings.priority.value + 1}; while (need_marks > 0 && part_idx < parts_infos.size()) { @@ -597,7 +597,7 @@ MergeTreePrefetchedReadPool::ThreadsTasks MergeTreePrefetchedReadPool::createThr { prefetch_queue.emplace(TaskHolder(read_task.get(), i)); } - ++priority; + ++priority.value; result_threads_tasks[i].push_back(std::move(read_task)); } diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index a412dbf0811..7a815777125 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -53,12 +53,11 @@ private: using ThreadTasks = std::deque; using ThreadsTasks = std::map; - /// smaller `priority` means more priority std::future createPrefetchedReader( const IMergeTreeDataPart & data_part, const NamesAndTypesList & columns, const MarkRanges & required_ranges, - int64_t priority) const; + Priority priority) const; void createPrefetchedReaderForTask(MergeTreeReadTask & task) const; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 26a7cb2b50b..d22684eaa9d 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -314,7 +314,7 @@ void MergeTreeReaderCompact::readData( last_read_granule.emplace(from_mark, column_position); } -void MergeTreeReaderCompact::prefetchBeginOfRange(int64_t priority) +void MergeTreeReaderCompact::prefetchBeginOfRange(Priority priority) { if (!initialized) { diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index a994e72d3ff..f180d7508f7 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -38,7 +38,7 @@ public: bool canReadIncompleteGranules() const override { return false; } - void prefetchBeginOfRange(int64_t priority) override; + void prefetchBeginOfRange(Priority priority) override; private: bool isContinuousReading(size_t mark, size_t column_position); diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 69617fdf9e3..baacfa55c94 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -58,7 +58,7 @@ MergeTreeReaderWide::MergeTreeReaderWide( } } -void MergeTreeReaderWide::prefetchBeginOfRange(int64_t priority) +void MergeTreeReaderWide::prefetchBeginOfRange(Priority priority) { prefetched_streams.clear(); @@ -90,7 +90,7 @@ void MergeTreeReaderWide::prefetchBeginOfRange(int64_t priority) } void MergeTreeReaderWide::prefetchForAllColumns( - int64_t priority, size_t num_columns, size_t from_mark, size_t current_task_last_mark, bool continue_reading) + Priority priority, size_t num_columns, size_t from_mark, size_t current_task_last_mark, bool continue_reading) { bool do_prefetch = data_part_info_for_read->getDataPartStorage()->isStoredOnRemoteDisk() ? settings.read_settings.remote_fs_prefetch @@ -137,7 +137,7 @@ size_t MergeTreeReaderWide::readRows( if (num_columns == 0) return max_rows_to_read; - prefetchForAllColumns(/* priority */0, num_columns, from_mark, current_task_last_mark, continue_reading); + prefetchForAllColumns(Priority{}, num_columns, from_mark, current_task_last_mark, continue_reading); for (size_t pos = 0; pos < num_columns; ++pos) { @@ -305,7 +305,7 @@ void MergeTreeReaderWide::deserializePrefix( } void MergeTreeReaderWide::prefetchForColumn( - int64_t priority, + Priority priority, const NameAndTypePair & name_and_type, const SerializationPtr & serialization, size_t from_mark, diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index 08bb17c5d56..c31b1baf32e 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -33,14 +33,14 @@ public: bool canReadIncompleteGranules() const override { return true; } - void prefetchBeginOfRange(int64_t priority) override; + void prefetchBeginOfRange(Priority priority) override; using FileStreams = std::map>; private: FileStreams streams; - void prefetchForAllColumns(int64_t priority, size_t num_columns, size_t from_mark, size_t current_task_last_mark, bool continue_reading); + void prefetchForAllColumns(Priority priority, size_t num_columns, size_t from_mark, size_t current_task_last_mark, bool continue_reading); void addStreams( const NameAndTypePair & name_and_type, @@ -55,7 +55,7 @@ private: /// Make next readData more simple by calling 'prefetch' of all related ReadBuffers (column streams). void prefetchForColumn( - int64_t priority, + Priority priority, const NameAndTypePair & name_and_type, const SerializationPtr & serialization, size_t from_mark, diff --git a/src/Storages/MergeTree/MergeTreeSource.cpp b/src/Storages/MergeTree/MergeTreeSource.cpp index 500327afd61..53695639769 100644 --- a/src/Storages/MergeTree/MergeTreeSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSource.cpp @@ -84,7 +84,7 @@ struct MergeTreeSource::AsyncReadingState { try { - callback_runner(std::move(job), 0); + callback_runner(std::move(job), Priority{}); } catch (...) { diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 51acb6a3a7d..80a49b2cd1d 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -356,7 +356,7 @@ private: request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); return outcome; - }, 0); + }, Priority{}); } std::mutex mutex; @@ -619,7 +619,7 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() std::future StorageS3Source::createReaderAsync() { - return create_reader_scheduler([this] { return createReader(); }, 0); + return create_reader_scheduler([this] { return createReader(); }, Priority{}); } StorageS3Source::ReadBufferOrFactory StorageS3Source::createS3ReadBuffer(const String & key, size_t object_size) From 3ef04ad4177091302c57772f5e4f9c6fc32f9855 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 26 May 2023 10:56:24 -0300 Subject: [PATCH 42/46] Doc. clarify dns_max_consecutive_failures one more time (#50212) --- src/Core/ServerSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 1441c22507d..511b8d68f6d 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -63,7 +63,7 @@ namespace DB \ M(Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0) \ M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \ - M(UInt32, dns_max_consecutive_failures, 1024, "Max connection failures before dropping host from ClickHouse DNS cache.", 0) \ + M(UInt32, dns_max_consecutive_failures, 1024, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0) \ \ M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \ M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \ From 4c218f28924fbd021341aafd8a27dbdd38c52e74 Mon Sep 17 00:00:00 2001 From: ltrk2 <107155950+ltrk2@users.noreply.github.com> Date: Fri, 26 May 2023 09:59:03 -0400 Subject: [PATCH 43/46] Upgrade boost submodule (#50188) --- contrib/boost | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boost b/contrib/boost index 8fe7b3326ef..aec12eea7fc 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit 8fe7b3326ef482ee6ecdf5a4f698f2b8c2780f98 +Subproject commit aec12eea7fc762721ae16943d1361340c66c9c17 From b9ef5151ad5aefa44a41e14302cdaca1815749b3 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 26 May 2023 11:18:25 -0300 Subject: [PATCH 44/46] Update settings.md --- .../en/operations/server-configuration-parameters/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 99630043095..76ff8ea049e 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -917,7 +917,7 @@ We recommend using this option in macOS since the `getrlimit()` function returns Restriction on deleting tables. -If the size of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `max_table_size_to_drop` (in bytes), you can’t delete it using a [DROP](sql-reference/statements/drop.md) query or [TRUNCATE](../../sql-reference/statements/truncate.md) query. +If the size of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `max_table_size_to_drop` (in bytes), you can’t delete it using a [DROP](../../sql-reference/statements/drop.md) query or [TRUNCATE](../../sql-reference/statements/truncate.md) query. This setting does not require a restart of the Clickhouse server to apply. Another way to disable the restriction is to create the `/flags/force_drop_table` file. @@ -944,7 +944,7 @@ Default value: 50 GB. The value 0 means that you can drop partitions without any restrictions. :::note -This limitation does not restrict drop table and truncate table, see [max_table_size_to_drop](#max_table_size_to_drop) +This limitation does not restrict drop table and truncate table, see [max_table_size_to_drop](#max-table-size-to-drop) ::: **Example** From edffedbabf84f2e4d539b684a65b3085430a45aa Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 26 May 2023 15:10:08 +0000 Subject: [PATCH 45/46] fix --- src/Common/AsyncLoader.h | 12 ++++++++++-- src/Common/tests/gtest_async_loader.cpp | 4 +++- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Common/AsyncLoader.h b/src/Common/AsyncLoader.h index c3d11b26336..15f7ae9722b 100644 --- a/src/Common/AsyncLoader.h +++ b/src/Common/AsyncLoader.h @@ -235,6 +235,14 @@ inline LoadJobSet getGoals(const LoadTaskPtrs & tasks) return result; } +inline LoadJobSet getGoalsOr(const LoadTaskPtrs & tasks, const LoadJobSet & alternative) +{ + LoadJobSet result; + for (const auto & task : tasks) + result.insert(task->goals().begin(), task->goals().end()); + return result.empty() ? alternative : result; +} + inline LoadJobSet joinJobs(const LoadJobSet & jobs1, const LoadJobSet & jobs2) { LoadJobSet result; @@ -368,7 +376,7 @@ public: // WARNING: all tasks instances should be destructed before associated AsyncLoader. ~AsyncLoader(); - // Start workers to execute scheduled load jobs. + // Start workers to execute scheduled load jobs. Note that AsyncLoader is constructed as already started. void start(); // Wait for all load jobs to finish, including all new jobs. So at first take care to stop adding new jobs. @@ -390,7 +398,7 @@ public: void schedule(const LoadTaskPtr & task); // Schedule all tasks atomically. To ensure only highest priority jobs among all tasks are run first. - void schedule(const std::vector & tasks); + void schedule(const LoadTaskPtrs & tasks); // Increase priority of a job and all its dependencies recursively. // Jobs from higher (than `new_pool`) priority pools are not changed. diff --git a/src/Common/tests/gtest_async_loader.cpp b/src/Common/tests/gtest_async_loader.cpp index 6a7d2d727a9..af685b10f4e 100644 --- a/src/Common/tests/gtest_async_loader.cpp +++ b/src/Common/tests/gtest_async_loader.cpp @@ -44,7 +44,9 @@ struct AsyncLoaderTest explicit AsyncLoaderTest(std::vector initializers) : loader(getPoolInitializers(initializers), /* log_failures = */ false, /* log_progress = */ false) - {} + { + loader.stop(); // All tests call `start()` manually to better control ordering + } explicit AsyncLoaderTest(size_t max_threads = 1) : AsyncLoaderTest({{.max_threads = max_threads}}) From 0e2156c77693ba0cfdd3c62b113f7f3a57ca2ffa Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 27 May 2023 14:35:37 +0200 Subject: [PATCH 46/46] Fix checksum calculation for backup entries (#50264) * Fix possible deadlock in calculating backup entries' checksums. * Fix data type used to store checksums. * Allow ReadBufferFromMemory::seek() to go to the end of the buffer (seek(0, SEEK_SET) must be allowed for an empty buffer) * Add tests. --------- Co-authored-by: Nikita Mikhaylov --- src/Backups/BackupEntryFromImmutableFile.cpp | 31 +- src/Backups/BackupEntryFromImmutableFile.h | 2 +- .../BackupEntryWithChecksumCalculation.cpp | 31 +- src/Backups/tests/gtest_backup_entries.cpp | 350 ++++++++++++++++++ src/IO/ReadBufferFromMemory.cpp | 4 +- 5 files changed, 400 insertions(+), 18 deletions(-) create mode 100644 src/Backups/tests/gtest_backup_entries.cpp diff --git a/src/Backups/BackupEntryFromImmutableFile.cpp b/src/Backups/BackupEntryFromImmutableFile.cpp index d066db0c6ae..93d555065ec 100644 --- a/src/Backups/BackupEntryFromImmutableFile.cpp +++ b/src/Backups/BackupEntryFromImmutableFile.cpp @@ -59,16 +59,31 @@ UInt64 BackupEntryFromImmutableFile::getSize() const UInt128 BackupEntryFromImmutableFile::getChecksum() const { - std::lock_guard lock{size_and_checksum_mutex}; - if (!checksum_adjusted) { - if (!checksum) - checksum = BackupEntryWithChecksumCalculation::getChecksum(); - else if (copy_encrypted) - checksum = combineChecksums(*checksum, disk->getEncryptedFileIV(file_path)); - checksum_adjusted = true; + std::lock_guard lock{size_and_checksum_mutex}; + if (checksum_adjusted) + return *checksum; + + if (checksum) + { + if (copy_encrypted) + checksum = combineChecksums(*checksum, disk->getEncryptedFileIV(file_path)); + checksum_adjusted = true; + return *checksum; + } + } + + auto calculated_checksum = BackupEntryWithChecksumCalculation::getChecksum(); + + { + std::lock_guard lock{size_and_checksum_mutex}; + if (!checksum_adjusted) + { + checksum = calculated_checksum; + checksum_adjusted = true; + } + return *checksum; } - return *checksum; } std::optional BackupEntryFromImmutableFile::getPartialChecksum(size_t prefix_length) const diff --git a/src/Backups/BackupEntryFromImmutableFile.h b/src/Backups/BackupEntryFromImmutableFile.h index 850a86a3264..37bc6b43cd3 100644 --- a/src/Backups/BackupEntryFromImmutableFile.h +++ b/src/Backups/BackupEntryFromImmutableFile.h @@ -44,7 +44,7 @@ private: const DataSourceDescription data_source_description; const bool copy_encrypted; mutable std::optional file_size; - mutable std::optional checksum; + mutable std::optional checksum; mutable bool file_size_adjusted = false; mutable bool checksum_adjusted = false; mutable std::mutex size_and_checksum_mutex; diff --git a/src/Backups/BackupEntryWithChecksumCalculation.cpp b/src/Backups/BackupEntryWithChecksumCalculation.cpp index 1e634e6bb73..610b46238ba 100644 --- a/src/Backups/BackupEntryWithChecksumCalculation.cpp +++ b/src/Backups/BackupEntryWithChecksumCalculation.cpp @@ -8,15 +8,32 @@ namespace DB template UInt128 BackupEntryWithChecksumCalculation::getChecksum() const { - std::lock_guard lock{checksum_calculation_mutex}; - if (!calculated_checksum) { - auto read_buffer = this->getReadBuffer(ReadSettings{}.adjustBufferSize(this->getSize())); - HashingReadBuffer hashing_read_buffer(*read_buffer); - hashing_read_buffer.ignoreAll(); - calculated_checksum = hashing_read_buffer.getHash(); + std::lock_guard lock{checksum_calculation_mutex}; + if (calculated_checksum) + return *calculated_checksum; + } + + size_t size = this->getSize(); + + { + std::lock_guard lock{checksum_calculation_mutex}; + if (!calculated_checksum) + { + if (size == 0) + { + calculated_checksum = 0; + } + else + { + auto read_buffer = this->getReadBuffer(ReadSettings{}.adjustBufferSize(size)); + HashingReadBuffer hashing_read_buffer(*read_buffer); + hashing_read_buffer.ignoreAll(); + calculated_checksum = hashing_read_buffer.getHash(); + } + } + return *calculated_checksum; } - return *calculated_checksum; } template diff --git a/src/Backups/tests/gtest_backup_entries.cpp b/src/Backups/tests/gtest_backup_entries.cpp new file mode 100644 index 00000000000..3077bedad0e --- /dev/null +++ b/src/Backups/tests/gtest_backup_entries.cpp @@ -0,0 +1,350 @@ +#include + +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include + +using namespace DB; + + +class BackupEntriesTest : public ::testing::Test +{ +protected: + void SetUp() override + { + /// Make local disk. + temp_dir = std::make_unique(); + temp_dir->createDirectories(); + local_disk = std::make_shared("local_disk", temp_dir->path() + "/", 0); + + /// Make encrypted disk. + auto settings = std::make_unique(); + settings->wrapped_disk = local_disk; + settings->current_algorithm = FileEncryption::Algorithm::AES_128_CTR; + settings->keys[0] = "1234567890123456"; + settings->current_key_id = 0; + settings->disk_path = "encrypted/"; + encrypted_disk = std::make_shared("encrypted_disk", std::move(settings), true); + } + + void TearDown() override + { + encrypted_disk.reset(); + local_disk.reset(); + } + + static void writeFile(DiskPtr disk, const String & filepath) + { + auto buf = disk->writeFile(filepath, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {}); + writeString(std::string_view{"Some text"}, *buf); + buf->finalize(); + } + + static void writeEmptyFile(DiskPtr disk, const String & filepath) + { + auto buf = disk->writeFile(filepath, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, {}); + buf->finalize(); + } + + static void appendFile(DiskPtr disk, const String & filepath) + { + auto buf = disk->writeFile(filepath, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append, {}); + writeString(std::string_view{"Appended"}, *buf); + buf->finalize(); + } + + static String getChecksum(const BackupEntryPtr & backup_entry) + { + return getHexUIntUppercase(backup_entry->getChecksum()); + } + + static const constexpr std::string_view NO_CHECKSUM = "no checksum"; + + static String getPartialChecksum(const BackupEntryPtr & backup_entry, size_t prefix_length) + { + auto partial_checksum = backup_entry->getPartialChecksum(prefix_length); + if (!partial_checksum) + return String{NO_CHECKSUM}; + return getHexUIntUppercase(*partial_checksum); + } + + static String readAll(const BackupEntryPtr & backup_entry) + { + auto in = backup_entry->getReadBuffer({}); + String str; + readStringUntilEOF(str, *in); + return str; + } + + std::unique_ptr temp_dir; + std::shared_ptr local_disk; + std::shared_ptr encrypted_disk; +}; + + +static const constexpr std::string_view ZERO_CHECKSUM = "00000000000000000000000000000000"; + +static const constexpr std::string_view SOME_TEXT_CHECKSUM = "28B5529750AC210952FFD366774363ED"; +static const constexpr std::string_view S_CHECKSUM = "C27395C39AFB5557BFE47661CC9EB86C"; +static const constexpr std::string_view SOME_TEX_CHECKSUM = "D00D9BE8D87919A165F14EDD31088A0E"; +static const constexpr std::string_view SOME_TEXT_APPENDED_CHECKSUM = "5A1F10F638DC7A226231F3FD927D1726"; + +static const constexpr std::string_view PRECALCULATED_CHECKSUM = "1122334455667788AABBCCDDAABBCCDD"; +static const constexpr UInt128 PRECALCULATED_CHECKSUM_UINT128 = (UInt128(0x1122334455667788) << 64) | 0xAABBCCDDAABBCCDD; +static const size_t PRECALCULATED_SIZE = 123; + +TEST_F(BackupEntriesTest, BackupEntryFromImmutableFile) +{ + writeFile(local_disk, "a.txt"); + + auto entry = std::make_shared(local_disk, "a.txt"); + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 8), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 9), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1000), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); + + writeEmptyFile(local_disk, "empty.txt"); + + auto empty_entry = std::make_shared(local_disk, "empty.txt"); + EXPECT_EQ(empty_entry->getSize(), 0); + EXPECT_EQ(getChecksum(empty_entry), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 1), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 1000), ZERO_CHECKSUM); + EXPECT_EQ(readAll(empty_entry), ""); + + auto precalculated_entry = std::make_shared(local_disk, "a.txt", false, PRECALCULATED_SIZE, PRECALCULATED_CHECKSUM_UINT128); + EXPECT_EQ(precalculated_entry->getSize(), PRECALCULATED_SIZE); + + EXPECT_EQ(getChecksum(precalculated_entry), PRECALCULATED_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 1), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, PRECALCULATED_SIZE - 1), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, PRECALCULATED_SIZE), PRECALCULATED_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 1000), PRECALCULATED_CHECKSUM); + EXPECT_EQ(readAll(precalculated_entry), "Some text"); +} + +TEST_F(BackupEntriesTest, BackupEntryFromAppendOnlyFile) +{ + writeFile(local_disk, "a.txt"); + + auto entry = std::make_shared(local_disk, "a.txt"); + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), S_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 8), SOME_TEX_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 9), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1000), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); + + appendFile(local_disk, "a.txt"); + + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), S_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 8), SOME_TEX_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 9), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1000), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); + + auto appended_entry = std::make_shared(local_disk, "a.txt"); + EXPECT_EQ(appended_entry->getSize(), 17); + EXPECT_EQ(getChecksum(appended_entry), SOME_TEXT_APPENDED_CHECKSUM); + EXPECT_EQ(getPartialChecksum(appended_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(appended_entry, 1), S_CHECKSUM); + EXPECT_EQ(getPartialChecksum(appended_entry, 8), SOME_TEX_CHECKSUM); + EXPECT_EQ(getPartialChecksum(appended_entry, 9), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(appended_entry, 22), SOME_TEXT_APPENDED_CHECKSUM); + EXPECT_EQ(getPartialChecksum(appended_entry, 1000), SOME_TEXT_APPENDED_CHECKSUM); + EXPECT_EQ(readAll(appended_entry), "Some textAppended"); + + writeEmptyFile(local_disk, "empty_appended.txt"); + + auto empty_entry = std::make_shared(local_disk, "empty_appended.txt"); + EXPECT_EQ(empty_entry->getSize(), 0); + EXPECT_EQ(getChecksum(empty_entry), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 1), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 1000), ZERO_CHECKSUM); + EXPECT_EQ(readAll(empty_entry), ""); + + appendFile(local_disk, "empty_appended.txt"); + EXPECT_EQ(empty_entry->getSize(), 0); + EXPECT_EQ(getChecksum(empty_entry), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 1), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(empty_entry, 1000), ZERO_CHECKSUM); + EXPECT_EQ(readAll(empty_entry), ""); +} + +TEST_F(BackupEntriesTest, PartialChecksumBeforeFullChecksum) +{ + writeFile(local_disk, "a.txt"); + + auto entry = std::make_shared(local_disk, "a.txt"); + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); + + entry = std::make_shared(local_disk, "a.txt"); + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getPartialChecksum(entry, 1), S_CHECKSUM); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); +} + +TEST_F(BackupEntriesTest, BackupEntryFromSmallFile) +{ + writeFile(local_disk, "a.txt"); + auto entry = std::make_shared(local_disk, "a.txt"); + + local_disk->removeFile("a.txt"); + + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), S_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 8), SOME_TEX_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 9), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1000), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); +} + +TEST_F(BackupEntriesTest, DecryptedEntriesFromEncryptedDisk) +{ + { + writeFile(encrypted_disk, "a.txt"); + std::pair test_cases[] + = {{std::make_shared(encrypted_disk, "a.txt"), false}, + {std::make_shared(encrypted_disk, "a.txt"), true}, + {std::make_shared(encrypted_disk, "a.txt"), true}}; + for (const auto & [entry, partial_checksum_allowed] : test_cases) + { + EXPECT_EQ(entry->getSize(), 9); + EXPECT_EQ(getChecksum(entry), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), partial_checksum_allowed ? S_CHECKSUM : NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 8), partial_checksum_allowed ? SOME_TEX_CHECKSUM : NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 9), SOME_TEXT_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1000), SOME_TEXT_CHECKSUM); + EXPECT_EQ(readAll(entry), "Some text"); + } + } + + { + writeEmptyFile(encrypted_disk, "empty.txt"); + BackupEntryPtr entries[] + = {std::make_shared(encrypted_disk, "empty.txt"), + std::make_shared(encrypted_disk, "empty.txt"), + std::make_shared(encrypted_disk, "empty.txt")}; + for (const auto & entry : entries) + { + EXPECT_EQ(entry->getSize(), 0); + EXPECT_EQ(getChecksum(entry), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), ZERO_CHECKSUM); + EXPECT_EQ(readAll(entry), ""); + } + } + + { + auto precalculated_entry = std::make_shared(encrypted_disk, "a.txt", false, PRECALCULATED_SIZE, PRECALCULATED_CHECKSUM_UINT128); + EXPECT_EQ(precalculated_entry->getSize(), PRECALCULATED_SIZE); + EXPECT_EQ(getChecksum(precalculated_entry), PRECALCULATED_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 1), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, PRECALCULATED_SIZE), PRECALCULATED_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 1000), PRECALCULATED_CHECKSUM); + EXPECT_EQ(readAll(precalculated_entry), "Some text"); + } +} + +TEST_F(BackupEntriesTest, EncryptedEntriesFromEncryptedDisk) +{ + { + writeFile(encrypted_disk, "a.txt"); + BackupEntryPtr entries[] + = {std::make_shared(encrypted_disk, "a.txt", /* copy_encrypted= */ true), + std::make_shared(encrypted_disk, "a.txt", /* copy_encrypted= */ true), + std::make_shared(encrypted_disk, "a.txt", /* copy_encrypted= */ true)}; + + auto encrypted_checksum = getChecksum(entries[0]); + EXPECT_NE(encrypted_checksum, NO_CHECKSUM); + EXPECT_NE(encrypted_checksum, ZERO_CHECKSUM); + EXPECT_NE(encrypted_checksum, SOME_TEXT_CHECKSUM); + + auto partial_checksum = getPartialChecksum(entries[1], 9); + EXPECT_NE(partial_checksum, NO_CHECKSUM); + EXPECT_NE(partial_checksum, ZERO_CHECKSUM); + EXPECT_NE(partial_checksum, SOME_TEXT_CHECKSUM); + EXPECT_NE(partial_checksum, encrypted_checksum); + + auto encrypted_data = readAll(entries[0]); + EXPECT_EQ(encrypted_data.size(), 9 + FileEncryption::Header::kSize); + + for (const auto & entry : entries) + { + EXPECT_EQ(entry->getSize(), 9 + FileEncryption::Header::kSize); + EXPECT_EQ(getChecksum(entry), encrypted_checksum); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + auto encrypted_checksum_9 = getPartialChecksum(entry, 9); + EXPECT_TRUE(encrypted_checksum_9 == NO_CHECKSUM || encrypted_checksum_9 == partial_checksum); + EXPECT_EQ(getPartialChecksum(entry, 9 + FileEncryption::Header::kSize), encrypted_checksum); + EXPECT_EQ(getPartialChecksum(entry, 1000), encrypted_checksum); + EXPECT_EQ(readAll(entry), encrypted_data); + } + } + + { + writeEmptyFile(encrypted_disk, "empty.txt"); + BackupEntryPtr entries[] + = {std::make_shared(encrypted_disk, "empty.txt", /* copy_encrypted= */ true), + std::make_shared(encrypted_disk, "empty.txt", /* copy_encrypted= */ true), + std::make_shared(encrypted_disk, "empty.txt", /* copy_encrypted= */ true)}; + for (const auto & entry : entries) + { + EXPECT_EQ(entry->getSize(), 0); + EXPECT_EQ(getChecksum(entry), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(entry, 1), ZERO_CHECKSUM); + EXPECT_EQ(readAll(entry), ""); + } + } + + { + auto precalculated_entry = std::make_shared(encrypted_disk, "a.txt", /* copy_encrypted= */ true, PRECALCULATED_SIZE, PRECALCULATED_CHECKSUM_UINT128); + EXPECT_EQ(precalculated_entry->getSize(), PRECALCULATED_SIZE + FileEncryption::Header::kSize); + + auto encrypted_checksum = getChecksum(precalculated_entry); + EXPECT_NE(encrypted_checksum, NO_CHECKSUM); + EXPECT_NE(encrypted_checksum, ZERO_CHECKSUM); + EXPECT_NE(encrypted_checksum, SOME_TEXT_CHECKSUM); + EXPECT_NE(encrypted_checksum, PRECALCULATED_CHECKSUM); + + EXPECT_EQ(getPartialChecksum(precalculated_entry, 0), ZERO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 1), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, PRECALCULATED_SIZE), NO_CHECKSUM); + EXPECT_EQ(getPartialChecksum(precalculated_entry, PRECALCULATED_SIZE + FileEncryption::Header::kSize), encrypted_checksum); + EXPECT_EQ(getPartialChecksum(precalculated_entry, 1000), encrypted_checksum); + + auto encrypted_data = readAll(precalculated_entry); + EXPECT_EQ(encrypted_data.size(), 9 + FileEncryption::Header::kSize); + } +} diff --git a/src/IO/ReadBufferFromMemory.cpp b/src/IO/ReadBufferFromMemory.cpp index ea0054f24d4..ede2c531e43 100644 --- a/src/IO/ReadBufferFromMemory.cpp +++ b/src/IO/ReadBufferFromMemory.cpp @@ -12,7 +12,7 @@ off_t ReadBufferFromMemory::seek(off_t offset, int whence) { if (whence == SEEK_SET) { - if (offset >= 0 && internal_buffer.begin() + offset < internal_buffer.end()) + if (offset >= 0 && internal_buffer.begin() + offset <= internal_buffer.end()) { pos = internal_buffer.begin() + offset; working_buffer = internal_buffer; /// We need to restore `working_buffer` in case the position was at EOF before this seek(). @@ -25,7 +25,7 @@ off_t ReadBufferFromMemory::seek(off_t offset, int whence) else if (whence == SEEK_CUR) { Position new_pos = pos + offset; - if (new_pos >= internal_buffer.begin() && new_pos < internal_buffer.end()) + if (new_pos >= internal_buffer.begin() && new_pos <= internal_buffer.end()) { pos = new_pos; working_buffer = internal_buffer; /// We need to restore `working_buffer` in case the position was at EOF before this seek().