From dd5435501a510f591b0742c0a18426c25f0ecbac Mon Sep 17 00:00:00 2001 From: Javi santana bot Date: Fri, 20 Dec 2019 11:00:21 +0100 Subject: [PATCH 01/85] check for active replicas when waiting for commands --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index b3f69d463f2..88c731fb88f 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3844,9 +3844,15 @@ void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const Repli { LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name); - Strings replicas = getZooKeeper()->getChildren(zookeeper_path + "/replicas"); + auto zookeeper = getZooKeeper() + Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); for (const String & replica : replicas) - waitForReplicaToProcessLogEntry(replica, entry); + { + if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + { + waitForReplicaToProcessLogEntry(replica, entry); + } + } LOG_DEBUG(log, "Finished waiting for all replicas to process " << entry.znode_name); } From a5a8372713e28c67af195b0dfbae63b9e8db01fd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Dec 2019 15:58:09 +0300 Subject: [PATCH 02/85] Update pipelineExecutor --- .../Processors/Executors/PipelineExecutor.cpp | 200 +++++++++--------- .../Processors/Executors/PipelineExecutor.h | 6 +- 2 files changed, 105 insertions(+), 101 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 6addec11975..7bd098cd43a 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -222,18 +222,18 @@ void PipelineExecutor::expandPipeline(Stack & stack, UInt64 pid) } } -bool PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Stack & stack) +void PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, size_t thread_number) { /// In this method we have ownership on edge, but node can be concurrently accessed. auto & node = graph[edge.to]; - std::lock_guard guard(node.status_mutex); + std::unique_lock lock(node.status_mutex); ExecStatus status = node.status; if (status == ExecStatus::Finished) - return false; + return; if (edge.backward) node.updated_output_ports.push_back(edge.output_port_number); @@ -243,14 +243,11 @@ bool PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Stack & stac if (status == ExecStatus::Idle) { node.status = ExecStatus::Preparing; - stack.push(edge.to); - return true; + prepareProcessor(edge.to, thread_number, std::move(lock)); } - - return false; } -bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & parents, size_t thread_number, bool async) +bool PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, std::unique_lock node_lock) { /// In this method we have ownership on node. auto & node = graph[pid]; @@ -264,7 +261,7 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & pa { /// Stopwatch watch; - std::lock_guard guard(node.status_mutex); + std::unique_lock lock(std::move(node_lock)); auto status = node.processor->prepare(node.updated_input_ports, node.updated_output_ports); node.updated_input_ports.clear(); @@ -303,9 +300,7 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & pa } case IProcessor::Status::Wait: { - if (!async) - throw Exception("Processor returned status Wait before Async.", ErrorCodes::LOGICAL_ERROR); - break; + throw Exception("Wait is temporary not supported.", ErrorCodes::LOGICAL_ERROR); } case IProcessor::Status::ExpandPipeline: { @@ -337,18 +332,20 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & pa if (need_traverse) { - for (auto & edge : updated_back_edges) - tryAddProcessorToStackIfUpdated(*edge, parents); - for (auto & edge : updated_direct_edges) - tryAddProcessorToStackIfUpdated(*edge, children); + tryAddProcessorToStackIfUpdated(*edge, thread_number); + + for (auto & edge : updated_back_edges) + tryAddProcessorToStackIfUpdated(*edge, thread_number); } if (need_expand_pipeline) { + Stack stack; + executor_contexts[thread_number]->task_list.emplace_back( node.execution_state.get(), - &parents + &stack ); ExpandPipelineTask * desired = &executor_contexts[thread_number]->task_list.back(); @@ -363,7 +360,14 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & pa doExpandPipeline(desired, true); /// Add itself back to be prepared again. - children.push(pid); + stack.push(pid); + + while (!stack.empty()) + { + auto item = stack.top(); + prepareProcessor(item, thread_number, std::unique_lock(graph[item].status_mutex)); + stack.pop(); + } } return false; @@ -475,11 +479,11 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads Stopwatch total_time_watch; ExecutionState * state = nullptr; - auto prepare_processor = [&](UInt64 pid, Stack & children, Stack & parents) + auto prepare_processor = [&](UInt64 pid) { try { - return prepareProcessor(pid, children, parents, thread_num, false); + return prepareProcessor(pid, thread_num, std::unique_lock(graph[pid].status_mutex)); } catch (...) { @@ -492,14 +496,14 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads using Queue = std::queue; - auto prepare_all_processors = [&](Queue & queue, Stack & stack, Stack & children, Stack & parents) + auto prepare_all_processors = [&](Queue & queue) { while (!stack.empty() && !finished) { auto current_processor = stack.top(); stack.pop(); - if (prepare_processor(current_processor, children, parents)) + if (prepare_processor(current_processor)) queue.push(graph[current_processor].execution_state.get()); } }; @@ -511,62 +515,62 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads executor_contexts[executor]->condvar.notify_one(); }; - auto process_pinned_tasks = [&](Queue & queue) - { - Queue tmp_queue; - - struct PinnedTask - { - ExecutionState * task; - size_t thread_num; - }; - - std::stack pinned_tasks; - - while (!queue.empty()) - { - auto task = queue.front(); - queue.pop(); - - auto stream = task->processor->getStream(); - if (stream != IProcessor::NO_STREAM) - pinned_tasks.push({.task = task, .thread_num = stream % num_threads}); - else - tmp_queue.push(task); - } - - if (!pinned_tasks.empty()) - { - std::stack threads_to_wake; - - { - std::lock_guard lock(task_queue_mutex); - - while (!pinned_tasks.empty()) - { - auto & pinned_task = pinned_tasks.top(); - auto thread = pinned_task.thread_num; - - executor_contexts[thread]->pinned_tasks.push(pinned_task.task); - pinned_tasks.pop(); - - if (threads_queue.has(thread)) - { - threads_queue.pop(thread); - threads_to_wake.push(thread); - } - } - } - - while (!threads_to_wake.empty()) - { - wake_up_executor(threads_to_wake.top()); - threads_to_wake.pop(); - } - } - - queue.swap(tmp_queue); - }; +// auto process_pinned_tasks = [&](Queue & queue) +// { +// Queue tmp_queue; +// +// struct PinnedTask +// { +// ExecutionState * task; +// size_t thread_num; +// }; +// +// std::stack pinned_tasks; +// +// while (!queue.empty()) +// { +// auto task = queue.front(); +// queue.pop(); +// +// auto stream = task->processor->getStream(); +// if (stream != IProcessor::NO_STREAM) +// pinned_tasks.push({.task = task, .thread_num = stream % num_threads}); +// else +// tmp_queue.push(task); +// } +// +// if (!pinned_tasks.empty()) +// { +// std::stack threads_to_wake; +// +// { +// std::lock_guard lock(task_queue_mutex); +// +// while (!pinned_tasks.empty()) +// { +// auto & pinned_task = pinned_tasks.top(); +// auto thread = pinned_task.thread_num; +// +// executor_contexts[thread]->pinned_tasks.push(pinned_task.task); +// pinned_tasks.pop(); +// +// if (threads_queue.has(thread)) +// { +// threads_queue.pop(thread); +// threads_to_wake.push(thread); +// } +// } +// } +// +// while (!threads_to_wake.empty()) +// { +// wake_up_executor(threads_to_wake.top()); +// threads_to_wake.pop(); +// } +// } +// +// queue.swap(tmp_queue); +// }; while (!finished) { @@ -577,13 +581,13 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads { std::unique_lock lock(task_queue_mutex); - if (!executor_contexts[thread_num]->pinned_tasks.empty()) - { - state = executor_contexts[thread_num]->pinned_tasks.front(); - executor_contexts[thread_num]->pinned_tasks.pop(); - - break; - } +// if (!executor_contexts[thread_num]->pinned_tasks.empty()) +// { +// state = executor_contexts[thread_num]->pinned_tasks.front(); +// executor_contexts[thread_num]->pinned_tasks.pop(); +// +// break; +// } if (!task_queue.empty()) { @@ -662,7 +666,7 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads /// Process all neighbours. Children will be on the top of stack, then parents. prepare_all_processors(queue, children, children, parents); - process_pinned_tasks(queue); + //process_pinned_tasks(queue); /// Take local task from queue if has one. if (!state && !queue.empty()) @@ -672,20 +676,20 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads } prepare_all_processors(queue, parents, parents, parents); - process_pinned_tasks(queue); + //process_pinned_tasks(queue); /// Take pinned task if has one. - { - std::lock_guard guard(task_queue_mutex); - if (!executor_contexts[thread_num]->pinned_tasks.empty()) - { - if (state) - queue.push(state); - - state = executor_contexts[thread_num]->pinned_tasks.front(); - executor_contexts[thread_num]->pinned_tasks.pop(); - } - } +// { +// std::lock_guard guard(task_queue_mutex); +// if (!executor_contexts[thread_num]->pinned_tasks.empty()) +// { +// if (state) +// queue.push(state); +// +// state = executor_contexts[thread_num]->pinned_tasks.front(); +// executor_contexts[thread_num]->pinned_tasks.pop(); +// } +// } /// Push other tasks to global queue. if (!queue.empty()) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.h b/dbms/src/Processors/Executors/PipelineExecutor.h index aded3de3008..5601f7612d9 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.h +++ b/dbms/src/Processors/Executors/PipelineExecutor.h @@ -173,7 +173,7 @@ private: std::mutex mutex; bool wake_flag = false; - std::queue pinned_tasks; + /// std::queue pinned_tasks; }; std::vector> executor_contexts; @@ -190,14 +190,14 @@ private: /// Pipeline execution related methods. void addChildlessProcessorsToStack(Stack & stack); - bool tryAddProcessorToStackIfUpdated(Edge & edge, Stack & stack); + void tryAddProcessorToStackIfUpdated(Edge & edge, size_t thread_number); static void addJob(ExecutionState * execution_state); // TODO: void addAsyncJob(UInt64 pid); /// Prepare processor with pid number. /// Check parents and children of current processor and push them to stacks if they also need to be prepared. /// If processor wants to be expanded, ExpandPipelineTask from thread_number's execution context will be used. - bool prepareProcessor(UInt64 pid, Stack & children, Stack & parents, size_t thread_number, bool async); + bool prepareProcessor(UInt64 pid, size_t thread_number, std::unique_lock node_lock); void doExpandPipeline(ExpandPipelineTask * task, bool processing); void executeImpl(size_t num_threads); From 3b2d6a66d1871c8c34b8c096a818e975c1d4ab0b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Dec 2019 17:23:34 +0300 Subject: [PATCH 03/85] Update pipelineExecutor --- .../Processors/Executors/PipelineExecutor.cpp | 67 +++++++++---------- .../Processors/Executors/PipelineExecutor.h | 6 +- 2 files changed, 37 insertions(+), 36 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 7bd098cd43a..4a26f681b73 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -222,7 +222,7 @@ void PipelineExecutor::expandPipeline(Stack & stack, UInt64 pid) } } -void PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, size_t thread_number) +void PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Queue & queue, size_t thread_number) { /// In this method we have ownership on edge, but node can be concurrently accessed. @@ -243,11 +243,11 @@ void PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, size_t threa if (status == ExecStatus::Idle) { node.status = ExecStatus::Preparing; - prepareProcessor(edge.to, thread_number, std::move(lock)); + prepareProcessor(edge.to, thread_number, queue, std::move(lock)); } } -bool PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, std::unique_lock node_lock) +void PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, Queue & queue, std::unique_lock node_lock) { /// In this method we have ownership on node. auto & node = graph[pid]; @@ -288,7 +288,8 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, std::u case IProcessor::Status::Ready: { node.status = ExecStatus::Executing; - return true; + queue.push(node.execution_state.get()); + break; } case IProcessor::Status::Async: { @@ -333,10 +334,10 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, std::u if (need_traverse) { for (auto & edge : updated_direct_edges) - tryAddProcessorToStackIfUpdated(*edge, thread_number); + tryAddProcessorToStackIfUpdated(*edge, queue, thread_number); for (auto & edge : updated_back_edges) - tryAddProcessorToStackIfUpdated(*edge, thread_number); + tryAddProcessorToStackIfUpdated(*edge, queue, thread_number); } if (need_expand_pipeline) @@ -365,12 +366,10 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, std::u while (!stack.empty()) { auto item = stack.top(); - prepareProcessor(item, thread_number, std::unique_lock(graph[item].status_mutex)); + prepareProcessor(item, thread_number, queue, std::unique_lock(graph[item].status_mutex)); stack.pop(); } } - - return false; } void PipelineExecutor::doExpandPipeline(ExpandPipelineTask * task, bool processing) @@ -479,34 +478,31 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads Stopwatch total_time_watch; ExecutionState * state = nullptr; - auto prepare_processor = [&](UInt64 pid) + auto prepare_processor = [&](UInt64 pid, Queue & queue) { try { - return prepareProcessor(pid, thread_num, std::unique_lock(graph[pid].status_mutex)); + prepareProcessor(pid, thread_num, queue, std::unique_lock(graph[pid].status_mutex)); } catch (...) { graph[pid].execution_state->exception = std::current_exception(); finish(); } - - return false; }; - using Queue = std::queue; - auto prepare_all_processors = [&](Queue & queue) - { - while (!stack.empty() && !finished) - { - auto current_processor = stack.top(); - stack.pop(); - - if (prepare_processor(current_processor)) - queue.push(graph[current_processor].execution_state.get()); - } - }; +// auto prepare_all_processors = [&](Queue & queue) +// { +// while (!stack.empty() && !finished) +// { +// auto current_processor = stack.top(); +// stack.pop(); +// +// if (prepare_processor(current_processor)) +// queue.push(graph[current_processor].execution_state.get()); +// } +// }; auto wake_up_executor = [&](size_t executor) { @@ -648,12 +644,12 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads if (finished) break; + state = nullptr; + // Stopwatch processing_time_watch; /// Try to execute neighbour processor. { - Stack children; - Stack parents; Queue queue; ++num_processing_executors; @@ -661,11 +657,10 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads doExpandPipeline(task, true); /// Execute again if can. - if (!prepare_processor(state->processors_id, children, parents)) - state = nullptr; + prepare_processor(state->processors_id, queue); /// Process all neighbours. Children will be on the top of stack, then parents. - prepare_all_processors(queue, children, children, parents); + /// prepare_all_processors(queue, children, children, parents); //process_pinned_tasks(queue); /// Take local task from queue if has one. @@ -675,7 +670,7 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads queue.pop(); } - prepare_all_processors(queue, parents, parents, parents); + /// prepare_all_processors(queue, parents, parents, parents); //process_pinned_tasks(queue); /// Take pinned task if has one. @@ -767,15 +762,19 @@ void PipelineExecutor::executeImpl(size_t num_threads) { std::lock_guard lock(task_queue_mutex); + Queue queue; + while (!stack.empty()) { UInt64 proc = stack.top(); stack.pop(); - if (prepareProcessor(proc, stack, stack, 0, false)) + prepareProcessor(proc, 0, queue, std::unique_lock(graph[proc].status_mutex)); + + while (!queue.empty()) { - auto cur_state = graph[proc].execution_state.get(); - task_queue.push(cur_state); + task_queue.push(queue.front()); + queue.pop(); } } } diff --git a/dbms/src/Processors/Executors/PipelineExecutor.h b/dbms/src/Processors/Executors/PipelineExecutor.h index 5601f7612d9..f6e5284699a 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.h +++ b/dbms/src/Processors/Executors/PipelineExecutor.h @@ -188,16 +188,18 @@ private: void buildGraph(); void expandPipeline(Stack & stack, UInt64 pid); + using Queue = std::queue; + /// Pipeline execution related methods. void addChildlessProcessorsToStack(Stack & stack); - void tryAddProcessorToStackIfUpdated(Edge & edge, size_t thread_number); + void tryAddProcessorToStackIfUpdated(Edge & edge, Queue & queue, size_t thread_number); static void addJob(ExecutionState * execution_state); // TODO: void addAsyncJob(UInt64 pid); /// Prepare processor with pid number. /// Check parents and children of current processor and push them to stacks if they also need to be prepared. /// If processor wants to be expanded, ExpandPipelineTask from thread_number's execution context will be used. - bool prepareProcessor(UInt64 pid, size_t thread_number, std::unique_lock node_lock); + void prepareProcessor(UInt64 pid, size_t thread_number, Queue & queue, std::unique_lock node_lock); void doExpandPipeline(ExpandPipelineTask * task, bool processing); void executeImpl(size_t num_threads); From fbbab2486f0313b94567b3ef07d781d9f6541738 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Dec 2019 17:47:48 +0300 Subject: [PATCH 04/85] Update pipelineExecutor --- dbms/src/Processors/Executors/PipelineExecutor.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 4a26f681b73..85f370ab21d 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -644,8 +644,6 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads if (finished) break; - state = nullptr; - // Stopwatch processing_time_watch; /// Try to execute neighbour processor. @@ -658,6 +656,7 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads /// Execute again if can. prepare_processor(state->processors_id, queue); + state = nullptr; /// Process all neighbours. Children will be on the top of stack, then parents. /// prepare_all_processors(queue, children, children, parents); From 9417f672b0e20b5bec84b7ed31cb39864c2bb66e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Dec 2019 18:51:25 +0300 Subject: [PATCH 05/85] Enable processors by default. --- dbms/src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index db1b3e6da59..cc12f11427f 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -364,7 +364,7 @@ struct Settings : public SettingsCollection M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql' and 'odbc' table functions.", 0) \ M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.", 0) \ \ - M(SettingBool, experimental_use_processors, false, "Use processors pipeline.", 0) \ + M(SettingBool, experimental_use_processors, true, "Use processors pipeline.", 0) \ \ M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \ M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.", 0) \ From 9d9ab833b767c0b46181bf9ff30046afd70c3c39 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Dec 2019 19:15:31 +0300 Subject: [PATCH 06/85] Add threads quota for processors. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 3 +++ dbms/src/Processors/Executors/PipelineExecutor.cpp | 14 ++++++++++++-- dbms/src/Processors/Executors/PipelineExecutor.h | 3 +++ dbms/src/Processors/IProcessor.h | 5 +++++ dbms/src/Processors/Pipe.cpp | 9 +++++++++ dbms/src/Processors/Pipe.h | 2 ++ 6 files changed, 34 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 32f314c5007..19434fd06d9 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1789,6 +1789,9 @@ void InterpreterSelectQuery::executeFetchColumns( // pipes[i].pinSources(i); // } + for (auto & pipe : pipes) + pipe.enableQuota(); + pipeline.init(std::move(pipes)); } else diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 85f370ab21d..e590d9dbff1 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -590,7 +590,10 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads state = task_queue.front(); task_queue.pop(); - if (!task_queue.empty() && !threads_queue.empty()) + if (state->has_quota) + --task_quota; + + if (!task_queue.empty() && !threads_queue.empty() && task_quota > threads_queue.size()) { auto thread_to_wake = threads_queue.pop_any(); lock.unlock(); @@ -693,14 +696,19 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads while (!queue.empty() && !finished) { task_queue.push(queue.front()); + if (queue.front()->has_quota) + ++task_quota; queue.pop(); } if (!threads_queue.empty()) { auto thread_to_wake = threads_queue.pop_any(); + bool wake_up = task_quota > threads_queue.size(); lock.unlock(); - wake_up_executor(thread_to_wake); + + if (wake_up) + wake_up_executor(thread_to_wake); } } @@ -773,6 +781,8 @@ void PipelineExecutor::executeImpl(size_t num_threads) while (!queue.empty()) { task_queue.push(queue.front()); + if (queue.front()->has_quota) + ++task_quota; queue.pop(); } } diff --git a/dbms/src/Processors/Executors/PipelineExecutor.h b/dbms/src/Processors/Executors/PipelineExecutor.h index f6e5284699a..168c0ed2b56 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.h +++ b/dbms/src/Processors/Executors/PipelineExecutor.h @@ -84,6 +84,7 @@ private: IProcessor * processor = nullptr; UInt64 processors_id = 0; + bool has_quota = false; /// Counters for profiling. size_t num_executed_jobs = 0; @@ -117,6 +118,7 @@ private: execution_state = std::make_unique(); execution_state->processor = processor; execution_state->processors_id = processor_id; + execution_state->has_quota = processor->hasQuota(); } Node(Node && other) noexcept @@ -137,6 +139,7 @@ private: /// Queue with pointers to tasks. Each thread will concurrently read from it until finished flag is set. /// Stores processors need to be prepared. Preparing status is already set for them. TaskQueue task_queue; + size_t task_quota = 0; ThreadsQueue threads_queue; std::mutex task_queue_mutex; diff --git a/dbms/src/Processors/IProcessor.h b/dbms/src/Processors/IProcessor.h index 852bde2d467..5296f36de87 100644 --- a/dbms/src/Processors/IProcessor.h +++ b/dbms/src/Processors/IProcessor.h @@ -272,12 +272,17 @@ public: size_t getStream() const { return stream_number; } constexpr static size_t NO_STREAM = std::numeric_limits::max(); + void enableQuota() { has_quota = true; } + bool hasQuota() const { return has_quota; } + private: std::atomic is_cancelled{false}; std::string processor_description; size_t stream_number = NO_STREAM; + + bool has_quota = false; }; diff --git a/dbms/src/Processors/Pipe.cpp b/dbms/src/Processors/Pipe.cpp index 17b44a48ea1..27daadac8a0 100644 --- a/dbms/src/Processors/Pipe.cpp +++ b/dbms/src/Processors/Pipe.cpp @@ -115,4 +115,13 @@ void Pipe::pinSources(size_t executor_number) } } +void Pipe::enableQuota() +{ + for (auto & processor : processors) + { + if (auto * source = dynamic_cast(processor.get())) + source->enableQuota(); + } +} + } diff --git a/dbms/src/Processors/Pipe.h b/dbms/src/Processors/Pipe.h index d734c89f485..3d121d3b2e3 100644 --- a/dbms/src/Processors/Pipe.h +++ b/dbms/src/Processors/Pipe.h @@ -42,6 +42,8 @@ public: /// Set information about preferred executor number for sources. void pinSources(size_t executor_number); + void enableQuota(); + void setTotalsPort(OutputPort * totals_) { totals = totals_; } OutputPort * getTotalsPort() const { return totals; } From bb19e22bd73e0dc187ed3dc8d471c93178c93e61 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Dec 2019 19:34:08 +0300 Subject: [PATCH 07/85] Add threads quota for processors. --- dbms/src/Processors/Executors/PipelineExecutor.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index e590d9dbff1..15ec4d66ff8 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -701,14 +701,12 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads queue.pop(); } - if (!threads_queue.empty()) + if (!threads_queue.empty() && task_quota > threads_queue.size()) { auto thread_to_wake = threads_queue.pop_any(); - bool wake_up = task_quota > threads_queue.size(); lock.unlock(); - if (wake_up) - wake_up_executor(thread_to_wake); + wake_up_executor(thread_to_wake); } } From 5a5d15c095d45a859942780fe32a35c7ee8fe4de Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Dec 2019 19:52:15 +0300 Subject: [PATCH 08/85] Enable quota for more processors. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 10 ++++++++++ dbms/src/Processors/QueryPipeline.cpp | 6 ++++++ dbms/src/Processors/QueryPipeline.h | 2 ++ 3 files changed, 18 insertions(+) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 19434fd06d9..7dd9aec7ee5 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1970,6 +1970,8 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const return std::make_shared(header, transform_params); }); } + + pipeline.enableQuotaForCurrentStreams(); } @@ -2083,6 +2085,8 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPipeline & pipeline, bo pipeline.addPipe(std::move(pipe)); } + + pipeline.enableQuotaForCurrentStreams(); } @@ -2316,6 +2320,8 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting pipeline.addPipe({ std::move(transform) }); } + pipeline.enableQuotaForCurrentStreams(); + if (need_finish_sorting) { pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) @@ -2355,6 +2361,8 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting settings.max_bytes_before_remerge_sort, settings.max_bytes_before_external_sort, context->getTemporaryPath(), settings.min_free_disk_space_for_temporary_data); }); + + pipeline.enableQuotaForCurrentStreams(); } @@ -2416,6 +2424,8 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline, const settings.max_block_size, limit); pipeline.addPipe({ std::move(transform) }); + + pipeline.enableQuotaForCurrentStreams(); } } diff --git a/dbms/src/Processors/QueryPipeline.cpp b/dbms/src/Processors/QueryPipeline.cpp index fd75d7f57cf..df10fae9a38 100644 --- a/dbms/src/Processors/QueryPipeline.cpp +++ b/dbms/src/Processors/QueryPipeline.cpp @@ -278,6 +278,12 @@ void QueryPipeline::resize(size_t num_streams, bool force) processors.emplace_back(std::move(resize)); } +void QueryPipeline::enableQuotaForCurrentStreams() +{ + for (auto & stream : streams) + stream->getProcessor().enableQuota(); +} + void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform) { checkInitialized(); diff --git a/dbms/src/Processors/QueryPipeline.h b/dbms/src/Processors/QueryPipeline.h index e32ed6a0abe..c27e570018f 100644 --- a/dbms/src/Processors/QueryPipeline.h +++ b/dbms/src/Processors/QueryPipeline.h @@ -63,6 +63,8 @@ public: void resize(size_t num_streams, bool force = false); + void enableQuotaForCurrentStreams(); + void unitePipelines(std::vector && pipelines, const Block & common_header, const Context & context); PipelineExecutorPtr execute(); From 0034afdf10644938171775adfc5c3951ce3b3679 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Dec 2019 21:23:31 +0300 Subject: [PATCH 09/85] Better task queue for PipelineExecutor. --- .../Processors/Executors/PipelineExecutor.cpp | 26 ++++----- .../Processors/Executors/PipelineExecutor.h | 55 ++++++++++++++++++- 2 files changed, 65 insertions(+), 16 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 15ec4d66ff8..d041bd2465c 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -587,13 +587,9 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads if (!task_queue.empty()) { - state = task_queue.front(); - task_queue.pop(); + state = task_queue.pop(thread_num); - if (state->has_quota) - --task_quota; - - if (!task_queue.empty() && !threads_queue.empty() && task_quota > threads_queue.size()) + if (!task_queue.empty() && !threads_queue.empty() && task_queue.quota() > threads_queue.size()) { auto thread_to_wake = threads_queue.pop_any(); lock.unlock(); @@ -666,7 +662,7 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads //process_pinned_tasks(queue); /// Take local task from queue if has one. - if (!state && !queue.empty()) + if (!queue.empty()) { state = queue.front(); queue.pop(); @@ -695,13 +691,11 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads while (!queue.empty() && !finished) { - task_queue.push(queue.front()); - if (queue.front()->has_quota) - ++task_quota; + task_queue.push(queue.front(), thread_num); queue.pop(); } - if (!threads_queue.empty() && task_quota > threads_queue.size()) + if (!threads_queue.empty() && task_queue.quota() > threads_queue.size()) { auto thread_to_wake = threads_queue.pop_any(); lock.unlock(); @@ -734,6 +728,7 @@ void PipelineExecutor::executeImpl(size_t num_threads) Stack stack; threads_queue.init(num_threads); + task_queue.init(num_threads); { std::lock_guard guard(executor_contexts_mutex); @@ -768,6 +763,7 @@ void PipelineExecutor::executeImpl(size_t num_threads) std::lock_guard lock(task_queue_mutex); Queue queue; + size_t next_thread = 0; while (!stack.empty()) { @@ -778,10 +774,12 @@ void PipelineExecutor::executeImpl(size_t num_threads) while (!queue.empty()) { - task_queue.push(queue.front()); - if (queue.front()->has_quota) - ++task_quota; + task_queue.push(queue.front(), next_thread); queue.pop(); + + ++next_thread; + if (next_thread >= num_threads) + next_thread = 0; } } } diff --git a/dbms/src/Processors/Executors/PipelineExecutor.h b/dbms/src/Processors/Executors/PipelineExecutor.h index 168c0ed2b56..d38ac4d255b 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.h +++ b/dbms/src/Processors/Executors/PipelineExecutor.h @@ -134,12 +134,63 @@ private: using Stack = std::stack; - using TaskQueue = std::queue; + class TaskQueue + { + public: + void init(size_t num_threads) { queues.resize(num_threads); } + + void push(ExecutionState * state, size_t thread_num) + { + queues[thread_num].push(state); + + ++size_; + + if (state->has_quota) + ++quota_; + } + + ExecutionState * pop(size_t thread_num) + { + if (size_ == 0) + throw Exception("TaskQueue is not empty.", ErrorCodes::LOGICAL_ERROR); + + for (size_t i = 0; i < queues.size(); ++i) + { + if (!queues[thread_num].empty()) + { + ExecutionState * state = queues[thread_num].front(); + queues[thread_num].pop(); + + --size_; + + if (state->has_quota) + ++quota_; + + return state; + } + + ++thread_num; + if (thread_num >= queues.size()) + thread_num = 0; + } + + throw Exception("TaskQueue is not empty.", ErrorCodes::LOGICAL_ERROR); + } + + size_t size() const { return size_; } + bool empty() const { return size_ == 0; } + size_t quota() const { return quota_; } + + private: + using Queue = std::queue; + std::vector queues; + size_t size_ = 0; + size_t quota_ = 0; + }; /// Queue with pointers to tasks. Each thread will concurrently read from it until finished flag is set. /// Stores processors need to be prepared. Preparing status is already set for them. TaskQueue task_queue; - size_t task_quota = 0; ThreadsQueue threads_queue; std::mutex task_queue_mutex; From 330676737a5b6c7f9f4de7df3ef5571aff9913af Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 27 Dec 2019 13:59:43 +0300 Subject: [PATCH 10/85] Disable quota . --- dbms/src/Processors/Executors/PipelineExecutor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index d041bd2465c..03c54be5a72 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -589,7 +589,7 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads { state = task_queue.pop(thread_num); - if (!task_queue.empty() && !threads_queue.empty() && task_queue.quota() > threads_queue.size()) + if (!task_queue.empty() && !threads_queue.empty() /*&& task_queue.quota() > threads_queue.size()*/) { auto thread_to_wake = threads_queue.pop_any(); lock.unlock(); @@ -695,7 +695,7 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads queue.pop(); } - if (!threads_queue.empty() && task_queue.quota() > threads_queue.size()) + if (!threads_queue.empty() /* && task_queue.quota() > threads_queue.size()*/) { auto thread_to_wake = threads_queue.pop_any(); lock.unlock(); From 05fe9ef179a24d7bead4773c31ab9b2f3bc239c3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 27 Dec 2019 21:42:47 +0300 Subject: [PATCH 11/85] Add sort improvement to processors. --- .../Transforms/SortingTransform.cpp | 34 +++++++------------ .../Processors/Transforms/SortingTransform.h | 8 ++--- 2 files changed, 17 insertions(+), 25 deletions(-) diff --git a/dbms/src/Processors/Transforms/SortingTransform.cpp b/dbms/src/Processors/Transforms/SortingTransform.cpp index ab87591c0d6..1062beb7dbe 100644 --- a/dbms/src/Processors/Transforms/SortingTransform.cpp +++ b/dbms/src/Processors/Transforms/SortingTransform.cpp @@ -41,15 +41,9 @@ MergeSorter::MergeSorter(Chunks chunks_, SortDescription & description_, size_t chunks.swap(nonempty_chunks); if (!has_collation) - { - for (auto & cursor : cursors) - queue_without_collation.push(SortCursor(&cursor)); - } + queue_without_collation = SortingHeap(cursors); else - { - for (auto & cursor : cursors) - queue_with_collation.push(SortCursorWithCollation(&cursor)); - } + queue_with_collation = SortingHeap(cursors); } @@ -66,13 +60,13 @@ Chunk MergeSorter::read() } return !has_collation - ? mergeImpl(queue_without_collation) - : mergeImpl(queue_with_collation); + ? mergeImpl(queue_without_collation) + : mergeImpl(queue_with_collation); } -template -Chunk MergeSorter::mergeImpl(std::priority_queue & queue) +template +Chunk MergeSorter::mergeImpl(TSortingHeap & queue) { size_t num_columns = chunks[0].getNumColumns(); @@ -81,29 +75,27 @@ Chunk MergeSorter::mergeImpl(std::priority_queue & queue) /// Take rows from queue in right order and push to 'merged'. size_t merged_rows = 0; - while (!queue.empty()) + while (queue.isValid()) { - TSortCursor current = queue.top(); - queue.pop(); + auto current = queue.current(); + /// Append a row from queue. for (size_t i = 0; i < num_columns; ++i) merged_columns[i]->insertFrom(*current->all_columns[i], current->pos); ++total_merged_rows; ++merged_rows; - if (!current->isLast()) - { - current->next(); - queue.push(current); - } - + /// We don't need more rows because of limit has reached. if (limit && total_merged_rows == limit) { chunks.clear(); return Chunk(std::move(merged_columns), merged_rows); } + queue.next(); + + /// It's enough for current output block but we will continue. if (merged_rows == max_merged_block_size) return Chunk(std::move(merged_columns), merged_rows); } diff --git a/dbms/src/Processors/Transforms/SortingTransform.h b/dbms/src/Processors/Transforms/SortingTransform.h index 2703501c81a..207f670acf3 100644 --- a/dbms/src/Processors/Transforms/SortingTransform.h +++ b/dbms/src/Processors/Transforms/SortingTransform.h @@ -32,14 +32,14 @@ private: bool has_collation = false; - std::priority_queue queue_without_collation; - std::priority_queue queue_with_collation; + SortingHeap queue_without_collation; + SortingHeap queue_with_collation; /** Two different cursors are supported - with and without Collation. * Templates are used (instead of virtual functions in SortCursor) for zero-overhead. */ - template - Chunk mergeImpl(std::priority_queue & queue); + template + Chunk mergeImpl(TSortingHeap & queue); }; From 755cd1bea24401e7706f68431ffebf474747b53d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 28 Dec 2019 21:42:07 +0300 Subject: [PATCH 12/85] Disable logging in processors pipeline. --- .../Processors/Executors/PipelineExecutor.cpp | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 03c54be5a72..a880f9fcb8e 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -470,12 +470,12 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads #endif - UInt64 total_time_ns = 0; - UInt64 execution_time_ns = 0; - UInt64 processing_time_ns = 0; - UInt64 wait_time_ns = 0; +// UInt64 total_time_ns = 0; +// UInt64 execution_time_ns = 0; +// UInt64 processing_time_ns = 0; +// UInt64 wait_time_ns = 0; - Stopwatch total_time_watch; +// Stopwatch total_time_watch; ExecutionState * state = nullptr; auto prepare_processor = [&](UInt64 pid, Queue & queue) @@ -713,14 +713,15 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads } } - total_time_ns = total_time_watch.elapsed(); - wait_time_ns = total_time_ns - execution_time_ns - processing_time_ns; - +// total_time_ns = total_time_watch.elapsed(); +// wait_time_ns = total_time_ns - execution_time_ns - processing_time_ns; +/* LOG_TRACE(log, "Thread finished." << " Total time: " << (total_time_ns / 1e9) << " sec." << " Execution time: " << (execution_time_ns / 1e9) << " sec." << " Processing time: " << (processing_time_ns / 1e9) << " sec." << " Wait time: " << (wait_time_ns / 1e9) << "sec."); +*/ } void PipelineExecutor::executeImpl(size_t num_threads) From d3060f42e0c50b49ea3c948920f355efc2a5c15e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 29 Dec 2019 10:30:26 +0300 Subject: [PATCH 13/85] Remove commented code. --- .../Processors/Executors/PipelineExecutor.cpp | 98 ------------------- 1 file changed, 98 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index a880f9fcb8e..2d9fae7a102 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -491,19 +491,6 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads } }; - -// auto prepare_all_processors = [&](Queue & queue) -// { -// while (!stack.empty() && !finished) -// { -// auto current_processor = stack.top(); -// stack.pop(); -// -// if (prepare_processor(current_processor)) -// queue.push(graph[current_processor].execution_state.get()); -// } -// }; - auto wake_up_executor = [&](size_t executor) { std::lock_guard guard(executor_contexts[executor]->mutex); @@ -511,63 +498,6 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads executor_contexts[executor]->condvar.notify_one(); }; -// auto process_pinned_tasks = [&](Queue & queue) -// { -// Queue tmp_queue; -// -// struct PinnedTask -// { -// ExecutionState * task; -// size_t thread_num; -// }; -// -// std::stack pinned_tasks; -// -// while (!queue.empty()) -// { -// auto task = queue.front(); -// queue.pop(); -// -// auto stream = task->processor->getStream(); -// if (stream != IProcessor::NO_STREAM) -// pinned_tasks.push({.task = task, .thread_num = stream % num_threads}); -// else -// tmp_queue.push(task); -// } -// -// if (!pinned_tasks.empty()) -// { -// std::stack threads_to_wake; -// -// { -// std::lock_guard lock(task_queue_mutex); -// -// while (!pinned_tasks.empty()) -// { -// auto & pinned_task = pinned_tasks.top(); -// auto thread = pinned_task.thread_num; -// -// executor_contexts[thread]->pinned_tasks.push(pinned_task.task); -// pinned_tasks.pop(); -// -// if (threads_queue.has(thread)) -// { -// threads_queue.pop(thread); -// threads_to_wake.push(thread); -// } -// } -// } -// -// while (!threads_to_wake.empty()) -// { -// wake_up_executor(threads_to_wake.top()); -// threads_to_wake.pop(); -// } -// } -// -// queue.swap(tmp_queue); -// }; - while (!finished) { /// First, find any processor to execute. @@ -577,14 +507,6 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads { std::unique_lock lock(task_queue_mutex); -// if (!executor_contexts[thread_num]->pinned_tasks.empty()) -// { -// state = executor_contexts[thread_num]->pinned_tasks.front(); -// executor_contexts[thread_num]->pinned_tasks.pop(); -// -// break; -// } - if (!task_queue.empty()) { state = task_queue.pop(thread_num); @@ -657,10 +579,6 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads prepare_processor(state->processors_id, queue); state = nullptr; - /// Process all neighbours. Children will be on the top of stack, then parents. - /// prepare_all_processors(queue, children, children, parents); - //process_pinned_tasks(queue); - /// Take local task from queue if has one. if (!queue.empty()) { @@ -668,22 +586,6 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads queue.pop(); } - /// prepare_all_processors(queue, parents, parents, parents); - //process_pinned_tasks(queue); - - /// Take pinned task if has one. -// { -// std::lock_guard guard(task_queue_mutex); -// if (!executor_contexts[thread_num]->pinned_tasks.empty()) -// { -// if (state) -// queue.push(state); -// -// state = executor_contexts[thread_num]->pinned_tasks.front(); -// executor_contexts[thread_num]->pinned_tasks.pop(); -// } -// } - /// Push other tasks to global queue. if (!queue.empty()) { From 84072fe9774865cc29fdb95e8be383b86662f025 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 29 Dec 2019 23:09:11 +0300 Subject: [PATCH 14/85] Try disable affinity. --- dbms/src/Processors/Executors/PipelineExecutor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 2d9fae7a102..2fdd256e97d 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -462,12 +462,13 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads #if !defined(__APPLE__) && !defined(__FreeBSD__) /// Specify CPU core for thread if can. /// It may reduce the number of context swithches. +/* cpu_set_t cpu_set; CPU_ZERO(&cpu_set); CPU_SET(thread_num, &cpu_set); if (sched_setaffinity(0, sizeof(cpu_set_t), &cpu_set) == -1) LOG_TRACE(log, "Cannot set affinity for thread " << num_threads); - +*/ #endif // UInt64 total_time_ns = 0; From 40b078452876d7f612538b98685c2bd05ac617de Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Dec 2019 00:07:20 +0300 Subject: [PATCH 15/85] Inline in sort cursor. --- dbms/src/Core/SortCursor.h | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/src/Core/SortCursor.h b/dbms/src/Core/SortCursor.h index 5b4db43024f..c0ac03fddee 100644 --- a/dbms/src/Core/SortCursor.h +++ b/dbms/src/Core/SortCursor.h @@ -119,7 +119,7 @@ struct SortCursor const SortCursorImpl * operator-> () const { return impl; } /// The specified row of this cursor is greater than the specified row of another cursor. - bool greaterAt(const SortCursor & rhs, size_t lhs_pos, size_t rhs_pos) const + bool ALWAYS_INLINE greaterAt(const SortCursor & rhs, size_t lhs_pos, size_t rhs_pos) const { for (size_t i = 0; i < impl->sort_columns_size; ++i) { @@ -135,7 +135,7 @@ struct SortCursor } /// Checks that all rows in the current block of this cursor are less than or equal to all the rows of the current block of another cursor. - bool totallyLessOrEquals(const SortCursor & rhs) const + bool ALWAYS_INLINE totallyLessOrEquals(const SortCursor & rhs) const { if (impl->rows == 0 || rhs.impl->rows == 0) return false; @@ -144,13 +144,13 @@ struct SortCursor return !greaterAt(rhs, impl->rows - 1, 0); } - bool greater(const SortCursor & rhs) const + bool ALWAYS_INLINE greater(const SortCursor & rhs) const { return greaterAt(rhs, impl->pos, rhs.impl->pos); } /// Inverted so that the priority queue elements are removed in ascending order. - bool operator< (const SortCursor & rhs) const + bool ALWAYS_INLINE operator< (const SortCursor & rhs) const { return greater(rhs); } @@ -233,7 +233,7 @@ public: Cursor & current() { return queue.front(); } - void next() + void ALWAYS_INLINE next() { assert(isValid()); @@ -253,7 +253,7 @@ private: /// This is adapted version of the function __sift_down from libc++. /// Why cannot simply use std::priority_queue? /// - because it doesn't support updating the top element and requires pop and push instead. - void updateTop() + void ALWAYS_INLINE updateTop() { size_t size = queue.size(); if (size < 2) From 1284d2eb4e81343626db8f825e7c85968b9e2a33 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Dec 2019 15:26:14 +0300 Subject: [PATCH 16/85] Try enable affinity --- dbms/src/Processors/Executors/PipelineExecutor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 2fdd256e97d..e704491fa72 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -462,13 +462,13 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads #if !defined(__APPLE__) && !defined(__FreeBSD__) /// Specify CPU core for thread if can. /// It may reduce the number of context swithches. -/* + cpu_set_t cpu_set; CPU_ZERO(&cpu_set); CPU_SET(thread_num, &cpu_set); if (sched_setaffinity(0, sizeof(cpu_set_t), &cpu_set) == -1) LOG_TRACE(log, "Cannot set affinity for thread " << num_threads); -*/ + #endif // UInt64 total_time_ns = 0; From 7ed3da8265b9bf09691d3f5dd6173633d1ef79a5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Dec 2019 20:31:47 +0300 Subject: [PATCH 17/85] Increase timeout for 00909_kill_not_initialized_query.sh. --- .../queries/0_stateless/00909_kill_not_initialized_query.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00909_kill_not_initialized_query.sh b/dbms/tests/queries/0_stateless/00909_kill_not_initialized_query.sh index 677709dd2c0..b5982fd0dc8 100755 --- a/dbms/tests/queries/0_stateless/00909_kill_not_initialized_query.sh +++ b/dbms/tests/queries/0_stateless/00909_kill_not_initialized_query.sh @@ -34,7 +34,7 @@ $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='$query_to_kill' ASYNC" &>/dev/nul sleep 1 # Kill $query_for_pending SYNC. This query is not blocker, so it should be killed fast. -timeout 10 $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='$query_for_pending' SYNC" &>/dev/null +timeout 20 $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='$query_for_pending' SYNC" &>/dev/null # Both queries have to be killed, doesn't matter with SYNC or ASYNC kill for run in {1..15} From 1138162dc8647eded841c392fbd67935e39e66ce Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Dec 2019 21:29:24 +0300 Subject: [PATCH 18/85] Better exceptions handling in PipelineExecutor. --- .../Processors/Executors/PipelineExecutor.cpp | 90 ++++++++++++------- .../Processors/Executors/PipelineExecutor.h | 8 +- 2 files changed, 61 insertions(+), 37 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index e704491fa72..b66ada72695 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -177,10 +177,20 @@ void PipelineExecutor::addJob(ExecutionState * execution_state) execution_state->job = std::move(job); } -void PipelineExecutor::expandPipeline(Stack & stack, UInt64 pid) +bool PipelineExecutor::expandPipeline(Stack & stack, UInt64 pid) { auto & cur_node = graph[pid]; - auto new_processors = cur_node.processor->expandPipeline(); + Processors new_processors; + + try + { + new_processors = cur_node.processor->expandPipeline(); + } + catch (...) + { + cur_node.execution_state->exception = std::current_exception(); + return false; + } for (const auto & processor : new_processors) { @@ -220,9 +230,11 @@ void PipelineExecutor::expandPipeline(Stack & stack, UInt64 pid) } } } + + return true; } -void PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Queue & queue, size_t thread_number) +bool PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Queue & queue, size_t thread_number) { /// In this method we have ownership on edge, but node can be concurrently accessed. @@ -233,7 +245,7 @@ void PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Queue & queu ExecStatus status = node.status; if (status == ExecStatus::Finished) - return; + return true; if (edge.backward) node.updated_output_ports.push_back(edge.output_port_number); @@ -243,11 +255,13 @@ void PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Queue & queu if (status == ExecStatus::Idle) { node.status = ExecStatus::Preparing; - prepareProcessor(edge.to, thread_number, queue, std::move(lock)); + return prepareProcessor(edge.to, thread_number, queue, std::move(lock)); } + + return true; } -void PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, Queue & queue, std::unique_lock node_lock) +bool PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, Queue & queue, std::unique_lock node_lock) { /// In this method we have ownership on node. auto & node = graph[pid]; @@ -263,12 +277,20 @@ void PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, Queue std::unique_lock lock(std::move(node_lock)); - auto status = node.processor->prepare(node.updated_input_ports, node.updated_output_ports); - node.updated_input_ports.clear(); - node.updated_output_ports.clear(); + try + { + node.last_processor_status = node.processor->prepare(node.updated_input_ports, node.updated_output_ports); + } + catch (...) + { + node.execution_state->exception = std::current_exception(); + return false; + } /// node.execution_state->preparation_time_ns += watch.elapsed(); - node.last_processor_status = status; + + node.updated_input_ports.clear(); + node.updated_output_ports.clear(); switch (node.last_processor_status) { @@ -334,10 +356,16 @@ void PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, Queue if (need_traverse) { for (auto & edge : updated_direct_edges) - tryAddProcessorToStackIfUpdated(*edge, queue, thread_number); + { + if (!tryAddProcessorToStackIfUpdated(*edge, queue, thread_number)) + return false; + } for (auto & edge : updated_back_edges) - tryAddProcessorToStackIfUpdated(*edge, queue, thread_number); + { + if (!tryAddProcessorToStackIfUpdated(*edge, queue, thread_number)) + return false; + } } if (need_expand_pipeline) @@ -354,25 +382,24 @@ void PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, Queue while (!expand_pipeline_task.compare_exchange_strong(expected, desired)) { - doExpandPipeline(expected, true); + if (!doExpandPipeline(expected, true)) + return false; + expected = nullptr; } - doExpandPipeline(desired, true); + if (!doExpandPipeline(desired, true)) + return false; - /// Add itself back to be prepared again. - stack.push(pid); - - while (!stack.empty()) - { - auto item = stack.top(); - prepareProcessor(item, thread_number, queue, std::unique_lock(graph[item].status_mutex)); - stack.pop(); - } + /// Prepare itself again. + if (!prepareProcessor(pid, thread_number, queue, std::unique_lock(graph[pid].status_mutex))) + return false; } + + return true; } -void PipelineExecutor::doExpandPipeline(ExpandPipelineTask * task, bool processing) +bool PipelineExecutor::doExpandPipeline(ExpandPipelineTask * task, bool processing) { std::unique_lock lock(task->mutex); @@ -384,16 +411,20 @@ void PipelineExecutor::doExpandPipeline(ExpandPipelineTask * task, bool processi return task->num_waiting_processing_threads >= num_processing_executors || expand_pipeline_task != task; }); + bool result = false; + /// After condvar.wait() task may point to trash. Can change it only if it is still in expand_pipeline_task. if (expand_pipeline_task == task) { - expandPipeline(*task->stack, task->node_to_expand->processors_id); + result = expandPipeline(*task->stack, task->node_to_expand->processors_id); expand_pipeline_task = nullptr; lock.unlock(); task->condvar.notify_all(); } + + return result; } void PipelineExecutor::cancel() @@ -481,15 +512,8 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads auto prepare_processor = [&](UInt64 pid, Queue & queue) { - try - { - prepareProcessor(pid, thread_num, queue, std::unique_lock(graph[pid].status_mutex)); - } - catch (...) - { - graph[pid].execution_state->exception = std::current_exception(); + if (!prepareProcessor(pid, thread_num, queue, std::unique_lock(graph[pid].status_mutex))) finish(); - } }; auto wake_up_executor = [&](size_t executor) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.h b/dbms/src/Processors/Executors/PipelineExecutor.h index d38ac4d255b..2231c19284b 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.h +++ b/dbms/src/Processors/Executors/PipelineExecutor.h @@ -240,21 +240,21 @@ private: /// Graph related methods. bool addEdges(UInt64 node); void buildGraph(); - void expandPipeline(Stack & stack, UInt64 pid); + bool expandPipeline(Stack & stack, UInt64 pid); using Queue = std::queue; /// Pipeline execution related methods. void addChildlessProcessorsToStack(Stack & stack); - void tryAddProcessorToStackIfUpdated(Edge & edge, Queue & queue, size_t thread_number); + bool tryAddProcessorToStackIfUpdated(Edge & edge, Queue & queue, size_t thread_number); static void addJob(ExecutionState * execution_state); // TODO: void addAsyncJob(UInt64 pid); /// Prepare processor with pid number. /// Check parents and children of current processor and push them to stacks if they also need to be prepared. /// If processor wants to be expanded, ExpandPipelineTask from thread_number's execution context will be used. - void prepareProcessor(UInt64 pid, size_t thread_number, Queue & queue, std::unique_lock node_lock); - void doExpandPipeline(ExpandPipelineTask * task, bool processing); + bool prepareProcessor(UInt64 pid, size_t thread_number, Queue & queue, std::unique_lock node_lock); + bool doExpandPipeline(ExpandPipelineTask * task, bool processing); void executeImpl(size_t num_threads); void executeSingleThread(size_t thread_num, size_t num_threads); From 96a732ac02b090613ab9b2d353fc3b9e26dcaba2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Dec 2019 22:45:27 +0300 Subject: [PATCH 19/85] Better exceptions handling in PipelineExecutor. --- dbms/src/Processors/Executors/PipelineExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index b66ada72695..4f7aa2e506e 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -411,7 +411,7 @@ bool PipelineExecutor::doExpandPipeline(ExpandPipelineTask * task, bool processi return task->num_waiting_processing_threads >= num_processing_executors || expand_pipeline_task != task; }); - bool result = false; + bool result = true; /// After condvar.wait() task may point to trash. Can change it only if it is still in expand_pipeline_task. if (expand_pipeline_task == task) From 42f86fe4f5381a4e75c3a242489854cc927b1b84 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Dec 2019 23:09:37 +0300 Subject: [PATCH 20/85] Better exceptions handling in PipelineExecutor. --- dbms/src/Processors/Executors/PipelineExecutor.cpp | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 4f7aa2e506e..a02cff3a1b9 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -391,9 +391,17 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, Queue if (!doExpandPipeline(desired, true)) return false; - /// Prepare itself again. - if (!prepareProcessor(pid, thread_number, queue, std::unique_lock(graph[pid].status_mutex))) - return false; + /// Add itself back to be prepared again. + stack.push(pid); + + while (!stack.empty()) + { + auto item = stack.top(); + if (!prepareProcessor(item, thread_number, queue, std::unique_lock(graph[item].status_mutex))) + return false; + + stack.pop(); + } } return true; From 04193460aaf4deb2f7f995afe3366dc7c173481c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 31 Dec 2019 12:10:44 +0300 Subject: [PATCH 21/85] Try disable affinity. --- dbms/src/Processors/Executors/PipelineExecutor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index a02cff3a1b9..fee1e37e885 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -501,13 +501,13 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads #if !defined(__APPLE__) && !defined(__FreeBSD__) /// Specify CPU core for thread if can. /// It may reduce the number of context swithches. - +/* cpu_set_t cpu_set; CPU_ZERO(&cpu_set); CPU_SET(thread_num, &cpu_set); if (sched_setaffinity(0, sizeof(cpu_set_t), &cpu_set) == -1) LOG_TRACE(log, "Cannot set affinity for thread " << num_threads); - +*/ #endif // UInt64 total_time_ns = 0; From 65a00150b2670c6d0309cb73040fd1e567e6b626 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 1 Jan 2020 13:42:46 +0300 Subject: [PATCH 22/85] Update num threads in processors. --- dbms/programs/server/TCPHandler.cpp | 2 +- dbms/src/Processors/QueryPipeline.cpp | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index cb215eb0af8..dda3ff3f561 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -546,7 +546,7 @@ void TCPHandler::processOrdinaryQueryWithProcessors(size_t num_threads) auto & pipeline = state.io.pipeline; if (pipeline.getMaxThreads()) - num_threads = pipeline.getMaxThreads(); + num_threads = std::min(num_threads, pipeline.getMaxThreads()); /// Send header-block, to allow client to prepare output format for data to send. { diff --git a/dbms/src/Processors/QueryPipeline.cpp b/dbms/src/Processors/QueryPipeline.cpp index df10fae9a38..13e91ac718d 100644 --- a/dbms/src/Processors/QueryPipeline.cpp +++ b/dbms/src/Processors/QueryPipeline.cpp @@ -496,6 +496,8 @@ void QueryPipeline::unitePipelines( table_locks.insert(table_locks.end(), std::make_move_iterator(pipeline.table_locks.begin()), std::make_move_iterator(pipeline.table_locks.end())); interpreter_context.insert(interpreter_context.end(), pipeline.interpreter_context.begin(), pipeline.interpreter_context.end()); storage_holder.insert(storage_holder.end(), pipeline.storage_holder.begin(), pipeline.storage_holder.end()); + + max_threads = std::max(max_threads, pipeline.max_threads); } if (!extremes.empty()) From 3f35d143b365f1bd2bf68b0c9911c53a73fe62a9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 1 Jan 2020 14:15:29 +0300 Subject: [PATCH 23/85] Update num threads in processors. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 7dd9aec7ee5..172dd1b7d12 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -485,8 +485,8 @@ BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams(QueryPipeli QueryPipeline InterpreterSelectQuery::executeWithProcessors() { QueryPipeline query_pipeline; - query_pipeline.setMaxThreads(context->getSettingsRef().max_threads); executeImpl(query_pipeline, input, query_pipeline); + query_pipeline.setMaxThreads(max_streams); query_pipeline.addInterpreterContext(context); query_pipeline.addStorageHolder(storage); return query_pipeline; From faf118fb15b344826361c8aaa83e75ab7ea5aed0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 1 Jan 2020 14:18:24 +0300 Subject: [PATCH 24/85] Do not create extra thread in PipelineExecutor if num_threads is 1. --- .../Processors/Executors/PipelineExecutor.cpp | 38 +++++++++++-------- 1 file changed, 22 insertions(+), 16 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index fee1e37e885..30cb08ae0ba 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -720,29 +720,35 @@ void PipelineExecutor::executeImpl(size_t num_threads) } } - for (size_t i = 0; i < num_threads; ++i) + if (num_threads > 1) { - threads.emplace_back([this, thread_group, thread_num = i, num_threads] + + for (size_t i = 0; i < num_threads; ++i) { - /// ThreadStatus thread_status; + threads.emplace_back([this, thread_group, thread_num = i, num_threads] + { + /// ThreadStatus thread_status; - setThreadName("QueryPipelineEx"); + setThreadName("QueryPipelineEx"); - if (thread_group) - CurrentThread::attachTo(thread_group); + if (thread_group) + CurrentThread::attachTo(thread_group); - SCOPE_EXIT( - if (thread_group) - CurrentThread::detachQueryIfNotDetached(); - ); + SCOPE_EXIT( + if (thread_group) + CurrentThread::detachQueryIfNotDetached(); + ); - executeSingleThread(thread_num, num_threads); - }); + executeSingleThread(thread_num, num_threads); + }); + } + + for (auto & thread : threads) + if (thread.joinable()) + thread.join(); } - - for (auto & thread : threads) - if (thread.joinable()) - thread.join(); + else + executeSingleThread(0, num_threads); finished_flag = true; } From ab80f70cfffb4fe1aaa8416d3dbba0cf77763bed Mon Sep 17 00:00:00 2001 From: Ramazan Polat Date: Wed, 1 Jan 2020 19:43:17 +0300 Subject: [PATCH 25/85] Much comprehensive documentation Added: - Usage of NULL values in conditionals - Using conditional results directly - Better examples --- .../functions/conditional_functions.md | 139 +++++++++++++++--- 1 file changed, 120 insertions(+), 19 deletions(-) diff --git a/docs/en/query_language/functions/conditional_functions.md b/docs/en/query_language/functions/conditional_functions.md index 074df25303f..a40675dbc81 100644 --- a/docs/en/query_language/functions/conditional_functions.md +++ b/docs/en/query_language/functions/conditional_functions.md @@ -1,19 +1,66 @@ # Conditional functions -## if(cond, then, else), cond ? operator then : else +## `if` function -Returns `then` if `cond != 0`, or `else` if `cond = 0`. -`cond` must be of type `UInt8`, and `then` and `else` must have the lowest common type. +Syntax: `if(cond, then, else)` -`then` and `else` can be `NULL` +Returns `then` if the `cond` is truthy(greater than zero), otherwise returns `else`. + +* `cond` must be of type of `UInt8`, and `then` and `else` must have the lowest common type. + +* `then` and `else` can be `NULL` + +**Example:** + +Take this `LEFT_RIGHT` table: + +```sql +SELECT * +FROM LEFT_RIGHT + +┌─left─┬─right─┐ +│ ᴺᵁᴸᴸ │ 4 │ +│ 1 │ 3 │ +│ 2 │ 2 │ +│ 3 │ 1 │ +│ 4 │ ᴺᵁᴸᴸ │ +└──────┴───────┘ +``` +The following query compares `left` and `right` values: + +```sql +SELECT + left, + right, + if(left < right, 'left is smaller than right', 'right is greater or equal than left') AS is_smaller +FROM LEFT_RIGHT +WHERE isNotNull(left) AND isNotNull(right) + +┌─left─┬─right─┬─is_smaller──────────────────────────┐ +│ 1 │ 3 │ left is smaller than right │ +│ 2 │ 2 │ right is greater or equal than left │ +│ 3 │ 1 │ right is greater or equal than left │ +└──────┴───────┴─────────────────────────────────────┘ +``` +Note: `NULL` values are not used in this example, check *`NULL` values in conditinals"* section below. + +## Ternary operator + +It works same as `if` function. + +Syntax: `cond ? then : else` + +Returns `then` if the `cond` is truthy(greater than zero), otherwise returns `else`. + +* `cond` must be of type of `UInt8`, and `then` and `else` must have the lowest common type. + +* `then` and `else` can be `NULL` ## multiIf Allows you to write the [CASE](../operators.md#operator_case) operator more compactly in the query. -```sql -multiIf(cond_1, then_1, cond_2, then_2...else) -``` +Syntax: `multiIf(cond_1, then_1, cond_2, then_2, ..., else)` **Parameters:** @@ -29,22 +76,76 @@ The function returns one of the values `then_N` or `else`, depending on the cond **Example** -Take the table +Again using `LEFT_RIGHT` table. -```text -┌─x─┬────y─┐ -│ 1 │ ᴺᵁᴸᴸ │ -│ 2 │ 3 │ -└───┴──────┘ +```sql +SELECT + left, + right, + multiIf(left < right, 'left is smaller', left > right, 'left is greater', left = right, 'Both equal', 'Null value') AS result +FROM LEFT_RIGHT + +┌─left─┬─right─┬─result──────────┐ +│ ᴺᵁᴸᴸ │ 4 │ Null value │ +│ 1 │ 3 │ left is smaller │ +│ 2 │ 2 │ Both equal │ +│ 3 │ 1 │ left is greater │ +│ 4 │ ᴺᵁᴸᴸ │ Null value │ +└──────┴───────┴─────────────────┘ +``` +## Using conditional results directly + +Conditionals always result to `0`, `1` or `NULL`. So you can use conditional results directly like this: + +```sql +SELECT left < right AS is_small +FROM LEFT_RIGHT + +┌─is_small─┐ +│ ᴺᵁᴸᴸ │ +│ 1 │ +│ 0 │ +│ 0 │ +│ ᴺᵁᴸᴸ │ +└──────────┘ ``` -Run the query `SELECT multiIf(isNull(y) x, y < 3, y, NULL) FROM t_null`. Result: -```text -┌─multiIf(isNull(y), x, less(y, 3), y, NULL)─┐ -│ 1 │ -│ ᴺᵁᴸᴸ │ -└────────────────────────────────────────────┘ +## `NULL` values in conditinals + +When `NULL` values are involved in conditionals, the result will also be `NULL`. + +```sql +SELECT + NULL < 1, + 2 < NULL, + NULL < NULL, + NULL = NULL + +┌─less(NULL, 1)─┬─less(2, NULL)─┬─less(NULL, NULL)─┬─equals(NULL, NULL)─┐ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +└───────────────┴───────────────┴──────────────────┴────────────────────┘ ``` +So you should construct your queries carefully if the types are `Nullable`. + +The following example demonstrates this by failing to add equals condition to `multiIf`. + +```sql +SELECT + left, + right, + multiIf(left < right, 'left is smaller', left > right, 'right is smaller', 'Both equal') AS faulty_result +FROM LEFT_RIGHT + +┌─left─┬─right─┬─faulty_result────┐ +│ ᴺᵁᴸᴸ │ 4 │ Both equal │ +│ 1 │ 3 │ left is smaller │ +│ 2 │ 2 │ Both equal │ +│ 3 │ 1 │ right is smaller │ +│ 4 │ ᴺᵁᴸᴸ │ Both equal │ +└──────┴───────┴──────────────────┘ +``` + + [Original article](https://clickhouse.yandex/docs/en/query_language/functions/conditional_functions/) From cc67ab11076437ba296cd8f025f6c7753858f7b8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 1 Jan 2020 21:42:15 +0300 Subject: [PATCH 26/85] Enable affinity only for multiple threads. --- .../Processors/Executors/PipelineExecutor.cpp | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 30cb08ae0ba..e47483a20f9 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -501,13 +501,15 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads #if !defined(__APPLE__) && !defined(__FreeBSD__) /// Specify CPU core for thread if can. /// It may reduce the number of context swithches. -/* - cpu_set_t cpu_set; - CPU_ZERO(&cpu_set); - CPU_SET(thread_num, &cpu_set); - if (sched_setaffinity(0, sizeof(cpu_set_t), &cpu_set) == -1) - LOG_TRACE(log, "Cannot set affinity for thread " << num_threads); -*/ + if (num_threads > 1) + { + cpu_set_t cpu_set; + CPU_ZERO(&cpu_set); + CPU_SET(thread_num, &cpu_set); + + if (sched_setaffinity(0, sizeof(cpu_set_t), &cpu_set) == -1) + LOG_TRACE(log, "Cannot set affinity for thread " << num_threads); + } #endif // UInt64 total_time_ns = 0; From 85e1c88d1234a6f85bb5912de2ebb8b4707a1cfd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Jan 2020 09:27:53 +0300 Subject: [PATCH 27/85] Disable affinity. --- dbms/src/Processors/Executors/PipelineExecutor.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index e47483a20f9..bc0de1fb81d 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -501,6 +501,7 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads #if !defined(__APPLE__) && !defined(__FreeBSD__) /// Specify CPU core for thread if can. /// It may reduce the number of context swithches. + /* if (num_threads > 1) { cpu_set_t cpu_set; @@ -510,6 +511,7 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads if (sched_setaffinity(0, sizeof(cpu_set_t), &cpu_set) == -1) LOG_TRACE(log, "Cannot set affinity for thread " << num_threads); } + */ #endif // UInt64 total_time_ns = 0; From 213f84580fc86662cfb7638bf9402e144eb1298b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Jan 2020 10:48:51 +0300 Subject: [PATCH 28/85] Lower number of threads. --- dbms/src/Core/SettingsCollection.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Core/SettingsCollection.cpp b/dbms/src/Core/SettingsCollection.cpp index 60c18a19fa7..aa5fbcd58ef 100644 --- a/dbms/src/Core/SettingsCollection.cpp +++ b/dbms/src/Core/SettingsCollection.cpp @@ -216,6 +216,10 @@ void SettingMaxThreads::setAuto() UInt64 SettingMaxThreads::getAutoValue() const { static auto res = getNumberOfPhysicalCPUCores(); + + if (res > 32) + res /= 2; + return res; } From 93643975970c05a6c2b53cbf9cb7af5bc0cafea3 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 18 Dec 2019 11:56:03 +0800 Subject: [PATCH 29/85] try fix some predicate optimizer bug --- .../Interpreters/DatabaseAndTableWithAlias.h | 2 + dbms/src/Interpreters/ExpressionAnalyzer.cpp | 1 - .../ExtractExpressionInfoVisitor.cpp | 59 ++ .../ExtractExpressionInfoVisitor.h | 39 ++ .../ExtractFunctionDataVisitor.cpp | 16 - .../Interpreters/ExtractFunctionDataVisitor.h | 25 - .../FindIdentifierBestTableVisitor.cpp | 40 -- .../FindIdentifierBestTableVisitor.h | 27 - dbms/src/Interpreters/InDepthNodeVisitor.h | 8 +- .../PredicateExpressionsOptimizer.cpp | 512 +++--------------- .../PredicateExpressionsOptimizer.h | 82 +-- .../Interpreters/PredicateRewriteVisitor.cpp | 141 +++++ .../Interpreters/PredicateRewriteVisitor.h | 41 ++ dbms/src/Interpreters/SyntaxAnalyzer.cpp | 2 +- dbms/src/Storages/StorageView.cpp | 21 +- 15 files changed, 405 insertions(+), 611 deletions(-) create mode 100644 dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp create mode 100644 dbms/src/Interpreters/ExtractExpressionInfoVisitor.h delete mode 100644 dbms/src/Interpreters/ExtractFunctionDataVisitor.cpp delete mode 100644 dbms/src/Interpreters/ExtractFunctionDataVisitor.h delete mode 100644 dbms/src/Interpreters/FindIdentifierBestTableVisitor.cpp delete mode 100644 dbms/src/Interpreters/FindIdentifierBestTableVisitor.h create mode 100644 dbms/src/Interpreters/PredicateRewriteVisitor.cpp create mode 100644 dbms/src/Interpreters/PredicateRewriteVisitor.h diff --git a/dbms/src/Interpreters/DatabaseAndTableWithAlias.h b/dbms/src/Interpreters/DatabaseAndTableWithAlias.h index 3567a351b14..ad1f747b6fd 100644 --- a/dbms/src/Interpreters/DatabaseAndTableWithAlias.h +++ b/dbms/src/Interpreters/DatabaseAndTableWithAlias.h @@ -72,4 +72,6 @@ private: std::vector getDatabaseAndTables(const ASTSelectQuery & select_query, const String & current_database); std::optional getDatabaseAndTable(const ASTSelectQuery & select, size_t table_number); +using TablesWithColumnNames = std::vector; + } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 59dff858cf0..fa199f94efc 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -26,7 +26,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp new file mode 100644 index 00000000000..4bd0446cd97 --- /dev/null +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp @@ -0,0 +1,59 @@ +#include +#include +#include +#include + + +namespace DB +{ + +void ExpressionInfoMatcher::visit(const ASTPtr & ast, Data & data) +{ + if (const auto * function = ast->as()) + visit(*function, ast, data); + else if (const auto * identifier = ast->as()) + visit(*identifier, ast, data); +} + +void ExpressionInfoMatcher::visit(const ASTFunction & ast_function, const ASTPtr &, Data & data) +{ + if (ast_function.name == "arrayJoin") + data.is_array_join = true; + else if (AggregateFunctionFactory::instance().isAggregateFunctionName(ast_function.name)) + data.is_aggregate_function = true; + else + { + const auto & function = FunctionFactory::instance().tryGet(ast_function.name, data.context); + + /// Skip lambda, tuple and other special functions + if (function && function->isStateful()) + data.is_stateful_function = true; + } +} + +void ExpressionInfoMatcher::visit(const ASTIdentifier & identifier, const ASTPtr &, Data & data) +{ + if (!identifier.compound()) + { + for (size_t index = 0; index < data.tables.size(); ++index) + { + const auto & columns = data.tables[index].columns; + + // TODO: make sure no collision ever happens + if (std::find(columns.begin(), columns.end(), identifier.name) != columns.end()) + { + data.unique_reference_tables_pos.emplace(index); + break; + } + } + } + else + { + size_t best_table_pos = 0; + if (IdentifierSemantic::chooseTable(identifier, data.tables, best_table_pos)) + data.unique_reference_tables_pos.emplace(best_table_pos); + } +} + +} + diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h new file mode 100644 index 00000000000..6ef9960b918 --- /dev/null +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class Context; + +struct ExpressionInfoMatcher +{ + struct Data + { + const Context & context; + const std::vector & tables; + + bool is_array_join = false; + bool is_stateful_function = false; + bool is_aggregate_function = false; + std::unordered_set unique_reference_tables_pos; + }; + + static void visit(const ASTPtr & ast, Data & data); + + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } + + static void visit(const ASTFunction & ast_function, const ASTPtr &, Data & data); + + static void visit(const ASTIdentifier & identifier, const ASTPtr &, Data & data); +}; + + +using ExpressionInfoVisitor = ConstInDepthNodeVisitor; + +} diff --git a/dbms/src/Interpreters/ExtractFunctionDataVisitor.cpp b/dbms/src/Interpreters/ExtractFunctionDataVisitor.cpp deleted file mode 100644 index d7a0d9001d5..00000000000 --- a/dbms/src/Interpreters/ExtractFunctionDataVisitor.cpp +++ /dev/null @@ -1,16 +0,0 @@ -#include -#include - - -namespace DB -{ - -void ExtractFunctionData::visit(ASTFunction & function, ASTPtr &) -{ - if (AggregateFunctionFactory::instance().isAggregateFunctionName(function.name)) - aggregate_functions.emplace_back(&function); - else - functions.emplace_back(&function); -} - -} diff --git a/dbms/src/Interpreters/ExtractFunctionDataVisitor.h b/dbms/src/Interpreters/ExtractFunctionDataVisitor.h deleted file mode 100644 index ed3dbb868c4..00000000000 --- a/dbms/src/Interpreters/ExtractFunctionDataVisitor.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - -namespace DB -{ - -struct ExtractFunctionData -{ - using TypeToVisit = ASTFunction; - - std::vector functions; - std::vector aggregate_functions; - - void visit(ASTFunction & identifier, ASTPtr &); -}; - -using ExtractFunctionMatcher = OneTypeMatcher; -using ExtractFunctionVisitor = InDepthNodeVisitor; - -} diff --git a/dbms/src/Interpreters/FindIdentifierBestTableVisitor.cpp b/dbms/src/Interpreters/FindIdentifierBestTableVisitor.cpp deleted file mode 100644 index 56897ec15c7..00000000000 --- a/dbms/src/Interpreters/FindIdentifierBestTableVisitor.cpp +++ /dev/null @@ -1,40 +0,0 @@ -#include -#include - - -namespace DB -{ - -FindIdentifierBestTableData::FindIdentifierBestTableData(const std::vector & tables_) - : tables(tables_) -{ -} - -void FindIdentifierBestTableData::visit(ASTIdentifier & identifier, ASTPtr &) -{ - const DatabaseAndTableWithAlias * best_table = nullptr; - - if (!identifier.compound()) - { - for (const auto & table_names : tables) - { - auto & columns = table_names.columns; - if (std::find(columns.begin(), columns.end(), identifier.name) != columns.end()) - { - // TODO: make sure no collision ever happens - if (!best_table) - best_table = &table_names.table; - } - } - } - else - { - size_t best_table_pos = 0; - if (IdentifierSemantic::chooseTable(identifier, tables, best_table_pos)) - best_table = &tables[best_table_pos].table; - } - - identifier_table.emplace_back(&identifier, best_table); -} - -} diff --git a/dbms/src/Interpreters/FindIdentifierBestTableVisitor.h b/dbms/src/Interpreters/FindIdentifierBestTableVisitor.h deleted file mode 100644 index 498ee60ab0b..00000000000 --- a/dbms/src/Interpreters/FindIdentifierBestTableVisitor.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -struct FindIdentifierBestTableData -{ - using TypeToVisit = ASTIdentifier; - using IdentifierWithTable = std::pair; - - const std::vector & tables; - std::vector identifier_table; - - FindIdentifierBestTableData(const std::vector & tables_); - - void visit(ASTIdentifier & identifier, ASTPtr &); -}; - -using FindIdentifierBestTableMatcher = OneTypeMatcher; -using FindIdentifierBestTableVisitor = InDepthNodeVisitor; - -} diff --git a/dbms/src/Interpreters/InDepthNodeVisitor.h b/dbms/src/Interpreters/InDepthNodeVisitor.h index 18b84b11b24..7bb4f5e4d54 100644 --- a/dbms/src/Interpreters/InDepthNodeVisitor.h +++ b/dbms/src/Interpreters/InDepthNodeVisitor.h @@ -59,7 +59,13 @@ public: using Data = Data_; using TypeToVisit = typename Data::TypeToVisit; - static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return visit_children; } + static bool needChildVisit(const ASTPtr & node, const ASTPtr &) + { + if (node && node->as()) + return visit_children; + + return true; + } static void visit(T & ast, Data & data) { diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index 050ee637d18..a5cc578a98a 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -2,9 +2,9 @@ #include #include -#include -#include #include +#include +#include #include #include #include @@ -13,9 +13,6 @@ #include #include #include -#include -#include -#include #include #include #include @@ -23,9 +20,9 @@ #include #include #include -#include -#include +#include #include +#include #include @@ -38,155 +35,99 @@ namespace ErrorCodes extern const int UNKNOWN_ELEMENT_IN_AST; } -namespace -{ - -constexpr auto and_function_name = "and"; - -String qualifiedName(ASTIdentifier * identifier, const String & prefix) -{ - if (identifier->isShort()) - return prefix + identifier->getAliasOrColumnName(); - return identifier->getAliasOrColumnName(); -} - -} - PredicateExpressionsOptimizer::PredicateExpressionsOptimizer( - ASTSelectQuery * ast_select_, ExtractedSettings && settings_, const Context & context_) - : ast_select(ast_select_), settings(settings_), context(context_) + const Context & context_, const TablesWithColumnNames & tables_with_columns_, const Settings & settings_) + : context(context_), tables_with_columns(tables_with_columns_), settings(settings_) { } -bool PredicateExpressionsOptimizer::optimize() +bool PredicateExpressionsOptimizer::optimize(ASTSelectQuery & select_query) { - if (!settings.enable_optimize_predicate_expression || !ast_select || !ast_select->tables() || ast_select->tables()->children.empty()) + if (!settings.enable_optimize_predicate_expression || !select_query.tables() || select_query.tables()->children.empty()) return false; - if (!ast_select->where() && !ast_select->prewhere()) + if ((!select_query.where() && !select_query.prewhere()) || select_query.array_join_expression_list()) return false; - if (ast_select->array_join_expression_list()) - return false; + const auto & tables_predicates = extractTablesPredicates(select_query.where(), select_query.prewhere()); - SubqueriesProjectionColumns all_subquery_projection_columns = getAllSubqueryProjectionColumns(); + if (!tables_predicates.empty()) + return tryRewritePredicatesToTables(select_query.refTables()->children, tables_predicates); - bool is_rewrite_subqueries = false; - if (!all_subquery_projection_columns.empty()) - { - is_rewrite_subqueries |= optimizeImpl(ast_select->where(), all_subquery_projection_columns, OptimizeKind::PUSH_TO_WHERE); - is_rewrite_subqueries |= optimizeImpl(ast_select->prewhere(), all_subquery_projection_columns, OptimizeKind::PUSH_TO_PREWHERE); - } - - return is_rewrite_subqueries; + return false; } -bool PredicateExpressionsOptimizer::optimizeImpl( - const ASTPtr & outer_expression, const SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind expression_kind) +static ASTs splitConjunctionPredicate(const std::initializer_list & predicates) { - /// split predicate with `and` - std::vector outer_predicate_expressions = splitConjunctionPredicate(outer_expression); + std::vector res; - std::vector table_expressions = getTableExpressions(*ast_select); - std::vector tables_with_columns = getDatabaseAndTablesWithColumnNames(table_expressions, context); - - bool is_rewrite_subquery = false; - for (auto & outer_predicate : outer_predicate_expressions) + auto remove_expression_at_index = [&res] (const size_t index) { - if (isArrayJoinFunction(outer_predicate)) + if (index < res.size() - 1) + std::swap(res[index], res.back()); + res.pop_back(); + }; + + for (const auto & predicate : predicates) + { + if (!predicate) continue; - auto outer_predicate_dependencies = getDependenciesAndQualifiers(outer_predicate, tables_with_columns); + res.emplace_back(predicate); - /// TODO: remove origin expression - for (const auto & [subquery, projection_columns] : subqueries_projection_columns) + for (size_t idx = 0; idx < res.size();) { - OptimizeKind optimize_kind = OptimizeKind::NONE; - if (allowPushDown(subquery, outer_predicate, projection_columns, outer_predicate_dependencies, optimize_kind)) + const auto & expression = res.at(idx); + + if (const auto * function = expression->as(); function && function->name == "and") { - if (optimize_kind == OptimizeKind::NONE) - optimize_kind = expression_kind; + for (auto & child : function->arguments->children) + res.emplace_back(child); - ASTPtr inner_predicate = outer_predicate->clone(); - cleanExpressionAlias(inner_predicate); /// clears the alias name contained in the outer predicate + remove_expression_at_index(idx); + continue; + } + ++idx; + } + } - std::vector inner_predicate_dependencies = - getDependenciesAndQualifiers(inner_predicate, tables_with_columns); + return res; +} - setNewAliasesForInnerPredicate(projection_columns, inner_predicate_dependencies); +std::vector PredicateExpressionsOptimizer::extractTablesPredicates(const ASTPtr & where, const ASTPtr & prewhere) +{ + std::vector tables_predicates(tables_with_columns.size()); - switch (optimize_kind) - { - case OptimizeKind::NONE: continue; - case OptimizeKind::PUSH_TO_WHERE: - is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::WHERE); - continue; - case OptimizeKind::PUSH_TO_HAVING: - is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::HAVING); - continue; - case OptimizeKind::PUSH_TO_PREWHERE: - is_rewrite_subquery |= optimizeExpression(inner_predicate, subquery, ASTSelectQuery::Expression::PREWHERE); - continue; - } + for (const auto & predicate_expression : splitConjunctionPredicate({where, prewhere})) + { + ExpressionInfoVisitor::Data expression_info{.context = context, .tables = tables_with_columns}; + ExpressionInfoVisitor(expression_info).visit(predicate_expression); + + if (expression_info.is_stateful_function) + return {}; /// give up the optimization when the predicate contains stateful function + + if (!expression_info.is_array_join) + { + if (expression_info.unique_reference_tables_pos.size() == 1) + tables_predicates[*expression_info.unique_reference_tables_pos.begin()].emplace_back(predicate_expression); + else if (expression_info.unique_reference_tables_pos.size() == 0) + { + for (size_t index = 0; index < tables_predicates.size(); ++index) + tables_predicates[index].emplace_back(predicate_expression); } } } - return is_rewrite_subquery; + + return tables_predicates; /// everything is OK, it can be optimized } -bool PredicateExpressionsOptimizer::allowPushDown( - const ASTSelectQuery * subquery, - const ASTPtr &, - const std::vector & projection_columns, - const std::vector & dependencies, - OptimizeKind & optimize_kind) +bool PredicateExpressionsOptimizer::tryRewritePredicatesToTables(ASTs & tables_element, const std::vector & tables_predicates) { - if (!subquery - || (!settings.enable_optimize_predicate_expression_to_final_subquery && subquery->final()) - || subquery->limitBy() || subquery->limitLength() - || subquery->with() || subquery->withFill()) - return false; - else + bool is_rewrite_tables = false; + + for (size_t index = tables_element.size(); index > 0; --index) { - ASTPtr expr_list = ast_select->select(); - ExtractFunctionVisitor::Data extract_data; - ExtractFunctionVisitor(extract_data).visit(expr_list); - - for (const auto & subquery_function : extract_data.functions) - { - const auto & function = FunctionFactory::instance().tryGet(subquery_function->name, context); - - /// Skip lambda, tuple and other special functions - if (function && function->isStateful()) - return false; - } - } - - const auto * ast_join = ast_select->join(); - const ASTTableExpression * left_table_expr = nullptr; - const ASTTableExpression * right_table_expr = nullptr; - const ASTSelectQuery * left_subquery = nullptr; - const ASTSelectQuery * right_subquery = nullptr; - - if (ast_join) - { - left_table_expr = ast_select - ->tables()->as() - ->children[0]->as() - ->table_expression->as(); - right_table_expr = ast_select - ->tables()->as() - ->children[1]->as() - ->table_expression->as(); - - if (left_table_expr && left_table_expr->subquery) - left_subquery = left_table_expr->subquery - ->children[0]->as() - ->list_of_selects->children[0]->as(); - if (right_table_expr && right_table_expr->subquery) - right_subquery = right_table_expr->subquery - ->children[0]->as() - ->list_of_selects->children[0]->as(); + size_t table_pos = index - 1; /// NOTE: the syntactic way of pushdown has limitations and should be partially disabled in case of JOINs. /// Let's take a look at the query: @@ -201,326 +142,37 @@ bool PredicateExpressionsOptimizer::allowPushDown( /// It happens because the not-matching columns are replaced with a global default values on JOIN. /// Same is true for RIGHT JOIN and FULL JOIN. - /// Check right side for LEFT'o'FULL JOIN - if (isLeftOrFull(ast_join->table_join->as()->kind) && right_subquery == subquery) - return false; - - /// Check left side for RIGHT'o'FULL JOIN - if (isRightOrFull(ast_join->table_join->as()->kind) && left_subquery == subquery) - return false; - } - - return checkDependencies(projection_columns, dependencies, optimize_kind); -} - -bool PredicateExpressionsOptimizer::checkDependencies( - const std::vector & projection_columns, - const std::vector & dependencies, - OptimizeKind & optimize_kind) -{ - for (const auto & [identifier, prefix] : dependencies) - { - bool is_found = false; - String qualified_name = qualifiedName(identifier, prefix); - - for (const auto & [ast, alias] : projection_columns) + if (const auto & table_element = tables_element[table_pos]->as()) { - if (alias == qualified_name) - { - is_found = true; - ASTPtr projection_column = ast; - ExtractFunctionVisitor::Data extract_data; - ExtractFunctionVisitor(extract_data).visit(projection_column); + if (table_element->table_join && isLeft(table_element->table_join->as()->kind)) + continue; /// Skip right table optimization - if (!extract_data.aggregate_functions.empty()) - optimize_kind = OptimizeKind::PUSH_TO_HAVING; - } - } + if (table_element->table_join && isFull(table_element->table_join->as()->kind)) + break; /// Skip left and right table optimization - if (!is_found) - return false; - } + is_rewrite_tables |= tryRewritePredicatesToTable(tables_element[table_pos], tables_predicates[table_pos], + tables_with_columns[table_pos].columns); - return true; -} - -std::vector PredicateExpressionsOptimizer::splitConjunctionPredicate(const ASTPtr & predicate_expression) -{ - std::vector predicate_expressions; - - if (predicate_expression) - { - predicate_expressions.emplace_back(predicate_expression); - - auto remove_expression_at_index = [&predicate_expressions] (const size_t index) - { - if (index < predicate_expressions.size() - 1) - std::swap(predicate_expressions[index], predicate_expressions.back()); - predicate_expressions.pop_back(); - }; - - for (size_t idx = 0; idx < predicate_expressions.size();) - { - const auto expression = predicate_expressions.at(idx); - - if (const auto * function = expression->as()) - { - if (function->name == and_function_name) - { - for (auto & child : function->arguments->children) - predicate_expressions.emplace_back(child); - - remove_expression_at_index(idx); - continue; - } - } - ++idx; + if (table_element->table_join && isRight(table_element->table_join->as()->kind)) + break; /// Skip left table optimization } } - return predicate_expressions; + + return is_rewrite_tables; } -std::vector -PredicateExpressionsOptimizer::getDependenciesAndQualifiers(ASTPtr & expression, std::vector & tables) +bool PredicateExpressionsOptimizer::tryRewritePredicatesToTable(ASTPtr & table_element, const ASTs & table_predicates, const Names & table_column) const { - FindIdentifierBestTableVisitor::Data find_data(tables); - FindIdentifierBestTableVisitor(find_data).visit(expression); - - std::vector dependencies; - - for (const auto & [identifier, table] : find_data.identifier_table) + if (!table_predicates.empty()) { - String table_alias; - if (table) - table_alias = table->getQualifiedNamePrefix(); + PredicateRewriteVisitor::Data data( + context, table_predicates, table_column, settings.enable_optimize_predicate_expression_to_final_subquery); - dependencies.emplace_back(identifier, table_alias); + PredicateRewriteVisitor(data).visit(table_element); + return data.isRewrite(); } - return dependencies; -} - -void PredicateExpressionsOptimizer::setNewAliasesForInnerPredicate( - const std::vector & projection_columns, - const std::vector & dependencies) -{ - for (auto & [identifier, prefix] : dependencies) - { - String qualified_name = qualifiedName(identifier, prefix); - - for (auto & [ast, alias] : projection_columns) - { - if (alias == qualified_name) - { - String name; - if (auto * id = ast->as()) - { - name = id->tryGetAlias(); - if (name.empty()) - name = id->shortName(); - } - else - { - if (ast->tryGetAlias().empty()) - ast->setAlias(ast->getColumnName()); - name = ast->getAliasOrColumnName(); - } - - identifier->setShortName(name); - } - } - } -} - -bool PredicateExpressionsOptimizer::isArrayJoinFunction(const ASTPtr & node) -{ - if (const auto * function = node->as()) - { - if (function->name == "arrayJoin") - return true; - } - - for (auto & child : node->children) - if (isArrayJoinFunction(child)) - return true; - return false; } -bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expression, ASTSelectQuery * subquery, - ASTSelectQuery::Expression expr) -{ - ASTPtr subquery_expression = subquery->getExpression(expr, false); - subquery_expression = subquery_expression ? makeASTFunction(and_function_name, outer_expression, subquery_expression) : outer_expression; - - subquery->setExpression(expr, std::move(subquery_expression)); - return true; -} - -PredicateExpressionsOptimizer::SubqueriesProjectionColumns PredicateExpressionsOptimizer::getAllSubqueryProjectionColumns() -{ - SubqueriesProjectionColumns projection_columns; - - for (const auto & table_expression : getTableExpressions(*ast_select)) - if (table_expression->subquery) - getSubqueryProjectionColumns(table_expression->subquery, projection_columns); - - return projection_columns; -} - -void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(const ASTPtr & subquery, SubqueriesProjectionColumns & projection_columns) -{ - String qualified_name_prefix = subquery->tryGetAlias(); - if (!qualified_name_prefix.empty()) - qualified_name_prefix += '.'; - - const ASTPtr & subselect = subquery->children[0]; - - ASTs select_with_union_projections; - const auto * select_with_union_query = subselect->as(); - - for (auto & select : select_with_union_query->list_of_selects->children) - { - std::vector subquery_projections; - auto select_projection_columns = getSelectQueryProjectionColumns(select); - - if (!select_projection_columns.empty()) - { - if (select_with_union_projections.empty()) - select_with_union_projections = select_projection_columns; - - for (size_t i = 0; i < select_projection_columns.size(); i++) - subquery_projections.emplace_back(std::pair(select_projection_columns[i], - qualified_name_prefix + select_with_union_projections[i]->getAliasOrColumnName())); - - projection_columns.insert(std::pair(select->as(), subquery_projections)); - } - } -} - -ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast) -{ - ASTs projection_columns; - auto * select_query = ast->as(); - - /// first should normalize query tree. - std::unordered_map aliases; - std::vector tables = getDatabaseAndTables(*select_query, context.getCurrentDatabase()); - - /// TODO: get tables from evaluateAsterisk instead of tablesOnly() to extract asterisks in general way - std::vector tables_with_columns = TranslateQualifiedNamesVisitor::Data::tablesOnly(tables); - TranslateQualifiedNamesVisitor::Data qn_visitor_data({}, std::move(tables_with_columns), false); - TranslateQualifiedNamesVisitor(qn_visitor_data).visit(ast); - - QueryAliasesVisitor::Data query_aliases_data{aliases}; - QueryAliasesVisitor(query_aliases_data).visit(ast); - - MarkTableIdentifiersVisitor::Data mark_tables_data{aliases}; - MarkTableIdentifiersVisitor(mark_tables_data).visit(ast); - - QueryNormalizer::Data normalizer_data(aliases, settings); - QueryNormalizer(normalizer_data).visit(ast); - - for (const auto & projection_column : select_query->select()->children) - { - if (projection_column->as() || projection_column->as() || projection_column->as()) - { - ASTs evaluated_columns = evaluateAsterisk(select_query, projection_column); - - for (const auto & column : evaluated_columns) - projection_columns.emplace_back(column); - - continue; - } - - projection_columns.emplace_back(projection_column); - } - return projection_columns; -} - -ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk) -{ - /// SELECT *, SELECT dummy, SELECT 1 AS id - if (!select_query->tables() || select_query->tables()->children.empty()) - return {}; - - std::vector tables_expression = getTableExpressions(*select_query); - - if (const auto * qualified_asterisk = asterisk->as()) - { - if (qualified_asterisk->children.size() != 1) - throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR); - - DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]); - - for (auto it = tables_expression.begin(); it != tables_expression.end();) - { - const ASTTableExpression * table_expression = *it; - DatabaseAndTableWithAlias database_and_table_with_alias(*table_expression, context.getCurrentDatabase()); - - if (ident_db_and_name.satisfies(database_and_table_with_alias, true)) - ++it; - else - it = tables_expression.erase(it); /// It's not a required table - } - } - - ASTs projection_columns; - for (auto & table_expression : tables_expression) - { - if (table_expression->subquery) - { - const auto * subquery = table_expression->subquery->as(); - const auto * select_with_union_query = subquery->children[0]->as(); - const auto subquery_projections = getSelectQueryProjectionColumns(select_with_union_query->list_of_selects->children[0]); - projection_columns.insert(projection_columns.end(), subquery_projections.begin(), subquery_projections.end()); - } - else - { - StoragePtr storage; - - if (table_expression->table_function) - { - auto query_context = const_cast(&context.getQueryContext()); - storage = query_context->executeTableFunction(table_expression->table_function); - } - else if (table_expression->database_and_table_name) - { - const auto * database_and_table_ast = table_expression->database_and_table_name->as(); - DatabaseAndTableWithAlias database_and_table_name(*database_and_table_ast); - storage = context.getTable(database_and_table_name.database, database_and_table_name.table); - } - else - throw Exception("Logical error: unexpected table expression", ErrorCodes::LOGICAL_ERROR); - - const auto block = storage->getSampleBlock(); - if (const auto * asterisk_pattern = asterisk->as()) - { - for (size_t idx = 0; idx < block.columns(); ++idx) - { - auto & col = block.getByPosition(idx); - if (asterisk_pattern->isColumnMatching(col.name)) - projection_columns.emplace_back(std::make_shared(col.name)); - } - } - else - { - for (size_t idx = 0; idx < block.columns(); ++idx) - projection_columns.emplace_back(std::make_shared(block.getByPosition(idx).name)); - } - } - } - return projection_columns; -} - -void PredicateExpressionsOptimizer::cleanExpressionAlias(ASTPtr & expression) -{ - const auto my_alias = expression->tryGetAlias(); - if (!my_alias.empty()) - expression->setAlias(""); - - for (auto & child : expression->children) - cleanExpressionAlias(child); -} - } diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h index ca2c8b8766d..19010c3a328 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h @@ -1,15 +1,13 @@ #pragma once -#include "DatabaseAndTableWithAlias.h" #include -#include +#include namespace DB { -class ASTIdentifier; -class ASTSubquery; class Context; +class Settings; /** This class provides functions for Push-Down predicate expressions * @@ -24,87 +22,35 @@ class Context; */ class PredicateExpressionsOptimizer { - using ProjectionWithAlias = std::pair; - using SubqueriesProjectionColumns = std::map>; - using IdentifierWithQualifier = std::pair; +public: + PredicateExpressionsOptimizer(const Context & context_, const TablesWithColumnNames & tables_with_columns_, const Settings & settings_); + bool optimize(ASTSelectQuery & select_query); + +private: /// Extracts settings, mostly to show which are used and which are not. struct ExtractedSettings { - /// QueryNormalizer settings - const UInt64 max_ast_depth; - const UInt64 max_expanded_ast_elements; - const String count_distinct_implementation; - - /// for PredicateExpressionsOptimizer const bool enable_optimize_predicate_expression; const bool enable_optimize_predicate_expression_to_final_subquery; - const bool join_use_nulls; template ExtractedSettings(const T & settings_) - : max_ast_depth(settings_.max_ast_depth), - max_expanded_ast_elements(settings_.max_expanded_ast_elements), - count_distinct_implementation(settings_.count_distinct_implementation), - enable_optimize_predicate_expression(settings_.enable_optimize_predicate_expression), - enable_optimize_predicate_expression_to_final_subquery(settings_.enable_optimize_predicate_expression_to_final_subquery), - join_use_nulls(settings_.join_use_nulls) + : enable_optimize_predicate_expression(settings_.enable_optimize_predicate_expression), + enable_optimize_predicate_expression_to_final_subquery(settings_.enable_optimize_predicate_expression_to_final_subquery) {} }; -public: - PredicateExpressionsOptimizer(ASTSelectQuery * ast_select_, ExtractedSettings && settings_, const Context & context_); - - bool optimize(); - -private: - ASTSelectQuery * ast_select; - const ExtractedSettings settings; const Context & context; + const std::vector & tables_with_columns; - enum OptimizeKind - { - NONE, - PUSH_TO_PREWHERE, - PUSH_TO_WHERE, - PUSH_TO_HAVING, - }; + const ExtractedSettings settings; - bool isArrayJoinFunction(const ASTPtr & node); + std::vector extractTablesPredicates(const ASTPtr & where, const ASTPtr & prewhere); - std::vector splitConjunctionPredicate(const ASTPtr & predicate_expression); + bool tryRewritePredicatesToTables(ASTs & tables_element, const std::vector & tables_predicates); - std::vector getDependenciesAndQualifiers(ASTPtr & expression, - std::vector & tables_with_aliases); - - bool optimizeExpression(const ASTPtr & outer_expression, ASTSelectQuery * subquery, ASTSelectQuery::Expression expr); - - bool optimizeImpl(const ASTPtr & outer_expression, const SubqueriesProjectionColumns & subqueries_projection_columns, OptimizeKind optimize_kind); - - bool allowPushDown( - const ASTSelectQuery * subquery, - const ASTPtr & outer_predicate, - const std::vector & subquery_projection_columns, - const std::vector & outer_predicate_dependencies, - OptimizeKind & optimize_kind); - - bool checkDependencies( - const std::vector & projection_columns, - const std::vector & dependencies, - OptimizeKind & optimize_kind); - - void setNewAliasesForInnerPredicate(const std::vector & projection_columns, - const std::vector & inner_predicate_dependencies); - - SubqueriesProjectionColumns getAllSubqueryProjectionColumns(); - - void getSubqueryProjectionColumns(const ASTPtr & subquery, SubqueriesProjectionColumns & all_subquery_projection_columns); - - ASTs getSelectQueryProjectionColumns(ASTPtr & ast); - - ASTs evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk); - - void cleanExpressionAlias(ASTPtr & expression); + bool tryRewritePredicatesToTable(ASTPtr & table_element, const ASTs & table_predicates, const Names & table_column) const; }; } diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp new file mode 100644 index 00000000000..c968f65b008 --- /dev/null +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp @@ -0,0 +1,141 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +PredicateRewriteVisitorData::PredicateRewriteVisitorData( + const Context & context_, const ASTs & predicates_, const Names & colunm_names_, bool optimize_final_) + : context(context_), predicates(predicates_), column_names(colunm_names_), optimize_final(optimize_final_) +{ +} + +void PredicateRewriteVisitorData::visit(ASTSelectWithUnionQuery & union_select_query, ASTPtr &) +{ + auto & internal_select_list = union_select_query.list_of_selects->children; + + if (internal_select_list.size() > 0) + visitFirstInternalSelect(*internal_select_list[0]->as(), internal_select_list[0]); + + for (size_t index = 1; index < internal_select_list.size(); ++index) + visitOtherInternalSelect(*internal_select_list[index]->as(), internal_select_list[index]); +} + +void PredicateRewriteVisitorData::visitFirstInternalSelect(ASTSelectQuery & select_query, ASTPtr &) +{ + is_rewrite |= rewriteSubquery(select_query, column_names, column_names); +} + +void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & select_query, ASTPtr &) +{ + /// For non first select, its alias has no more significance, so we can set a temporary alias for them + ASTPtr temp_internal_select = select_query.clone(); + ASTSelectQuery * temp_select_query = temp_internal_select->as(); + + size_t alias_index = 0; + for (const auto ref_select : temp_select_query->refSelect()->children) + { + if (!ref_select->as() && !ref_select->as() && !ref_select->as()) + { + if (const auto & alias = ref_select->tryGetAlias(); alias.empty()) + ref_select->setAlias("--predicate_optimizer_" + toString(alias_index++)); + } + } + + const Names & internal_columns = InterpreterSelectQuery( + temp_internal_select, context, SelectQueryOptions().analyze()).getSampleBlock().getNames(); + + if ((is_rewrite |= rewriteSubquery(*temp_select_query, column_names, internal_columns))) + { + select_query.setExpression(ASTSelectQuery::Expression::SELECT, std::move(temp_select_query->refSelect())); + + if (temp_select_query->where()) + select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(temp_select_query->refWhere())); + + if (temp_select_query->having()) + select_query.setExpression(ASTSelectQuery::Expression::HAVING, std::move(temp_select_query->refHaving())); + } +} + +static void cleanAliasAndCollectIdentifiers(ASTPtr & predicate, std::vector & identifiers) +{ + for (auto & children : predicate->children) + cleanAliasAndCollectIdentifiers(children, identifiers); + + if (const auto alias = predicate->tryGetAlias(); !alias.empty()) + predicate->setAlias(""); + + if (ASTIdentifier * identifier = predicate->as()) + identifiers.emplace_back(identifier); +} + +bool PredicateRewriteVisitorData::allowPushDown(const ASTSelectQuery &subquery, NameSet & aggregate_column) +{ + if ((!optimize_final && subquery.final()) + || subquery.limitBy() || subquery.limitLength() + || subquery.with() || subquery.withFill()) + return false; + + for (const auto & select_expression : subquery.select()->children) + { + ExpressionInfoVisitor::Data expression_info{.context = context, .tables = {}}; + ExpressionInfoVisitor(expression_info).visit(select_expression); + + if (expression_info.is_stateful_function) + return false; + else if (expression_info.is_aggregate_function) + aggregate_column.emplace(select_expression->getAliasOrColumnName()); + } + + return true; +} + +bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, const Names & outer_columns, const Names & inner_columns) +{ + NameSet aggregate_columns; + + if (!allowPushDown(subquery, aggregate_columns)) + return false; + + for (const auto & predicate : predicates) + { + std::vector identifiers; + ASTPtr optimize_predicate = predicate->clone(); + cleanAliasAndCollectIdentifiers(optimize_predicate, identifiers); + + ASTSelectQuery::Expression rewrite_to = ASTSelectQuery::Expression::WHERE; + + for (size_t index = 0; index < identifiers.size(); ++index) + { + const auto & column_name = IdentifierSemantic::getColumnName(*identifiers[index]); + + const auto & iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); + + if (iterator == outer_columns.end()) + throw Exception("", ErrorCodes::LOGICAL_ERROR); + + if (aggregate_columns.count(*column_name)) + rewrite_to = ASTSelectQuery::Expression::HAVING; + + identifiers[index]->setShortName(inner_columns[iterator - outer_columns.begin()]); + } + + ASTPtr optimize_expression = subquery.getExpression(rewrite_to, false); + subquery.setExpression(rewrite_to, + optimize_expression ? makeASTFunction("and", optimize_predicate, optimize_expression) : optimize_predicate); + } + return true; +} + +} diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.h b/dbms/src/Interpreters/PredicateRewriteVisitor.h new file mode 100644 index 00000000000..ea7656256fe --- /dev/null +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.h @@ -0,0 +1,41 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class PredicateRewriteVisitorData +{ +public: + using TypeToVisit = ASTSelectWithUnionQuery; + + bool isRewrite() const { return is_rewrite; } + + void visit(ASTSelectWithUnionQuery & union_select_query, ASTPtr &); + + PredicateRewriteVisitorData(const Context & context_, const ASTs & predicates_, const Names & colunm_names_, bool optimize_final_); + +private: + const Context & context; + const ASTs & predicates; + const Names & column_names; + + bool optimize_final; + bool is_rewrite = false; + + void visitFirstInternalSelect(ASTSelectQuery & select_query, ASTPtr &); + + void visitOtherInternalSelect(ASTSelectQuery & select_query, ASTPtr &); + + bool allowPushDown(const ASTSelectQuery & subquery, NameSet & aggregate_column); + + bool rewriteSubquery(ASTSelectQuery & subquery, const Names & outer_columns, const Names & inner_columns); +}; + +using PredicateRewriteMatcher = OneTypeMatcher; +using PredicateRewriteVisitor = InDepthNodeVisitor; +} diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 85135c71c6f..678f42adf88 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -936,7 +936,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( getArrayJoinedColumns(query, result, select_query, result.source_columns, source_columns_set); /// Push the predicate expression down to the subqueries. - result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize(); + result.rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); setJoinStrictness(*select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys, result.analyzed_join->table_join); diff --git a/dbms/src/Storages/StorageView.cpp b/dbms/src/Storages/StorageView.cpp index 824856dfc4e..5c8543bbb33 100644 --- a/dbms/src/Storages/StorageView.cpp +++ b/dbms/src/Storages/StorageView.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -23,6 +24,7 @@ namespace ErrorCodes { extern const int INCORRECT_QUERY; extern const int LOGICAL_ERROR; + extern const int ALIAS_REQUIRED; } @@ -62,8 +64,23 @@ BlockInputStreams StorageView::read( replaceTableNameWithSubquery(new_outer_select, new_inner_query); - if (PredicateExpressionsOptimizer(new_outer_select, context.getSettings(), context).optimize()) - current_inner_query = new_inner_query; + /// TODO: remove getTableExpressions and getTablesWithColumns + { + const auto & table_expressions = getTableExpressions(*new_outer_select); + const auto & tables_with_columns = getDatabaseAndTablesWithColumnNames(table_expressions, context); + + auto & settings = context.getSettingsRef(); + if (settings.joined_subquery_requires_alias && tables_with_columns.size() > 1) + { + for (auto & pr : tables_with_columns) + if (pr.table.table.empty() && pr.table.alias.empty()) + throw Exception("Not unique subquery in FROM requires an alias (or joined_subquery_requires_alias=0 to disable restriction).", + ErrorCodes::ALIAS_REQUIRED); + } + + if (PredicateExpressionsOptimizer(context, tables_with_columns, context.getSettings()).optimize(*new_outer_select)) + current_inner_query = new_inner_query; + } } QueryPipeline pipeline; From d43eae2db8fa91491c35bb4006f79cc17a5e2c6e Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 2 Jan 2020 12:24:55 -0500 Subject: [PATCH 30/85] First changes. --- .../src/Storages/LiveView/StorageLiveView.cpp | 114 +++++++++++------- dbms/src/Storages/LiveView/StorageLiveView.h | 7 ++ 2 files changed, 76 insertions(+), 45 deletions(-) diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index db410eeb5e4..84ebb497243 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -95,6 +95,64 @@ static void extractDependentTable(ASTPtr & query, String & select_database_name, DB::ErrorCodes::LOGICAL_ERROR); } +BlocksPtrs StorageLiveView::collectMergeableBlocks(const Context & context) +{ + BlocksPtrs new_mergeable_blocks = std::make_shared>(); + BlocksPtr base_mergeable_blocks = std::make_shared(); + + InterpreterSelectQuery interpreter(mergeable_query, context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); + + auto view_mergeable_stream = std::make_shared(interpreter.execute().in); + + while (Block this_block = view_mergeable_stream->read()) + base_mergeable_blocks->push_back(this_block); + + new_mergeable_blocks->push_back(base_mergeable_blocks); + + mergeable_blocks = new_mergeable_blocks; +} + +BlockInputStreams blocksToInputStreams(BlocksPtrs blocks) +{ + BlockInputStreams streams; + + for (auto & blocks_ : *blocks) + { + if (blocks_->empty()) + continue; + + auto sample_block = blocks_->front().cloneEmpty(); + + BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); + + streams.push_back(std::move(stream)); + } + return streams; +} + +/// Complete query using input streams from mergeable blocks +BlockInputStreamPtr StorageLiveView::completeQuery(BlockInputStreams from) +{ + auto block_context = std::make_unique(global_context); + block_context->makeQueryContext(); + + auto blocks_storage = StorageBlocks::createStorage(database_name, table_name, parent_storage->getColumns(), + std::move(from), QueryProcessingStage::WithMergeableState); + + block_context->addExternalTable(table_name + "_blocks", blocks_storage); + + InterpreterSelectQuery select(inner_blocks_query->clone(), *block_context, StoragePtr(), SelectQueryOptions(QueryProcessingStage::Complete)); + BlockInputStreamPtr data = std::make_shared(select.execute().in); + + /// Squashing is needed here because the view query can generate a lot of blocks + /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY + /// and two-level aggregation is triggered). + data = std::make_shared( + data, global_context.getSettingsRef().min_insert_block_size_rows, + global_context.getSettingsRef().min_insert_block_size_bytes); + + return data; +} void StorageLiveView::writeIntoLiveView( StorageLiveView & live_view, @@ -132,41 +190,25 @@ void StorageLiveView::writeIntoLiveView( mergeable_blocks = live_view.getMergeableBlocks(); if (!mergeable_blocks || mergeable_blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh) { - mergeable_blocks = std::make_shared>(); - BlocksPtr base_mergeable_blocks = std::make_shared(); - InterpreterSelectQuery interpreter(mergeable_query, context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); - auto view_mergeable_stream = std::make_shared( - interpreter.execute().in); - while (Block this_block = view_mergeable_stream->read()) - base_mergeable_blocks->push_back(this_block); - mergeable_blocks->push_back(base_mergeable_blocks); - live_view.setMergeableBlocks(mergeable_blocks); - - /// Create from streams - for (auto & blocks_ : *mergeable_blocks) - { - if (blocks_->empty()) - continue; - auto sample_block = blocks_->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); - from.push_back(std::move(stream)); - } - + mergeable_blocks = live_view.collectMergeableBlocks(context); + from = live_view.blocksToInputStreams(mergeable_blocks); is_block_processed = true; } } - auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); - if (!is_block_processed) { BlockInputStreams streams = {std::make_shared(block)}; + auto blocks_storage = StorageBlocks::createStorage(live_view.database_name, live_view.table_name, - parent_storage->getColumns(), std::move(streams), QueryProcessingStage::FetchColumns); + live_view.getParentStorage()->getColumns(), std::move(streams), QueryProcessingStage::FetchColumns); + InterpreterSelectQuery select_block(mergeable_query, context, blocks_storage, QueryProcessingStage::WithMergeableState); + auto data_mergeable_stream = std::make_shared( select_block.execute().in); + while (Block this_block = data_mergeable_stream->read()) new_mergeable_blocks->push_back(this_block); @@ -178,31 +220,11 @@ void StorageLiveView::writeIntoLiveView( mergeable_blocks = live_view.getMergeableBlocks(); mergeable_blocks->push_back(new_mergeable_blocks); - - /// Create from streams - for (auto & blocks_ : *mergeable_blocks) - { - if (blocks_->empty()) - continue; - auto sample_block = blocks_->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); - from.push_back(std::move(stream)); - } + from = live_view.blocksToInputStreams(mergeable_blocks); } } - auto blocks_storage = StorageBlocks::createStorage(live_view.database_name, live_view.table_name, parent_storage->getColumns(), std::move(from), QueryProcessingStage::WithMergeableState); - block_context->addExternalTable(live_view.table_name + "_blocks", blocks_storage); - - InterpreterSelectQuery select(live_view.getInnerBlocksQuery(), *block_context, StoragePtr(), SelectQueryOptions(QueryProcessingStage::Complete)); - BlockInputStreamPtr data = std::make_shared(select.execute().in); - - /// Squashing is needed here because the view query can generate a lot of blocks - /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY - /// and two-level aggregation is triggered). - data = std::make_shared( - data, context.getGlobalContext().getSettingsRef().min_insert_block_size_rows, context.getGlobalContext().getSettingsRef().min_insert_block_size_bytes); - + BlockInputStreamPtr data = completeQuery(std::move(from)); copyData(*data, *output); } @@ -247,6 +269,8 @@ StorageLiveView::StorageLiveView( DatabaseAndTableName(select_database_name, select_table_name), DatabaseAndTableName(database_name, table_name)); + parent_storage = local_context.getTable(select_database_name, select_table_name); + is_temporary = query.temporary; temporary_live_view_timeout = local_context.getSettingsRef().temporary_live_view_timeout.totalSeconds(); diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index a5b0f15e879..46e5008ece3 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -45,6 +45,7 @@ public: String getDatabaseName() const override { return database_name; } String getSelectDatabaseName() const { return select_database_name; } String getSelectTableName() const { return select_table_name; } + StoragePtr getParentStorage() const { return parent_storage; } NameAndTypePair getColumn(const String & column_name) const override; bool hasColumn(const String & column_name) const override; @@ -139,6 +140,8 @@ public: std::shared_ptr getBlocksPtr() { return blocks_ptr; } BlocksPtrs getMergeableBlocks() { return mergeable_blocks; } + /// collect and set mergeable blocks. Must be called holding mutex + BlocksPtrs collectMergeableBlocks(const Context & context); void setMergeableBlocks(BlocksPtrs blocks) { mergeable_blocks = blocks; } std::shared_ptr getActivePtr() { return active_ptr; } @@ -147,6 +150,9 @@ public: Block getHeader() const; + /// convert blocks to input streams + static BlockInputStreams blocksToInputStreams(BlocksPtrs blocks); + static void writeIntoLiveView( StorageLiveView & live_view, const Block & block, @@ -162,6 +168,7 @@ private: ASTPtr inner_blocks_query; /// query over the mergeable blocks to produce final result Context & global_context; std::unique_ptr live_view_context; + StoragePtr parent_storage; bool is_temporary = false; /// Mutex to protect access to sample block From bb0454bb2e2b7c431b593370b06a7ec753d8648a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Jan 2020 21:11:43 +0300 Subject: [PATCH 31/85] Revert prev commit. --- dbms/src/Core/SettingsCollection.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/dbms/src/Core/SettingsCollection.cpp b/dbms/src/Core/SettingsCollection.cpp index aa5fbcd58ef..60c18a19fa7 100644 --- a/dbms/src/Core/SettingsCollection.cpp +++ b/dbms/src/Core/SettingsCollection.cpp @@ -216,10 +216,6 @@ void SettingMaxThreads::setAuto() UInt64 SettingMaxThreads::getAutoValue() const { static auto res = getNumberOfPhysicalCPUCores(); - - if (res > 32) - res /= 2; - return res; } From 2c4bf0581fa62ff63a552a4b20a7de0ce5dafac4 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 2 Jan 2020 20:31:04 +0100 Subject: [PATCH 32/85] Fixing compilation errors. --- dbms/src/Storages/LiveView/StorageLiveView.cpp | 13 ++++++++++--- dbms/src/Storages/LiveView/StorageLiveView.h | 4 ++++ 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 84ebb497243..e847c9b8b93 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -97,6 +97,11 @@ static void extractDependentTable(ASTPtr & query, String & select_database_name, BlocksPtrs StorageLiveView::collectMergeableBlocks(const Context & context) { + ASTPtr mergeable_query = inner_query; + + if (inner_subquery) + mergeable_query = inner_subquery; + BlocksPtrs new_mergeable_blocks = std::make_shared>(); BlocksPtr base_mergeable_blocks = std::make_shared(); @@ -110,9 +115,11 @@ BlocksPtrs StorageLiveView::collectMergeableBlocks(const Context & context) new_mergeable_blocks->push_back(base_mergeable_blocks); mergeable_blocks = new_mergeable_blocks; + + return mergeable_blocks; } -BlockInputStreams blocksToInputStreams(BlocksPtrs blocks) +BlockInputStreams StorageLiveView::blocksToInputStreams(BlocksPtrs blocks) { BlockInputStreams streams; @@ -131,7 +138,7 @@ BlockInputStreams blocksToInputStreams(BlocksPtrs blocks) } /// Complete query using input streams from mergeable blocks -BlockInputStreamPtr StorageLiveView::completeQuery(BlockInputStreams from) +BlockInputStreamPtr StorageLiveView::completeQuery(BlockInputStreams & from) { auto block_context = std::make_unique(global_context); block_context->makeQueryContext(); @@ -224,7 +231,7 @@ void StorageLiveView::writeIntoLiveView( } } - BlockInputStreamPtr data = completeQuery(std::move(from)); + BlockInputStreamPtr data = live_view.completeQuery(from); copyData(*data, *output); } diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index 46e5008ece3..49bff4bf2ca 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -140,8 +140,12 @@ public: std::shared_ptr getBlocksPtr() { return blocks_ptr; } BlocksPtrs getMergeableBlocks() { return mergeable_blocks; } + /// collect and set mergeable blocks. Must be called holding mutex BlocksPtrs collectMergeableBlocks(const Context & context); + /// Complete query using input streams from mergeable blocks + BlockInputStreamPtr completeQuery(BlockInputStreams & from); + void setMergeableBlocks(BlocksPtrs blocks) { mergeable_blocks = blocks; } std::shared_ptr getActivePtr() { return active_ptr; } From 25458a486589290549d251eaa1b23218682a6853 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Fri, 3 Jan 2020 02:48:15 +0100 Subject: [PATCH 33/85] Updates to make all live view tests to pass. --- .../src/Storages/LiveView/StorageLiveView.cpp | 87 ++++++------------- dbms/src/Storages/LiveView/StorageLiveView.h | 21 +++-- 2 files changed, 42 insertions(+), 66 deletions(-) diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index e847c9b8b93..75783364247 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -1,4 +1,4 @@ -/* iopyright (c) 2018 BlackBerry Limited +/* Copyright (c) 2018 BlackBerry Limited Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -95,50 +95,45 @@ static void extractDependentTable(ASTPtr & query, String & select_database_name, DB::ErrorCodes::LOGICAL_ERROR); } -BlocksPtrs StorageLiveView::collectMergeableBlocks(const Context & context) +MergeableBlocksPtr StorageLiveView::collectMergeableBlocks(const Context & context) { ASTPtr mergeable_query = inner_query; if (inner_subquery) mergeable_query = inner_subquery; - BlocksPtrs new_mergeable_blocks = std::make_shared>(); - BlocksPtr base_mergeable_blocks = std::make_shared(); + MergeableBlocksPtr new_mergeable_blocks = std::make_shared(); + BlocksPtrs new_blocks = std::make_shared>(); + BlocksPtr base_blocks = std::make_shared(); - InterpreterSelectQuery interpreter(mergeable_query, context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); + InterpreterSelectQuery interpreter(mergeable_query->clone(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); auto view_mergeable_stream = std::make_shared(interpreter.execute().in); while (Block this_block = view_mergeable_stream->read()) - base_mergeable_blocks->push_back(this_block); + base_blocks->push_back(this_block); - new_mergeable_blocks->push_back(base_mergeable_blocks); + new_blocks->push_back(base_blocks); - mergeable_blocks = new_mergeable_blocks; + new_mergeable_blocks->blocks = new_blocks; + new_mergeable_blocks->sample_block = view_mergeable_stream->getHeader(); - return mergeable_blocks; + return new_mergeable_blocks; } -BlockInputStreams StorageLiveView::blocksToInputStreams(BlocksPtrs blocks) +BlockInputStreams StorageLiveView::blocksToInputStreams(BlocksPtrs blocks, Block & sample_block) { BlockInputStreams streams; - for (auto & blocks_ : *blocks) { - if (blocks_->empty()) - continue; - - auto sample_block = blocks_->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); - streams.push_back(std::move(stream)); } return streams; } /// Complete query using input streams from mergeable blocks -BlockInputStreamPtr StorageLiveView::completeQuery(BlockInputStreams & from) +BlockInputStreamPtr StorageLiveView::completeQuery(BlockInputStreams from) { auto block_context = std::make_unique(global_context); block_context->makeQueryContext(); @@ -167,8 +162,6 @@ void StorageLiveView::writeIntoLiveView( const Context & context) { BlockOutputStreamPtr output = std::make_shared(live_view); - auto block_context = std::make_unique(context.getGlobalContext()); - block_context->makeQueryContext(); /// Check if live view has any readers if not /// just reset blocks to empty and do nothing else @@ -184,28 +177,30 @@ void StorageLiveView::writeIntoLiveView( bool is_block_processed = false; BlockInputStreams from; - BlocksPtrs mergeable_blocks; + MergeableBlocksPtr mergeable_blocks; BlocksPtr new_mergeable_blocks = std::make_shared(); - ASTPtr mergeable_query = live_view.getInnerQuery(); - - if (live_view.getInnerSubQuery()) - mergeable_query = live_view.getInnerSubQuery(); { std::lock_guard lock(live_view.mutex); mergeable_blocks = live_view.getMergeableBlocks(); - if (!mergeable_blocks || mergeable_blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh) + if (!mergeable_blocks || mergeable_blocks->blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh) { mergeable_blocks = live_view.collectMergeableBlocks(context); - from = live_view.blocksToInputStreams(mergeable_blocks); + live_view.setMergeableBlocks(mergeable_blocks); + from = live_view.blocksToInputStreams(mergeable_blocks->blocks, mergeable_blocks->sample_block); is_block_processed = true; } } if (!is_block_processed) { - BlockInputStreams streams = {std::make_shared(block)}; + ASTPtr mergeable_query = live_view.getInnerQuery(); + + if (live_view.getInnerSubQuery()) + mergeable_query = live_view.getInnerSubQuery(); + + BlockInputStreams streams = {std::make_shared(block)}; auto blocks_storage = StorageBlocks::createStorage(live_view.database_name, live_view.table_name, live_view.getParentStorage()->getColumns(), std::move(streams), QueryProcessingStage::FetchColumns); @@ -226,8 +221,8 @@ void StorageLiveView::writeIntoLiveView( std::lock_guard lock(live_view.mutex); mergeable_blocks = live_view.getMergeableBlocks(); - mergeable_blocks->push_back(new_mergeable_blocks); - from = live_view.blocksToInputStreams(mergeable_blocks); + mergeable_blocks->blocks->push_back(new_mergeable_blocks); + from = live_view.blocksToInputStreams(mergeable_blocks->blocks, mergeable_blocks->sample_block); } } @@ -329,36 +324,10 @@ bool StorageLiveView::getNewBlocks() UInt128 key; BlocksPtr new_blocks = std::make_shared(); BlocksMetadataPtr new_blocks_metadata = std::make_shared(); - BlocksPtr new_mergeable_blocks = std::make_shared(); - ASTPtr mergeable_query = inner_query; - if (inner_subquery) - mergeable_query = inner_subquery; - - InterpreterSelectQuery interpreter(mergeable_query->clone(), *live_view_context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); - auto mergeable_stream = std::make_shared(interpreter.execute().in); - - while (Block block = mergeable_stream->read()) - new_mergeable_blocks->push_back(block); - - auto block_context = std::make_unique(global_context); - block_context->makeQueryContext(); - - mergeable_blocks = std::make_shared>(); - mergeable_blocks->push_back(new_mergeable_blocks); - BlockInputStreamPtr from = std::make_shared(std::make_shared(new_mergeable_blocks), mergeable_stream->getHeader()); - - auto blocks_storage = StorageBlocks::createStorage(database_name, table_name, global_context.getTable(select_database_name, select_table_name)->getColumns(), {from}, QueryProcessingStage::WithMergeableState); - block_context->addExternalTable(table_name + "_blocks", blocks_storage); - - InterpreterSelectQuery select(inner_blocks_query->clone(), *block_context, StoragePtr(), SelectQueryOptions(QueryProcessingStage::Complete)); - BlockInputStreamPtr data = std::make_shared(select.execute().in); - - /// Squashing is needed here because the view query can generate a lot of blocks - /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY - /// and two-level aggregation is triggered). - data = std::make_shared( - data, global_context.getSettingsRef().min_insert_block_size_rows, global_context.getSettingsRef().min_insert_block_size_bytes); + mergeable_blocks = collectMergeableBlocks(*live_view_context); + BlockInputStreams from = blocksToInputStreams(mergeable_blocks->blocks, mergeable_blocks->sample_block); + BlockInputStreamPtr data = completeQuery({from}); while (Block block = data->read()) { diff --git a/dbms/src/Storages/LiveView/StorageLiveView.h b/dbms/src/Storages/LiveView/StorageLiveView.h index 49bff4bf2ca..916406a1dbd 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.h +++ b/dbms/src/Storages/LiveView/StorageLiveView.h @@ -27,9 +27,16 @@ struct BlocksMetadata UInt64 version; }; +struct MergeableBlocks +{ + BlocksPtrs blocks; + Block sample_block; +}; + class IAST; using ASTPtr = std::shared_ptr; using BlocksMetadataPtr = std::shared_ptr; +using MergeableBlocksPtr = std::shared_ptr; class StorageLiveView : public ext::shared_ptr_helper, public IStorage { @@ -139,14 +146,14 @@ public: unsigned num_streams) override; std::shared_ptr getBlocksPtr() { return blocks_ptr; } - BlocksPtrs getMergeableBlocks() { return mergeable_blocks; } + MergeableBlocksPtr getMergeableBlocks() { return mergeable_blocks; } - /// collect and set mergeable blocks. Must be called holding mutex - BlocksPtrs collectMergeableBlocks(const Context & context); + /// Collect mergeable blocks and their sample. Must be called holding mutex + MergeableBlocksPtr collectMergeableBlocks(const Context & context); /// Complete query using input streams from mergeable blocks - BlockInputStreamPtr completeQuery(BlockInputStreams & from); + BlockInputStreamPtr completeQuery(BlockInputStreams from); - void setMergeableBlocks(BlocksPtrs blocks) { mergeable_blocks = blocks; } + void setMergeableBlocks(MergeableBlocksPtr blocks) { mergeable_blocks = blocks; } std::shared_ptr getActivePtr() { return active_ptr; } /// Read new data blocks that store query result @@ -155,7 +162,7 @@ public: Block getHeader() const; /// convert blocks to input streams - static BlockInputStreams blocksToInputStreams(BlocksPtrs blocks); + static BlockInputStreams blocksToInputStreams(BlocksPtrs blocks, Block & sample_block); static void writeIntoLiveView( StorageLiveView & live_view, @@ -191,7 +198,7 @@ private: std::shared_ptr blocks_ptr; /// Current data blocks metadata std::shared_ptr blocks_metadata_ptr; - BlocksPtrs mergeable_blocks; + MergeableBlocksPtr mergeable_blocks; /// Background thread for temporary tables /// which drops this table if there are no users From 6a5e3e2dd9c585159264a2ac4ad2a6d5ab123655 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Sat, 4 Jan 2020 12:31:45 +0800 Subject: [PATCH 34/85] try fix failure test & fix build failure --- .../ExtractExpressionInfoVisitor.cpp | 14 ++++ .../ExtractExpressionInfoVisitor.h | 5 +- .../PredicateExpressionsOptimizer.cpp | 81 ++++++++++++++----- .../PredicateExpressionsOptimizer.h | 17 ++-- .../Interpreters/PredicateRewriteVisitor.cpp | 79 ++++++------------ .../Interpreters/PredicateRewriteVisitor.h | 9 +-- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 6 +- .../00597_push_down_predicate.reference | 2 +- 8 files changed, 115 insertions(+), 98 deletions(-) diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp index 4bd0446cd97..2bf954a7f9e 100644 --- a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp @@ -55,5 +55,19 @@ void ExpressionInfoMatcher::visit(const ASTIdentifier & identifier, const ASTPtr } } +bool hasStatefulFunction(const ASTPtr & node, const Context & context) +{ + for (const auto & select_expression : node->children) + { + ExpressionInfoVisitor::Data expression_info{.context = context, .tables = {}}; + ExpressionInfoVisitor(expression_info).visit(select_expression); + + if (expression_info.is_stateful_function) + return true; + } + + return false; +} + } diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h index 6ef9960b918..e884a27775a 100644 --- a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h @@ -21,7 +21,7 @@ struct ExpressionInfoMatcher bool is_array_join = false; bool is_stateful_function = false; bool is_aggregate_function = false; - std::unordered_set unique_reference_tables_pos; + std::unordered_set unique_reference_tables_pos = {}; }; static void visit(const ASTPtr & ast, Data & data); @@ -33,7 +33,8 @@ struct ExpressionInfoMatcher static void visit(const ASTIdentifier & identifier, const ASTPtr &, Data & data); }; - using ExpressionInfoVisitor = ConstInDepthNodeVisitor; +bool hasStatefulFunction(const ASTPtr & node, const Context & context); + } diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index a5cc578a98a..9927091874c 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -1,29 +1,13 @@ -#include - -#include -#include -#include -#include #include -#include + #include #include -#include #include -#include -#include #include -#include #include -#include -#include -#include -#include -#include -#include #include +#include #include -#include namespace DB @@ -43,7 +27,13 @@ PredicateExpressionsOptimizer::PredicateExpressionsOptimizer( bool PredicateExpressionsOptimizer::optimize(ASTSelectQuery & select_query) { - if (!settings.enable_optimize_predicate_expression || !select_query.tables() || select_query.tables()->children.empty()) + if (!settings.enable_optimize_predicate_expression) + return false; + + if (select_query.having() && (!select_query.group_by_with_cube && !select_query.group_by_with_rollup && !select_query.group_by_with_totals)) + tryMovePredicatesFromHavingToWhere(select_query); + + if (!select_query.tables() || select_query.tables()->children.empty()) return false; if ((!select_query.where() && !select_query.prewhere()) || select_query.array_join_expression_list()) @@ -165,14 +155,61 @@ bool PredicateExpressionsOptimizer::tryRewritePredicatesToTable(ASTPtr & table_e { if (!table_predicates.empty()) { - PredicateRewriteVisitor::Data data( - context, table_predicates, table_column, settings.enable_optimize_predicate_expression_to_final_subquery); + auto optimize_final = settings.enable_optimize_predicate_expression_to_final_subquery; + PredicateRewriteVisitor::Data data(context, table_predicates, table_column, optimize_final); PredicateRewriteVisitor(data).visit(table_element); - return data.isRewrite(); + return data.is_rewrite; } return false; } +bool PredicateExpressionsOptimizer::tryMovePredicatesFromHavingToWhere(ASTSelectQuery & select_query) +{ + ASTs where_predicates; + ASTs having_predicates; + + const auto & reduce_predicates = [&](const ASTs & predicates) + { + ASTPtr res = predicates[0]; + for (size_t index = 1; index < predicates.size(); ++index) + res = makeASTFunction("and", res, predicates[index]); + + return res; + }; + + for (const auto & moving_predicate: splitConjunctionPredicate({select_query.having()})) + { + ExpressionInfoVisitor::Data expression_info{.context = context, .tables = {}}; + ExpressionInfoVisitor(expression_info).visit(moving_predicate); + + /// TODO: If there is no group by, where, and prewhere expression, we can push down the stateful function + if (expression_info.is_stateful_function) + return false; + + if (expression_info.is_aggregate_function) + having_predicates.emplace_back(moving_predicate); + else + where_predicates.emplace_back(moving_predicate); + } + + if (having_predicates.empty()) + select_query.setExpression(ASTSelectQuery::Expression::HAVING, {}); + else + { + auto having_predicate = reduce_predicates(having_predicates); + select_query.setExpression(ASTSelectQuery::Expression::HAVING, std::move(having_predicate)); + } + + if (!where_predicates.empty()) + { + auto moved_predicate = reduce_predicates(where_predicates); + moved_predicate = select_query.where() ? makeASTFunction("and", select_query.where(), moved_predicate) : moved_predicate; + select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(moved_predicate)); + } + + return true; +} + } diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h index 19010c3a328..da6b98987a6 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h @@ -7,18 +7,13 @@ namespace DB { class Context; -class Settings; +struct Settings; -/** This class provides functions for Push-Down predicate expressions - * - * The Example: - * - Query before optimization : - * SELECT id_1, name_1 FROM (SELECT id_1, name_1 FROM table_a UNION ALL SELECT id_2, name_2 FROM table_b) - * WHERE id_1 = 1 - * - Query after optimization : - * SELECT id_1, name_1 FROM (SELECT id_1, name_1 FROM table_a WHERE id_1 = 1 UNION ALL SELECT id_2, name_2 FROM table_b WHERE id_2 = 1) - * WHERE id_1 = 1 +/** Predicate optimization based on rewriting ast rules * For more details : https://github.com/ClickHouse/ClickHouse/pull/2015#issuecomment-374283452 + * The optimizer does two different optimizations + * - Move predicates from having to where + * - Push the predicate down from the current query to the having of the subquery */ class PredicateExpressionsOptimizer { @@ -51,6 +46,8 @@ private: bool tryRewritePredicatesToTables(ASTs & tables_element, const std::vector & tables_predicates); bool tryRewritePredicatesToTable(ASTPtr & table_element, const ASTs & table_predicates, const Names & table_column) const; + + bool tryMovePredicatesFromHavingToWhere(ASTSelectQuery & select_query); }; } diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp index c968f65b008..f009edec937 100644 --- a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp @@ -1,23 +1,23 @@ #include +#include #include +#include #include #include +#include #include #include -#include -#include -#include -#include #include +#include namespace DB { PredicateRewriteVisitorData::PredicateRewriteVisitorData( - const Context & context_, const ASTs & predicates_, const Names & colunm_names_, bool optimize_final_) - : context(context_), predicates(predicates_), column_names(colunm_names_), optimize_final(optimize_final_) + const Context & context_, const ASTs & predicates_, const Names & column_names_, bool optimize_final_) + : context(context_), predicates(predicates_), column_names(column_names_), optimize_final(optimize_final_) { } @@ -44,9 +44,10 @@ void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & sele ASTSelectQuery * temp_select_query = temp_internal_select->as(); size_t alias_index = 0; - for (const auto ref_select : temp_select_query->refSelect()->children) + for (auto & ref_select : temp_select_query->refSelect()->children) { - if (!ref_select->as() && !ref_select->as() && !ref_select->as()) + if (!ref_select->as() && !ref_select->as() && !ref_select->as() && + !ref_select->as()) { if (const auto & alias = ref_select->tryGetAlias(); alias.empty()) ref_select->setAlias("--predicate_optimizer_" + toString(alias_index++)); @@ -56,15 +57,11 @@ void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & sele const Names & internal_columns = InterpreterSelectQuery( temp_internal_select, context, SelectQueryOptions().analyze()).getSampleBlock().getNames(); - if ((is_rewrite |= rewriteSubquery(*temp_select_query, column_names, internal_columns))) + if (rewriteSubquery(*temp_select_query, column_names, internal_columns)) { + is_rewrite |= true; select_query.setExpression(ASTSelectQuery::Expression::SELECT, std::move(temp_select_query->refSelect())); - - if (temp_select_query->where()) - select_query.setExpression(ASTSelectQuery::Expression::WHERE, std::move(temp_select_query->refWhere())); - - if (temp_select_query->having()) - select_query.setExpression(ASTSelectQuery::Expression::HAVING, std::move(temp_select_query->refHaving())); + select_query.setExpression(ASTSelectQuery::Expression::HAVING, std::move(temp_select_query->refHaving())); } } @@ -80,32 +77,12 @@ static void cleanAliasAndCollectIdentifiers(ASTPtr & predicate, std::vectorchildren) - { - ExpressionInfoVisitor::Data expression_info{.context = context, .tables = {}}; - ExpressionInfoVisitor(expression_info).visit(select_expression); - - if (expression_info.is_stateful_function) - return false; - else if (expression_info.is_aggregate_function) - aggregate_column.emplace(select_expression->getAliasOrColumnName()); - } - - return true; -} - bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, const Names & outer_columns, const Names & inner_columns) { - NameSet aggregate_columns; - - if (!allowPushDown(subquery, aggregate_columns)) + if ((!optimize_final && subquery.final()) + || subquery.with() || subquery.withFill() + || subquery.limitBy() || subquery.limitLength() + || hasStatefulFunction(subquery.select(), context)) return false; for (const auto & predicate : predicates) @@ -114,27 +91,23 @@ bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, con ASTPtr optimize_predicate = predicate->clone(); cleanAliasAndCollectIdentifiers(optimize_predicate, identifiers); - ASTSelectQuery::Expression rewrite_to = ASTSelectQuery::Expression::WHERE; - for (size_t index = 0; index < identifiers.size(); ++index) { - const auto & column_name = IdentifierSemantic::getColumnName(*identifiers[index]); + const auto & column_name = identifiers[index]->shortName(); + const auto & outer_column_iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); - const auto & iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); + if (outer_column_iterator == outer_columns.end()) + throw Exception("LOGICAL ERROR: the column " + column_name + " does not exists.", ErrorCodes::LOGICAL_ERROR); - if (iterator == outer_columns.end()) - throw Exception("", ErrorCodes::LOGICAL_ERROR); - - if (aggregate_columns.count(*column_name)) - rewrite_to = ASTSelectQuery::Expression::HAVING; - - identifiers[index]->setShortName(inner_columns[iterator - outer_columns.begin()]); + identifiers[index]->setShortName(inner_columns[outer_column_iterator - outer_columns.begin()]); } - ASTPtr optimize_expression = subquery.getExpression(rewrite_to, false); - subquery.setExpression(rewrite_to, - optimize_expression ? makeASTFunction("and", optimize_predicate, optimize_expression) : optimize_predicate); + /// We only need to push all the predicates to subquery having + /// The subquery optimizer will move the appropriate predicates from having to where + subquery.setExpression(ASTSelectQuery::Expression::HAVING, + subquery.having() ? makeASTFunction("and", optimize_predicate, subquery.having()) : optimize_predicate); } + return true; } diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.h b/dbms/src/Interpreters/PredicateRewriteVisitor.h index ea7656256fe..e07df922c15 100644 --- a/dbms/src/Interpreters/PredicateRewriteVisitor.h +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.h @@ -11,28 +11,23 @@ namespace DB class PredicateRewriteVisitorData { public: + bool is_rewrite = false; using TypeToVisit = ASTSelectWithUnionQuery; - bool isRewrite() const { return is_rewrite; } - void visit(ASTSelectWithUnionQuery & union_select_query, ASTPtr &); - PredicateRewriteVisitorData(const Context & context_, const ASTs & predicates_, const Names & colunm_names_, bool optimize_final_); + PredicateRewriteVisitorData(const Context & context_, const ASTs & predicates_, const Names & column_names_, bool optimize_final_); private: const Context & context; const ASTs & predicates; const Names & column_names; - bool optimize_final; - bool is_rewrite = false; void visitFirstInternalSelect(ASTSelectQuery & select_query, ASTPtr &); void visitOtherInternalSelect(ASTSelectQuery & select_query, ASTPtr &); - bool allowPushDown(const ASTSelectQuery & subquery, NameSet & aggregate_column); - bool rewriteSubquery(ASTSelectQuery & subquery, const Names & outer_columns, const Names & inner_columns); }; diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 678f42adf88..3680947c8f1 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -920,6 +920,9 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( if (select_query) { + /// Push the predicate expression down to the subqueries. + result.rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); + /// GROUP BY injective function elimination. optimizeGroupBy(select_query, source_columns_set, context); @@ -935,9 +938,6 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( /// array_join_alias_to_name, array_join_result_to_source. getArrayJoinedColumns(query, result, select_query, result.source_columns, source_columns_set); - /// Push the predicate expression down to the subqueries. - result.rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); - setJoinStrictness(*select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys, result.analyzed_join->table_join); collectJoinedColumns(*result.analyzed_join, *select_query, tables_with_columns, result.aliases); diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference index f64243e9be7..9fde80689f1 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -13,7 +13,7 @@ SELECT \n a, \n b\nFROM \n(\n SELECT \n 1 AS a, \n 1 AS b -------Need push down------- SELECT toString(value) AS value\nFROM \n(\n SELECT 1 AS value\n) 1 -SELECT id\nFROM \n(\n SELECT 1 AS id\n UNION ALL\n SELECT 2 AS `2`\n WHERE 0\n)\nWHERE id = 1 +SELECT id\nFROM \n(\n SELECT 1 AS id\n UNION ALL\n SELECT 2 AS `--predicate_optimizer_0`\n WHERE 0\n)\nWHERE id = 1 1 SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n)\nWHERE id = 1 1 From fb8903259ec68428d00a27466267c46b07e1f1de Mon Sep 17 00:00:00 2001 From: Ramazan Polat Date: Sat, 4 Jan 2020 23:16:43 +0300 Subject: [PATCH 35/85] typo fixed. hyperlink added. --- docs/en/query_language/functions/conditional_functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/query_language/functions/conditional_functions.md b/docs/en/query_language/functions/conditional_functions.md index a40675dbc81..8b46566af31 100644 --- a/docs/en/query_language/functions/conditional_functions.md +++ b/docs/en/query_language/functions/conditional_functions.md @@ -42,7 +42,7 @@ WHERE isNotNull(left) AND isNotNull(right) │ 3 │ 1 │ right is greater or equal than left │ └──────┴───────┴─────────────────────────────────────┘ ``` -Note: `NULL` values are not used in this example, check *`NULL` values in conditinals"* section below. +Note: `NULL` values are not used in this example, check [NULL values in conditionals](#null-values-in-conditionals) section. ## Ternary operator @@ -111,7 +111,7 @@ FROM LEFT_RIGHT ``` -## `NULL` values in conditinals +## NULL values in conditionals When `NULL` values are involved in conditionals, the result will also be `NULL`. From f8aa0b90b1298a71e442039d1dd7887479aa5cc2 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 6 Jan 2020 11:55:07 +0800 Subject: [PATCH 36/85] add bugs test --- .../Interpreters/PredicateRewriteVisitor.cpp | 7 ++- .../01056_predicate_optimizer_bugs.reference | 18 ++++++++ .../01056_predicate_optimizer_bugs.sql | 45 +++++++++++++++++++ 3 files changed, 69 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference create mode 100644 dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp index f009edec937..781c1510679 100644 --- a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -68,7 +69,11 @@ void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & sele static void cleanAliasAndCollectIdentifiers(ASTPtr & predicate, std::vector & identifiers) { for (auto & children : predicate->children) - cleanAliasAndCollectIdentifiers(children, identifiers); + { + if (!children->as()) /// skip where x in (SELECT ...) + cleanAliasAndCollectIdentifiers(children, identifiers); + } + if (const auto alias = predicate->tryGetAlias(); !alias.empty()) predicate->setAlias(""); diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference new file mode 100644 index 00000000000..289639d35ab --- /dev/null +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -0,0 +1,18 @@ +SELECT count()\nFROM \n(\n SELECT \n a, \n runningDifference(a) AS rd\n FROM \n (\n SELECT a\n FROM \n (\n SELECT 1 AS a\n UNION ALL\n SELECT 2 AS a\n )\n ORDER BY a DESC\n )\n WHERE rd = -1\n) +1 +SELECT \n k, \n v, \n d, \n i\nFROM \n(\n SELECT \n t.1 AS k, \n t.2 AS v, \n runningDifference(v) AS d, \n runningDifference(xxHash32(t.1)) AS i\n FROM \n (\n SELECT arrayJoin([(\'a\', 1), (\'a\', 2), (\'a\', 3), (\'b\', 11), (\'b\', 13), (\'b\', 15)]) AS t\n )\n)\nWHERE i = 0 +a 1 0 0 +a 2 1 0 +a 3 1 0 +b 13 2 0 +b 15 2 0 +SELECT \n co, \n co2, \n co3, \n num\nFROM \n(\n SELECT \n co, \n co2, \n co3, \n count() AS num\n FROM \n (\n SELECT \n 1 AS co, \n 2 AS co2, \n 3 AS co3\n )\n GROUP BY \n co, \n co2, \n co3\n WITH CUBE\n HAVING (co != 0) AND (co2 != 2)\n)\nWHERE (co != 0) AND (co2 != 2) +1 0 3 1 +1 0 0 1 +SELECT alias AS name\nFROM \n(\n SELECT name AS alias\n FROM system.settings\n WHERE alias = \'enable_optimize_predicate_expression\'\n)\nANY INNER JOIN \n(\n SELECT name\n FROM system.settings\n) USING (name)\nWHERE name = \'enable_optimize_predicate_expression\' +enable_optimize_predicate_expression +1 val11 val21 val31 +SELECT ccc\nFROM \n(\n SELECT 1 AS ccc\n WHERE 0\n UNION ALL\n SELECT ccc\n FROM \n (\n SELECT 2 AS ccc\n )\n ANY INNER JOIN \n (\n SELECT 2 AS ccc\n ) USING (ccc)\n WHERE ccc > 1\n)\nWHERE ccc > 1 +2 +SELECT \n ts, \n id, \n id_b, \n b.ts, \n b.id, \n id_c\nFROM \n(\n SELECT \n ts, \n id, \n id_b\n FROM A\n WHERE ts <= toDateTime(\'1970-01-01 03:00:00\')\n) AS a\nALL LEFT JOIN B AS b ON b.id = id_b\nWHERE ts <= toDateTime(\'1970-01-01 03:00:00\') +SELECT \n ts AS `--a.ts`, \n id AS `--a.id`, \n id_b AS `--a.id_b`, \n b.ts AS `--b.ts`, \n b.id AS `--b.id`, \n id_c AS `--b.id_c`\nFROM \n(\n SELECT \n ts, \n id, \n id_b\n FROM A\n WHERE ts <= toDateTime(\'1970-01-01 03:00:00\')\n) AS a\nALL LEFT JOIN B AS b ON `--b.id` = `--a.id_b`\nWHERE `--a.ts` <= toDateTime(\'1970-01-01 03:00:00\') diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql new file mode 100644 index 00000000000..301ad2a707b --- /dev/null +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -0,0 +1,45 @@ +SET enable_debug_queries = 1; +SET enable_optimize_predicate_expression = 1; + +-- https://github.com/ClickHouse/ClickHouse/issues/3885 +-- https://github.com/ClickHouse/ClickHouse/issues/5485 +ANALYZE SELECT count() FROM (SELECT a, runningDifference(a) rd FROM ( SELECT * FROM ( SELECT 1 a UNION ALL SELECT 2 a ) ORDER BY a DESC ) WHERE rd=-1 ); +SELECT count() FROM (SELECT a, runningDifference(a) rd FROM ( SELECT * FROM ( SELECT 1 a UNION ALL SELECT 2 a ) ORDER BY a DESC ) WHERE rd=-1 ); + +ANALYZE SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(xxHash32(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; +SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(xxHash32(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; + +-- https://github.com/ClickHouse/ClickHouse/issues/5682 +ANALYZE SELECT co,co2,co3,num FROM ( SELECT co,co2,co3,count() AS num FROM ( SELECT 1 AS co,2 AS co2 ,3 AS co3 ) GROUP BY cube (co,co2,co3) ) WHERE co!=0 AND co2 !=2; +SELECT co,co2,co3,num FROM ( SELECT co,co2,co3,count() AS num FROM ( SELECT 1 AS co,2 AS co2 ,3 AS co3 ) GROUP BY cube (co,co2,co3) ) WHERE co!=0 AND co2 !=2; + +-- https://github.com/ClickHouse/ClickHouse/issues/6734 +ANALYZE SELECT alias AS name FROM ( SELECT name AS alias FROM system.settings ) ANY INNER JOIN ( SELECT name FROM system.settings ) USING (name) WHERE name = 'enable_optimize_predicate_expression'; +SELECT alias AS name FROM ( SELECT name AS alias FROM system.settings ) ANY INNER JOIN ( SELECT name FROM system.settings ) USING (name) WHERE name = 'enable_optimize_predicate_expression'; + +-- https://github.com/ClickHouse/ClickHouse/issues/6767 +CREATE TABLE t1 (id UInt32, value1 String ) ENGINE ReplacingMergeTree() ORDER BY id; +CREATE TABLE t2 (id UInt32, value2 String ) ENGINE ReplacingMergeTree() ORDER BY id; +CREATE TABLE t3 (id UInt32, value3 String ) ENGINE ReplacingMergeTree() ORDER BY id; + +INSERT INTO t1 (id, value1) VALUES (1, 'val11'); +INSERT INTO t2 (id, value2) VALUES (1, 'val21'); +INSERT INTO t3 (id, value3) VALUES (1, 'val31'); + +CREATE VIEW IF NOT EXISTS view1 AS SELECT t1.id AS id, t1.value1 AS value1, t2.value2 AS value2, t3.value3 AS value3 FROM t1 LEFT JOIN t2 ON t1.id = t2.id LEFT JOIN t3 ON t1.id = t3.id WHERE t1.id > 0; +SELECT * FROM view1 WHERE id = 1; + +-- https://github.com/ClickHouse/ClickHouse/issues/7136 +ANALYZE SELECT ccc FROM ( SELECT 1 AS ccc UNION ALL SELECT * FROM ( SELECT 2 AS ccc ) ANY INNER JOIN ( SELECT 2 AS ccc ) USING (ccc) ) WHERE ccc > 1; +SELECT ccc FROM ( SELECT 1 AS ccc UNION ALL SELECT * FROM ( SELECT 2 AS ccc ) ANY INNER JOIN ( SELECT 2 AS ccc ) USING (ccc) ) WHERE ccc > 1; + +-- https://github.com/ClickHouse/ClickHouse/issues/5674 +-- https://github.com/ClickHouse/ClickHouse/issues/4731 +-- https://github.com/ClickHouse/ClickHouse/issues/4904 +CREATE TABLE A (ts DateTime, id String, id_b String) ENGINE = MergeTree PARTITION BY toStartOfHour(ts) ORDER BY (ts,id); +CREATE TABLE B (ts DateTime, id String, id_c String) ENGINE = MergeTree PARTITION BY toStartOfHour(ts) ORDER BY (ts,id); + +ANALYZE SELECT ts, id, id_b, b.ts, b.id, id_c FROM (SELECT ts, id, id_b FROM A) AS a ALL LEFT JOIN B AS b ON b.id = a.id_b WHERE a.ts <= toDateTime('1970-01-01 03:00:00'); +ANALYZE SELECT ts AS `--a.ts`, id AS `--a.id`, id_b AS `--a.id_b`, b.ts AS `--b.ts`, b.id AS `--b.id`, id_c AS `--b.id_c` FROM (SELECT ts, id, id_b FROM A) AS a ALL LEFT JOIN B AS b ON `--b.id` = `--a.id_b` WHERE `--a.ts` <= toDateTime('1970-01-01 03:00:00'); + + From ce6e13549a0ff5f6176979bc8eb665e72327832d Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 6 Jan 2020 12:18:56 +0800 Subject: [PATCH 37/85] fix neighbor is stateful function --- dbms/src/Functions/neighbor.cpp | 2 ++ .../01056_predicate_optimizer_bugs.reference | 8 +++++ .../01056_predicate_optimizer_bugs.sql | 29 +++++++++++++++++++ 3 files changed, 39 insertions(+) diff --git a/dbms/src/Functions/neighbor.cpp b/dbms/src/Functions/neighbor.cpp index 0253aed65d3..c37a3313a80 100644 --- a/dbms/src/Functions/neighbor.cpp +++ b/dbms/src/Functions/neighbor.cpp @@ -40,6 +40,8 @@ public: bool isVariadic() const override { return true; } + bool isStateful() const override { return true; } + bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 289639d35ab..932651741e3 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -16,3 +16,11 @@ SELECT ccc\nFROM \n(\n SELECT 1 AS ccc\n WHERE 0\n UNION ALL\n SELEC 2 SELECT \n ts, \n id, \n id_b, \n b.ts, \n b.id, \n id_c\nFROM \n(\n SELECT \n ts, \n id, \n id_b\n FROM A\n WHERE ts <= toDateTime(\'1970-01-01 03:00:00\')\n) AS a\nALL LEFT JOIN B AS b ON b.id = id_b\nWHERE ts <= toDateTime(\'1970-01-01 03:00:00\') SELECT \n ts AS `--a.ts`, \n id AS `--a.id`, \n id_b AS `--a.id_b`, \n b.ts AS `--b.ts`, \n b.id AS `--b.id`, \n id_c AS `--b.id_c`\nFROM \n(\n SELECT \n ts, \n id, \n id_b\n FROM A\n WHERE ts <= toDateTime(\'1970-01-01 03:00:00\')\n) AS a\nALL LEFT JOIN B AS b ON `--b.id` = `--a.id_b`\nWHERE `--a.ts` <= toDateTime(\'1970-01-01 03:00:00\') +2 3 +3 4 +4 5 +5 0 +2 4 +4 0 +2 3 +4 5 diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index 301ad2a707b..c6a79924291 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -18,6 +18,11 @@ ANALYZE SELECT alias AS name FROM ( SELECT name AS alias FROM system.settings ) SELECT alias AS name FROM ( SELECT name AS alias FROM system.settings ) ANY INNER JOIN ( SELECT name FROM system.settings ) USING (name) WHERE name = 'enable_optimize_predicate_expression'; -- https://github.com/ClickHouse/ClickHouse/issues/6767 +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; +DROP TABLE IF EXISTS view1; + CREATE TABLE t1 (id UInt32, value1 String ) ENGINE ReplacingMergeTree() ORDER BY id; CREATE TABLE t2 (id UInt32, value2 String ) ENGINE ReplacingMergeTree() ORDER BY id; CREATE TABLE t3 (id UInt32, value3 String ) ENGINE ReplacingMergeTree() ORDER BY id; @@ -29,6 +34,11 @@ INSERT INTO t3 (id, value3) VALUES (1, 'val31'); CREATE VIEW IF NOT EXISTS view1 AS SELECT t1.id AS id, t1.value1 AS value1, t2.value2 AS value2, t3.value3 AS value3 FROM t1 LEFT JOIN t2 ON t1.id = t2.id LEFT JOIN t3 ON t1.id = t3.id WHERE t1.id > 0; SELECT * FROM view1 WHERE id = 1; +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t3; +DROP TABLE IF EXISTS view1; + -- https://github.com/ClickHouse/ClickHouse/issues/7136 ANALYZE SELECT ccc FROM ( SELECT 1 AS ccc UNION ALL SELECT * FROM ( SELECT 2 AS ccc ) ANY INNER JOIN ( SELECT 2 AS ccc ) USING (ccc) ) WHERE ccc > 1; SELECT ccc FROM ( SELECT 1 AS ccc UNION ALL SELECT * FROM ( SELECT 2 AS ccc ) ANY INNER JOIN ( SELECT 2 AS ccc ) USING (ccc) ) WHERE ccc > 1; @@ -36,10 +46,29 @@ SELECT ccc FROM ( SELECT 1 AS ccc UNION ALL SELECT * FROM ( SELECT 2 AS ccc ) AN -- https://github.com/ClickHouse/ClickHouse/issues/5674 -- https://github.com/ClickHouse/ClickHouse/issues/4731 -- https://github.com/ClickHouse/ClickHouse/issues/4904 +DROP TABLE IF EXISTS A; +DROP TABLE IF EXISTS B; + CREATE TABLE A (ts DateTime, id String, id_b String) ENGINE = MergeTree PARTITION BY toStartOfHour(ts) ORDER BY (ts,id); CREATE TABLE B (ts DateTime, id String, id_c String) ENGINE = MergeTree PARTITION BY toStartOfHour(ts) ORDER BY (ts,id); ANALYZE SELECT ts, id, id_b, b.ts, b.id, id_c FROM (SELECT ts, id, id_b FROM A) AS a ALL LEFT JOIN B AS b ON b.id = a.id_b WHERE a.ts <= toDateTime('1970-01-01 03:00:00'); ANALYZE SELECT ts AS `--a.ts`, id AS `--a.id`, id_b AS `--a.id_b`, b.ts AS `--b.ts`, b.id AS `--b.id`, id_c AS `--b.id_c` FROM (SELECT ts, id, id_b FROM A) AS a ALL LEFT JOIN B AS b ON `--b.id` = `--a.id_b` WHERE `--a.ts` <= toDateTime('1970-01-01 03:00:00'); +DROP TABLE IF EXISTS A; +DROP TABLE IF EXISTS B; + +-- https://github.com/ClickHouse/ClickHouse/issues/7802 +DROP TABLE IF EXISTS test; + +CREATE TABLE test ( A Int32, B Int32 ) ENGINE = Memory(); + +INSERT INTO test VALUES(1, 2)(0, 3)(1, 4)(0, 5); + +SELECT B, neighbor(B, 1) AS next_B FROM (SELECT * FROM test ORDER BY B); +SELECT B, neighbor(B, 1) AS next_B FROM (SELECT * FROM test ORDER BY B) WHERE A == 1; +SELECT B, next_B FROM (SELECT A, B, neighbor(B, 1) AS next_B FROM (SELECT * FROM test ORDER BY B)) WHERE A == 1; + +DROP TABLE IF EXISTS test; + From 6d3e56def4fa50c6d94e9517dfabb4dc1aec8abf Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 6 Jan 2020 18:07:41 +0800 Subject: [PATCH 38/85] fix unbundled failure test --- .../0_stateless/01056_predicate_optimizer_bugs.reference | 2 +- .../queries/0_stateless/01056_predicate_optimizer_bugs.sql | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 932651741e3..3e1791fabb8 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -1,6 +1,6 @@ SELECT count()\nFROM \n(\n SELECT \n a, \n runningDifference(a) AS rd\n FROM \n (\n SELECT a\n FROM \n (\n SELECT 1 AS a\n UNION ALL\n SELECT 2 AS a\n )\n ORDER BY a DESC\n )\n WHERE rd = -1\n) 1 -SELECT \n k, \n v, \n d, \n i\nFROM \n(\n SELECT \n t.1 AS k, \n t.2 AS v, \n runningDifference(v) AS d, \n runningDifference(xxHash32(t.1)) AS i\n FROM \n (\n SELECT arrayJoin([(\'a\', 1), (\'a\', 2), (\'a\', 3), (\'b\', 11), (\'b\', 13), (\'b\', 15)]) AS t\n )\n)\nWHERE i = 0 +SELECT \n k, \n v, \n d, \n i\nFROM \n(\n SELECT \n t.1 AS k, \n t.2 AS v, \n runningDifference(v) AS d, \n runningDifference(cityHash64(t.1)) AS i\n FROM \n (\n SELECT arrayJoin([(\'a\', 1), (\'a\', 2), (\'a\', 3), (\'b\', 11), (\'b\', 13), (\'b\', 15)]) AS t\n )\n)\nWHERE i = 0 a 1 0 0 a 2 1 0 a 3 1 0 diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index c6a79924291..59943742b3c 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -6,8 +6,8 @@ SET enable_optimize_predicate_expression = 1; ANALYZE SELECT count() FROM (SELECT a, runningDifference(a) rd FROM ( SELECT * FROM ( SELECT 1 a UNION ALL SELECT 2 a ) ORDER BY a DESC ) WHERE rd=-1 ); SELECT count() FROM (SELECT a, runningDifference(a) rd FROM ( SELECT * FROM ( SELECT 1 a UNION ALL SELECT 2 a ) ORDER BY a DESC ) WHERE rd=-1 ); -ANALYZE SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(xxHash32(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; -SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(xxHash32(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; +ANALYZE SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; +SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; -- https://github.com/ClickHouse/ClickHouse/issues/5682 ANALYZE SELECT co,co2,co3,num FROM ( SELECT co,co2,co3,count() AS num FROM ( SELECT 1 AS co,2 AS co2 ,3 AS co3 ) GROUP BY cube (co,co2,co3) ) WHERE co!=0 AND co2 !=2; From 2179935f1e7ccb772f3e89b1988b0962557e5e1f Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 6 Jan 2020 18:33:08 +0800 Subject: [PATCH 39/85] fix lambda function in predicate --- dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp | 6 ++++++ dbms/src/Interpreters/ExtractExpressionInfoVisitor.h | 2 +- dbms/src/Interpreters/PredicateRewriteVisitor.cpp | 9 +++++---- .../0_stateless/01056_predicate_optimizer_bugs.reference | 2 ++ .../0_stateless/01056_predicate_optimizer_bugs.sql | 2 ++ 5 files changed, 16 insertions(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp index 2bf954a7f9e..1240b6a09d6 100644 --- a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -55,6 +56,11 @@ void ExpressionInfoMatcher::visit(const ASTIdentifier & identifier, const ASTPtr } } +bool ExpressionInfoMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) +{ + return !node->as(); +} + bool hasStatefulFunction(const ASTPtr & node, const Context & context) { for (const auto & select_expression : node->children) diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h index e884a27775a..65d23057e52 100644 --- a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.h @@ -26,7 +26,7 @@ struct ExpressionInfoMatcher static void visit(const ASTPtr & ast, Data & data); - static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } + static bool needChildVisit(const ASTPtr & node, const ASTPtr &); static void visit(const ASTFunction & ast_function, const ASTPtr &, Data & data); diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp index 781c1510679..2575050825f 100644 --- a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp @@ -101,10 +101,11 @@ bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, con const auto & column_name = identifiers[index]->shortName(); const auto & outer_column_iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); - if (outer_column_iterator == outer_columns.end()) - throw Exception("LOGICAL ERROR: the column " + column_name + " does not exists.", ErrorCodes::LOGICAL_ERROR); - - identifiers[index]->setShortName(inner_columns[outer_column_iterator - outer_columns.begin()]); + if (outer_column_iterator != outer_columns.end()) + { + /// Some temporary identifiers may be included in the predicate, for example: WHERE arrayMap(x -> x, [column_a]) = [column_a] + identifiers[index]->setShortName(inner_columns[outer_column_iterator - outer_columns.begin()]); + } } /// We only need to push all the predicates to subquery having diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 3e1791fabb8..34cab3eb9b8 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -24,3 +24,5 @@ SELECT \n ts AS `--a.ts`, \n id AS `--a.id`, \n id_b AS `--a.id_b`, \n 4 0 2 3 4 5 +SELECT dummy\nFROM \n(\n SELECT dummy\n FROM system.one\n WHERE arrayMap(x -> (x + 1), [dummy]) = [1]\n)\nWHERE arrayMap(x -> (x + 1), [dummy]) = [1] +0 diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index 59943742b3c..b0da6a2d097 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -71,4 +71,6 @@ SELECT B, next_B FROM (SELECT A, B, neighbor(B, 1) AS next_B FROM (SELECT * FROM DROP TABLE IF EXISTS test; +ANALYZE SELECT * FROM (SELECT * FROM system.one) WHERE arrayMap(x -> x + 1, [dummy]) = [1]; +SELECT * FROM (SELECT * FROM system.one) WHERE arrayMap(x -> x + 1, [dummy]) = [1]; From 0f147e393f3bcab0984e593dc1d26624e3c0fdfa Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Mon, 6 Jan 2020 18:56:17 +0800 Subject: [PATCH 40/85] better skip lambda function --- dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp | 3 +++ dbms/src/Interpreters/PredicateRewriteVisitor.cpp | 10 ++++------ 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp index 1240b6a09d6..a5bcea73aac 100644 --- a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp @@ -58,6 +58,9 @@ void ExpressionInfoMatcher::visit(const ASTIdentifier & identifier, const ASTPtr bool ExpressionInfoMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) { + if (const auto & function = node->as(); function && function->name == "lambda") + return false; + return !node->as(); } diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp index 2575050825f..ec2088254c6 100644 --- a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp @@ -68,13 +68,14 @@ void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & sele static void cleanAliasAndCollectIdentifiers(ASTPtr & predicate, std::vector & identifiers) { - for (auto & children : predicate->children) + /// skip WHERE x in (SELECT ...) AND arrayMap(x -> x, [column_a]) + const auto & function = predicate->as(); + if (!predicate->as() && !(function && function->name == "lambda")) { - if (!children->as()) /// skip where x in (SELECT ...) + for (auto & children : predicate->children) cleanAliasAndCollectIdentifiers(children, identifiers); } - if (const auto alias = predicate->tryGetAlias(); !alias.empty()) predicate->setAlias(""); @@ -102,10 +103,7 @@ bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, con const auto & outer_column_iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); if (outer_column_iterator != outer_columns.end()) - { - /// Some temporary identifiers may be included in the predicate, for example: WHERE arrayMap(x -> x, [column_a]) = [column_a] identifiers[index]->setShortName(inner_columns[outer_column_iterator - outer_columns.begin()]); - } } /// We only need to push all the predicates to subquery having From f0d2ef5bf7a1dd326965dfa1ee5e256e9497ba92 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Mon, 6 Jan 2020 08:23:05 -0500 Subject: [PATCH 41/85] Removing tabs. --- dbms/src/Storages/LiveView/StorageLiveView.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/LiveView/StorageLiveView.cpp b/dbms/src/Storages/LiveView/StorageLiveView.cpp index 75783364247..eae8eaa1d3c 100644 --- a/dbms/src/Storages/LiveView/StorageLiveView.cpp +++ b/dbms/src/Storages/LiveView/StorageLiveView.cpp @@ -200,7 +200,7 @@ void StorageLiveView::writeIntoLiveView( if (live_view.getInnerSubQuery()) mergeable_query = live_view.getInnerSubQuery(); - BlockInputStreams streams = {std::make_shared(block)}; + BlockInputStreams streams = {std::make_shared(block)}; auto blocks_storage = StorageBlocks::createStorage(live_view.database_name, live_view.table_name, live_view.getParentStorage()->getColumns(), std::move(streams), QueryProcessingStage::FetchColumns); From 7fc8b59d490b723850ceea7389409c64dc7d46a9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Jan 2020 19:26:20 +0300 Subject: [PATCH 42/85] Update ConvertingAggregatedToChunksTransform transform. --- .../Transforms/AggregatingTransform.cpp | 139 +++++++----------- 1 file changed, 52 insertions(+), 87 deletions(-) diff --git a/dbms/src/Processors/Transforms/AggregatingTransform.cpp b/dbms/src/Processors/Transforms/AggregatingTransform.cpp index 72a5ff3bb7c..1b562510bd7 100644 --- a/dbms/src/Processors/Transforms/AggregatingTransform.cpp +++ b/dbms/src/Processors/Transforms/AggregatingTransform.cpp @@ -72,15 +72,33 @@ namespace class ConvertingAggregatedToChunksSource : public ISource { public: + static constexpr UInt32 NUM_BUCKETS = 256; + + struct SharedData + { + std::atomic next_bucket_to_merge = 0; + std::array, NUM_BUCKETS> source_for_bucket; + + SharedData() + { + for (auto & source : source_for_bucket) + source = -1; + } + }; + + using SharedDataPtr = std::shared_ptr; + ConvertingAggregatedToChunksSource( AggregatingTransformParamsPtr params_, ManyAggregatedDataVariantsPtr data_, - Arena * arena_, - std::shared_ptr> next_bucket_to_merge_) + SharedDataPtr shared_data_, + Int32 source_number_, + Arena * arena_) : ISource(params_->getHeader()) , params(std::move(params_)) , data(std::move(data_)) - , next_bucket_to_merge(std::move(next_bucket_to_merge_)) + , shared_data(std::move(shared_data_)) + , source_number(source_number_) , arena(arena_) {} @@ -89,23 +107,25 @@ public: protected: Chunk generate() override { - UInt32 bucket_num = next_bucket_to_merge->fetch_add(1); + UInt32 bucket_num = shared_data->next_bucket_to_merge.fetch_add(1); if (bucket_num >= NUM_BUCKETS) return {}; Block block = params->aggregator.mergeAndConvertOneBucketToBlock(*data, arena, params->final, bucket_num); + Chunk chunk = convertToChunk(block); - return convertToChunk(block); + shared_data->source_for_bucket[bucket_num] = source_number; + + return chunk; } private: AggregatingTransformParamsPtr params; ManyAggregatedDataVariantsPtr data; - std::shared_ptr> next_bucket_to_merge; + SharedDataPtr shared_data; + Int32 source_number; Arena * arena; - - static constexpr UInt32 NUM_BUCKETS = 256; }; /// Generates chunks with aggregated data. @@ -200,7 +220,7 @@ public: return Status::Ready; /// Two-level case. - return preparePullFromInputs(); + return prepareTwoLevel(); } private: @@ -220,38 +240,37 @@ private: } /// Read all sources and try to push current bucket. - IProcessor::Status preparePullFromInputs() + IProcessor::Status prepareTwoLevel() { - bool all_inputs_are_finished = true; + auto & output = outputs.front(); - for (auto & input : inputs) + Int32 next_input_num = shared_data->source_for_bucket[current_bucket_num]; + if (next_input_num < 0) + return Status::NeedData; + + auto next_input = std::next(inputs.begin(), next_input_num); + /// next_input can't be finished till data was not pulled. + if (!next_input->hasData()) + return Status::NeedData; + + output.push(next_input->pull()); + + ++current_bucket_num; + if (current_bucket_num == NUM_BUCKETS) { - if (input.isFinished()) - continue; - - all_inputs_are_finished = false; - - input.setNeeded(); - - if (input.hasData()) - ready_chunks.emplace_back(input.pull()); + output.finish(); + /// Do not close inputs, they must be finished. + return Status::Finished; } - moveReadyChunksToMap(); - - if (trySetCurrentChunkFromCurrentBucket()) - return preparePushToOutput(); - - if (all_inputs_are_finished) - throw Exception("All sources have finished before getting enough data in " - "ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR); - - return Status::NeedData; + return Status::PortFull; } private: AggregatingTransformParamsPtr params; ManyAggregatedDataVariantsPtr data; + ConvertingAggregatedToChunksSource::SharedDataPtr shared_data; + size_t num_threads; bool is_initialized = false; @@ -259,49 +278,12 @@ private: bool finished = false; Chunk current_chunk; - Chunks ready_chunks; UInt32 current_bucket_num = 0; static constexpr Int32 NUM_BUCKETS = 256; - std::map bucket_to_chunk; Processors processors; - static Int32 getBucketFromChunk(const Chunk & chunk) - { - auto & info = chunk.getChunkInfo(); - if (!info) - throw Exception("Chunk info was not set for chunk in " - "ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR); - - auto * agg_info = typeid_cast(info.get()); - if (!agg_info) - throw Exception("Chunk should have AggregatedChunkInfo in " - "ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR); - - return agg_info->bucket_num; - } - - void moveReadyChunksToMap() - { - for (auto & chunk : ready_chunks) - { - auto bucket = getBucketFromChunk(chunk); - - if (bucket < 0 || bucket >= NUM_BUCKETS) - throw Exception("Invalid bucket number " + toString(bucket) + " in " - "ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR); - - if (bucket_to_chunk.count(bucket)) - throw Exception("Found several chunks with the same bucket number in " - "ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR); - - bucket_to_chunk[bucket] = std::move(chunk); - } - - ready_chunks.clear(); - } - void setCurrentChunk(Chunk chunk) { if (has_input) @@ -366,34 +348,17 @@ private: void createSources() { AggregatedDataVariantsPtr & first = data->at(0); - auto next_bucket_to_merge = std::make_shared>(0); + shared_data = std::make_shared(); for (size_t thread = 0; thread < num_threads; ++thread) { Arena * arena = first->aggregates_pools.at(thread).get(); auto source = std::make_shared( - params, data, arena, next_bucket_to_merge); + params, data, shared_data, thread, arena); processors.emplace_back(std::move(source)); } } - - bool trySetCurrentChunkFromCurrentBucket() - { - auto it = bucket_to_chunk.find(current_bucket_num); - if (it != bucket_to_chunk.end()) - { - setCurrentChunk(std::move(it->second)); - ++current_bucket_num; - - if (current_bucket_num == NUM_BUCKETS) - finished = true; - - return true; - } - - return false; - } }; AggregatingTransform::AggregatingTransform(Block header, AggregatingTransformParamsPtr params_) From 65901f03b88a5beb0baf8be8e3cf3c3ffef908e0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Jan 2020 19:34:51 +0300 Subject: [PATCH 43/85] Update ConvertingAggregatedToChunksTransform transform. --- dbms/src/Processors/Transforms/AggregatingTransform.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Processors/Transforms/AggregatingTransform.cpp b/dbms/src/Processors/Transforms/AggregatingTransform.cpp index 1b562510bd7..7a5bf77da68 100644 --- a/dbms/src/Processors/Transforms/AggregatingTransform.cpp +++ b/dbms/src/Processors/Transforms/AggregatingTransform.cpp @@ -179,6 +179,7 @@ public: auto & out = source->getOutputs().front(); inputs.emplace_back(out.getHeader(), this); connect(out, inputs.back()); + inputs.back().setNeeded(); } return std::move(processors); From b5035e8924b071d3cb11ef4efb519f30b4ef081f Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 7 Jan 2020 11:31:03 +0800 Subject: [PATCH 44/85] no skip lambda function --- dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp | 3 --- dbms/src/Interpreters/PredicateRewriteVisitor.cpp | 7 ++++--- .../0_stateless/01056_predicate_optimizer_bugs.reference | 2 ++ .../queries/0_stateless/01056_predicate_optimizer_bugs.sql | 2 ++ 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp index a5bcea73aac..1240b6a09d6 100644 --- a/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp +++ b/dbms/src/Interpreters/ExtractExpressionInfoVisitor.cpp @@ -58,9 +58,6 @@ void ExpressionInfoMatcher::visit(const ASTIdentifier & identifier, const ASTPtr bool ExpressionInfoMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) { - if (const auto & function = node->as(); function && function->name == "lambda") - return false; - return !node->as(); } diff --git a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp index ec2088254c6..6bd16ddc066 100644 --- a/dbms/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/dbms/src/Interpreters/PredicateRewriteVisitor.cpp @@ -68,9 +68,8 @@ void PredicateRewriteVisitorData::visitOtherInternalSelect(ASTSelectQuery & sele static void cleanAliasAndCollectIdentifiers(ASTPtr & predicate, std::vector & identifiers) { - /// skip WHERE x in (SELECT ...) AND arrayMap(x -> x, [column_a]) - const auto & function = predicate->as(); - if (!predicate->as() && !(function && function->name == "lambda")) + /// Skip WHERE x in (SELECT ...) + if (!predicate->as()) { for (auto & children : predicate->children) cleanAliasAndCollectIdentifiers(children, identifiers); @@ -102,6 +101,8 @@ bool PredicateRewriteVisitorData::rewriteSubquery(ASTSelectQuery & subquery, con const auto & column_name = identifiers[index]->shortName(); const auto & outer_column_iterator = std::find(outer_columns.begin(), outer_columns.end(), column_name); + /// For lambda functions, we can't always find them in the list of columns + /// For example: SELECT * FROM system.one WHERE arrayMap(x -> x, [dummy]) = [0] if (outer_column_iterator != outer_columns.end()) identifiers[index]->setShortName(inner_columns[outer_column_iterator - outer_columns.begin()]); } diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 34cab3eb9b8..50ee6cba252 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -26,3 +26,5 @@ SELECT \n ts AS `--a.ts`, \n id AS `--a.id`, \n id_b AS `--a.id_b`, \n 4 5 SELECT dummy\nFROM \n(\n SELECT dummy\n FROM system.one\n WHERE arrayMap(x -> (x + 1), [dummy]) = [1]\n)\nWHERE arrayMap(x -> (x + 1), [dummy]) = [1] 0 +SELECT \n id, \n value, \n value_1\nFROM \n(\n SELECT \n 1 AS id, \n 2 AS value\n)\nALL INNER JOIN \n(\n SELECT \n 1 AS id, \n 3 AS value_1\n) USING (id)\nWHERE arrayMap(x -> ((x + value) + value_1), [1]) = [6] +1 2 3 diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index b0da6a2d097..549402243e7 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -74,3 +74,5 @@ DROP TABLE IF EXISTS test; ANALYZE SELECT * FROM (SELECT * FROM system.one) WHERE arrayMap(x -> x + 1, [dummy]) = [1]; SELECT * FROM (SELECT * FROM system.one) WHERE arrayMap(x -> x + 1, [dummy]) = [1]; +ANALYZE SELECT * FROM (SELECT 1 AS id, 2 AS value) INNER JOIN (SELECT 1 AS id, 3 AS value_1) USING id WHERE arrayMap(x -> x + value + value_1, [1]) = [6]; +SELECT * FROM (SELECT 1 AS id, 2 AS value) INNER JOIN (SELECT 1 AS id, 3 AS value_1) USING id WHERE arrayMap(x -> x + value + value_1, [1]) = [6]; From aa7c07c94ca66d9da61f266fc034daab6915a550 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 Jan 2020 08:41:18 +0300 Subject: [PATCH 45/85] Changes from #8545 to processors. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index b72b1cf3abd..973cbd98f8b 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -2342,9 +2342,6 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting return std::make_shared(header, output_order_descr, limit, do_count_rows); }); - /// If there are several streams, we merge them into one - pipeline.resize(1); - /// Merge the sorted blocks. pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr { @@ -2356,6 +2353,9 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting settings.max_bytes_before_remerge_sort, settings.max_bytes_before_external_sort, context->getTemporaryPath(), settings.min_free_disk_space_for_temporary_data); }); + + /// If there are several streams, we merge them into one + executeMergeSorted(pipeline, output_order_descr, limit); } From e1a3c7fb796bb7b2e6adb8ccd251fe4f9e8b7e82 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 Jan 2020 11:37:34 +0300 Subject: [PATCH 46/85] Changes from #8545 to processors. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 973cbd98f8b..d39e13ce4e7 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -2405,6 +2405,8 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline) void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline, const SortDescription & sort_description, UInt64 limit) { + pipeline.resize(pipeline.getNumMainStreams()); + /// If there are several streams, then we merge them into one if (pipeline.getNumStreams() > 1) { From 776ceeb427ba461dcac4b19819df9959d5454b78 Mon Sep 17 00:00:00 2001 From: Javi santana bot Date: Tue, 7 Jan 2020 19:16:37 +0100 Subject: [PATCH 47/85] Wait only on optimize commands --- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 8 ++++---- dbms/src/Storages/StorageReplicatedMergeTree.h | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 88c731fb88f..e0fe6d5708d 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3141,7 +3141,7 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p { /// NOTE Table lock must not be held while waiting. Some combination of R-W-R locks from different threads will yield to deadlock. for (auto & merge_entry : merge_entries) - waitForAllReplicasToProcessLogEntry(merge_entry); + waitForAllReplicasToProcessLogEntry(merge_entry, false); } return true; @@ -3840,15 +3840,15 @@ StorageReplicatedMergeTree::allocateBlockNumber( } -void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry) +void StorageReplicatedMergeTree::waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active) { LOG_DEBUG(log, "Waiting for all replicas to process " << entry.znode_name); - auto zookeeper = getZooKeeper() + auto zookeeper = getZooKeeper(); Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); for (const String & replica : replicas) { - if (zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) + if (wait_for_non_active || zookeeper->exists(zookeeper_path + "/replicas/" + replica + "/is_active")) { waitForReplicaToProcessLogEntry(replica, entry); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 3727d5de0d8..c779b9a87d3 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -486,7 +486,7 @@ private: * Because it effectively waits for other thread that usually has to also acquire a lock to proceed and this yields deadlock. * TODO: There are wrong usages of this method that are not fixed yet. */ - void waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry); + void waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry, bool wait_for_non_active = true); /** Wait until the specified replica executes the specified action from the log. * NOTE: See comment about locks above. From 573b6d23a884859c661de9b91dc2283e52956191 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Tue, 7 Jan 2020 22:40:48 +0100 Subject: [PATCH 48/85] make_changelog.py::process_unknown_commits unicode fix --- utils/make_changelog.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/make_changelog.py b/utils/make_changelog.py index 50959eb09b7..90d12844c33 100755 --- a/utils/make_changelog.py +++ b/utils/make_changelog.py @@ -207,7 +207,7 @@ def get_users_info(pull_requests, commits_info, token, max_retries, retry_timeou # List of unknown commits -> text description. def process_unknown_commits(commits, commits_info, users): - pattern = 'Commit: [{}]({})\nAuthor: {}\nMessage: {}' + pattern = u'Commit: [{}]({})\nAuthor: {}\nMessage: {}' texts = [] From 04f28e3aab7626dbd0012eb25116afb87b149af7 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 8 Jan 2020 11:22:26 +0800 Subject: [PATCH 49/85] remove union failure test --- .../0_stateless/01056_predicate_optimizer_bugs.reference | 2 -- .../queries/0_stateless/01056_predicate_optimizer_bugs.sql | 3 --- 2 files changed, 5 deletions(-) diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 50ee6cba252..019e95cb359 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -1,5 +1,3 @@ -SELECT count()\nFROM \n(\n SELECT \n a, \n runningDifference(a) AS rd\n FROM \n (\n SELECT a\n FROM \n (\n SELECT 1 AS a\n UNION ALL\n SELECT 2 AS a\n )\n ORDER BY a DESC\n )\n WHERE rd = -1\n) -1 SELECT \n k, \n v, \n d, \n i\nFROM \n(\n SELECT \n t.1 AS k, \n t.2 AS v, \n runningDifference(v) AS d, \n runningDifference(cityHash64(t.1)) AS i\n FROM \n (\n SELECT arrayJoin([(\'a\', 1), (\'a\', 2), (\'a\', 3), (\'b\', 11), (\'b\', 13), (\'b\', 15)]) AS t\n )\n)\nWHERE i = 0 a 1 0 0 a 2 1 0 diff --git a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index 549402243e7..e1e185be076 100644 --- a/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/dbms/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -3,9 +3,6 @@ SET enable_optimize_predicate_expression = 1; -- https://github.com/ClickHouse/ClickHouse/issues/3885 -- https://github.com/ClickHouse/ClickHouse/issues/5485 -ANALYZE SELECT count() FROM (SELECT a, runningDifference(a) rd FROM ( SELECT * FROM ( SELECT 1 a UNION ALL SELECT 2 a ) ORDER BY a DESC ) WHERE rd=-1 ); -SELECT count() FROM (SELECT a, runningDifference(a) rd FROM ( SELECT * FROM ( SELECT 1 a UNION ALL SELECT 2 a ) ORDER BY a DESC ) WHERE rd=-1 ); - ANALYZE SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; SELECT k, v, d, i FROM (SELECT t.1 AS k, t.2 AS v, runningDifference(v) AS d, runningDifference(cityHash64(t.1)) AS i FROM ( SELECT arrayJoin([('a', 1), ('a', 2), ('a', 3), ('b', 11), ('b', 13), ('b', 15)]) AS t)) WHERE i = 0; From 2084d262389570f746fdd48eca89b63b8da83b6a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jan 2020 13:20:55 +0300 Subject: [PATCH 50/85] Removed useless code, added comments --- dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index da3f1df8130..8041ad4dbe7 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -636,15 +636,16 @@ bool SplitTokenExtractor::next(const char * data, size_t len, size_t * pos, size { if (isASCII(data[*pos]) && !isAlphaNumericASCII(data[*pos])) { + /// Finish current token if any if (*token_len > 0) return true; *token_start = ++*pos; } else { - const size_t sz = UTF8::seqLength(static_cast(data[*pos])); - *pos += sz; - *token_len += sz; + /// Note that UTF-8 sequence is completely consisted of non-ASCII bytes. + ++*pos; + ++*token_len; } } return *token_len > 0; From da2783f21c021fbe747b9cfa8a277146ed7c7991 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Wed, 8 Jan 2020 17:35:28 +0300 Subject: [PATCH 51/85] Increased number of rows to make tests results noticeable. --- dbms/tests/performance/codecs_float_insert.xml | 12 +++++++++--- dbms/tests/performance/codecs_float_select.xml | 14 ++++++++++---- dbms/tests/performance/codecs_int_insert.xml | 12 +++++++++--- dbms/tests/performance/codecs_int_select.xml | 14 ++++++++++---- 4 files changed, 38 insertions(+), 14 deletions(-) diff --git a/dbms/tests/performance/codecs_float_insert.xml b/dbms/tests/performance/codecs_float_insert.xml index 2a39dfc48d6..0979ff79cce 100644 --- a/dbms/tests/performance/codecs_float_insert.xml +++ b/dbms/tests/performance/codecs_float_insert.xml @@ -37,14 +37,20 @@ rnd + + num_rows + + 10000000 + + CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple(); - INSERT INTO codec_seq_Float64_{codec} (n) SELECT number/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_mon_Float64_{codec} (n) SELECT number+sin(number) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_rnd_Float64_{codec} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 + INSERT INTO codec_seq_Float64_{codec} (n) SELECT number/pi() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_mon_Float64_{codec} (n) SELECT number+sin(number) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_rnd_Float64_{codec} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 DROP TABLE IF EXISTS codec_{seq_type}_{type}_{codec} diff --git a/dbms/tests/performance/codecs_float_select.xml b/dbms/tests/performance/codecs_float_select.xml index f23b363b914..db023f9f2e5 100644 --- a/dbms/tests/performance/codecs_float_select.xml +++ b/dbms/tests/performance/codecs_float_select.xml @@ -37,18 +37,24 @@ rnd + + num_rows + + 10000000 + + CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple(); - INSERT INTO codec_seq_Float64_{codec} (n) SELECT number/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_mon_Float64_{codec} (n) SELECT number+sin(number) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_rnd_Float64_{codec} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 + INSERT INTO codec_seq_Float64_{codec} (n) SELECT number/pi() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_mon_Float64_{codec} (n) SELECT number+sin(number) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_rnd_Float64_{codec} (n) SELECT (rand() - 4294967295)/pi() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 - SELECT count(n) FROM codec_{seq_type}_{type}_{codec} WHERE ignore(n) LIMIT 100000 SETTINGS max_threads=1 + SELECT count(n) FROM codec_{seq_type}_{type}_{codec} WHERE ignore(n) LIMIT {num_rows} SETTINGS max_threads=1 DROP TABLE IF EXISTS codec_{seq_type}_{type}_{codec} diff --git a/dbms/tests/performance/codecs_int_insert.xml b/dbms/tests/performance/codecs_int_insert.xml index 742693d49fe..1d3037cecb7 100644 --- a/dbms/tests/performance/codecs_int_insert.xml +++ b/dbms/tests/performance/codecs_int_insert.xml @@ -39,14 +39,20 @@ rnd + + num_rows + + 10000000 + + CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple(); - INSERT INTO codec_seq_UInt64_{codec} (n) SELECT number FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_mon_UInt64_{codec} (n) SELECT number*512+(rand()%512) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_rnd_UInt64_{codec} (n) SELECT rand() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 + INSERT INTO codec_seq_UInt64_{codec} (n) SELECT number FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_mon_UInt64_{codec} (n) SELECT number*512+(rand()%512) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_rnd_UInt64_{codec} (n) SELECT rand() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 DROP TABLE IF EXISTS codec_{seq_type}_{type}_{codec} diff --git a/dbms/tests/performance/codecs_int_select.xml b/dbms/tests/performance/codecs_int_select.xml index 9c007863cd8..b3fc0834c2d 100644 --- a/dbms/tests/performance/codecs_int_select.xml +++ b/dbms/tests/performance/codecs_int_select.xml @@ -39,18 +39,24 @@ rnd + + num_rows + + 10000000 + + CREATE TABLE IF NOT EXISTS codec_{seq_type}_{type}_{codec} (n {type} CODEC({codec})) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple(); - INSERT INTO codec_seq_UInt64_{codec} (n) SELECT number FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_mon_UInt64_{codec} (n) SELECT number*512+(rand()%512) FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 - INSERT INTO codec_rnd_UInt64_{codec} (n) SELECT rand() FROM system.numbers LIMIT 100000 SETTINGS max_threads=1 + INSERT INTO codec_seq_UInt64_{codec} (n) SELECT number FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_mon_UInt64_{codec} (n) SELECT number*512+(rand()%512) FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 + INSERT INTO codec_rnd_UInt64_{codec} (n) SELECT rand() FROM system.numbers LIMIT {num_rows} SETTINGS max_threads=1 - SELECT count(n) FROM codec_{seq_type}_{type}_{codec} WHERE ignore(n) LIMIT 100000 SETTINGS max_threads=1 + SELECT count(n) FROM codec_{seq_type}_{type}_{codec} WHERE ignore(n) LIMIT {num_rows} SETTINGS max_threads=1 DROP TABLE IF EXISTS codec_{seq_type}_{type}_{codec} From 5a723fa76958e8ae4b6f1934e5172df1867bbe2f Mon Sep 17 00:00:00 2001 From: Dmitrii Raev Date: Wed, 8 Jan 2020 16:56:40 +0200 Subject: [PATCH 52/85] Update apply_catboost_model.md Fix CreateTable SQL. Storage MergeTree requires 3 to 4 parameters --- docs/ru/guides/apply_catboost_model.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/guides/apply_catboost_model.md b/docs/ru/guides/apply_catboost_model.md index 9f93aacbd22..65cb3f58508 100644 --- a/docs/ru/guides/apply_catboost_model.md +++ b/docs/ru/guides/apply_catboost_model.md @@ -74,7 +74,7 @@ $ clickhouse client ROLE_FAMILY UInt32, ROLE_CODE UInt32 ) -ENGINE = MergeTree() +ENGINE = MergeTree(date, date, 8192) ``` **3.** Выйдите из клиента ClickHouse: @@ -227,4 +227,4 @@ FROM ``` !!! note "Примечание" - Подробнее про функции [avg()](../query_language/agg_functions/reference.md#agg_function-avg), [log()](../query_language/functions/math_functions.md). \ No newline at end of file + Подробнее про функции [avg()](../query_language/agg_functions/reference.md#agg_function-avg), [log()](../query_language/functions/math_functions.md). From 746dd1a8b72dbaf0cde40d88434933c1513ec2a0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 8 Jan 2020 22:03:10 +0700 Subject: [PATCH 53/85] PolicyInfo::setPolicy() now doesn't throw an excepion on empty filter. --- dbms/src/Access/RowPolicyContextFactory.cpp | 37 +++++++++++---------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/dbms/src/Access/RowPolicyContextFactory.cpp b/dbms/src/Access/RowPolicyContextFactory.cpp index e458f06ca94..77e5056e206 100644 --- a/dbms/src/Access/RowPolicyContextFactory.cpp +++ b/dbms/src/Access/RowPolicyContextFactory.cpp @@ -101,9 +101,6 @@ namespace public: void add(const ASTPtr & condition, bool is_restrictive) { - if (!condition) - return; - if (is_restrictive) restrictions.push_back(condition); else @@ -139,29 +136,32 @@ void RowPolicyContextFactory::PolicyInfo::setPolicy(const RowPolicyPtr & policy_ for (auto index : ext::range_with_static_cast(0, MAX_CONDITION_INDEX)) { + parsed_conditions[index] = nullptr; const String & condition = policy->conditions[index]; + if (condition.empty()) + continue; + auto previous_range = std::pair(std::begin(policy->conditions), std::begin(policy->conditions) + index); auto previous_it = std::find(previous_range.first, previous_range.second, condition); if (previous_it != previous_range.second) { /// The condition is already parsed before. parsed_conditions[index] = parsed_conditions[previous_it - previous_range.first]; + continue; } - else + + /// Try to parse the condition. + try { - /// Try to parse the condition. - try - { - ParserExpression parser; - parsed_conditions[index] = parseQuery(parser, condition, 0); - } - catch (...) - { - tryLogCurrentException( - &Poco::Logger::get("RowPolicy"), - String("Could not parse the condition ") + RowPolicy::conditionIndexToString(index) + " of row policy " - + backQuote(policy->getFullName())); - } + ParserExpression parser; + parsed_conditions[index] = parseQuery(parser, condition, 0); + } + catch (...) + { + tryLogCurrentException( + &Poco::Logger::get("RowPolicy"), + String("Could not parse the condition ") + RowPolicy::conditionIndexToString(index) + " of row policy " + + backQuote(policy->getFullName())); } } } @@ -290,7 +290,8 @@ void RowPolicyContextFactory::mixConditionsForContext(RowPolicyContext & context auto & mixers = map_of_mixers[std::pair{policy.getDatabase(), policy.getTableName()}]; mixers.policy_ids.push_back(policy_id); for (auto index : ext::range(0, MAX_CONDITION_INDEX)) - mixers.mixers[index].add(info.parsed_conditions[index], policy.isRestrictive()); + if (info.parsed_conditions[index]) + mixers.mixers[index].add(info.parsed_conditions[index], policy.isRestrictive()); } } From 189be696308b0874450ed932ce680c8bf011c1ce Mon Sep 17 00:00:00 2001 From: Dmitrii Raev Date: Wed, 8 Jan 2020 17:06:06 +0200 Subject: [PATCH 54/85] Update apply_catboost_model.md --- docs/en/guides/apply_catboost_model.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/guides/apply_catboost_model.md b/docs/en/guides/apply_catboost_model.md index 4665809bfa0..8329e4ed550 100644 --- a/docs/en/guides/apply_catboost_model.md +++ b/docs/en/guides/apply_catboost_model.md @@ -74,7 +74,7 @@ $ clickhouse client ROLE_FAMILY UInt32, ROLE_CODE UInt32 ) -ENGINE = MergeTree() +ENGINE = MergeTree(data, date, 8192) ``` **3.** Exit from ClickHouse console client: @@ -227,4 +227,4 @@ FROM ``` !!! note "Note" - More info about [avg()](../query_language/agg_functions/reference.md#agg_function-avg) and [log()](../query_language/functions/math_functions.md) functions. \ No newline at end of file + More info about [avg()](../query_language/agg_functions/reference.md#agg_function-avg) and [log()](../query_language/functions/math_functions.md) functions. From 79e9a1e3cbeec6b150f8c5c557aab4ba480fe89a Mon Sep 17 00:00:00 2001 From: Dmitrii Raev Date: Wed, 8 Jan 2020 17:07:01 +0200 Subject: [PATCH 55/85] Update apply_catboost_model.md Fix --- docs/en/guides/apply_catboost_model.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/guides/apply_catboost_model.md b/docs/en/guides/apply_catboost_model.md index 8329e4ed550..5e868bffe0d 100644 --- a/docs/en/guides/apply_catboost_model.md +++ b/docs/en/guides/apply_catboost_model.md @@ -74,7 +74,7 @@ $ clickhouse client ROLE_FAMILY UInt32, ROLE_CODE UInt32 ) -ENGINE = MergeTree(data, date, 8192) +ENGINE = MergeTree(date, date, 8192) ``` **3.** Exit from ClickHouse console client: From f56653f43ee40e626bb3b8e913735e60edf03d15 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Wed, 8 Jan 2020 14:13:14 +0800 Subject: [PATCH 56/85] ensure network be up before staring clickhouse --- debian/clickhouse-server.init | 4 ++-- debian/clickhouse-server.service | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/debian/clickhouse-server.init b/debian/clickhouse-server.init index 32282756719..213da9affb9 100755 --- a/debian/clickhouse-server.init +++ b/debian/clickhouse-server.init @@ -3,8 +3,8 @@ # Provides: clickhouse-server # Default-Start: 2 3 4 5 # Default-Stop: 0 1 6 -# Required-Start: -# Required-Stop: +# Required-Start: $network +# Required-Stop: $network # Short-Description: Yandex clickhouse-server daemon ### END INIT INFO diff --git a/debian/clickhouse-server.service b/debian/clickhouse-server.service index 4543b304197..b9681f9279e 100644 --- a/debian/clickhouse-server.service +++ b/debian/clickhouse-server.service @@ -1,5 +1,7 @@ [Unit] Description=ClickHouse Server (analytic DBMS for big data) +Requires=network-online.target +After=network-online.target [Service] Type=simple From dc7acc1028c0e0caa8737ccd94d797ebfe2ee1d0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 11:06:46 +0300 Subject: [PATCH 57/85] Changes from #8545 to processors. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index d39e13ce4e7..4965c25af5b 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -2350,7 +2350,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting return std::make_shared( header, output_order_descr, settings.max_block_size, limit, - settings.max_bytes_before_remerge_sort, + settings.max_bytes_before_remerge_sort / pipeline.getNumStreams(), settings.max_bytes_before_external_sort, context->getTemporaryPath(), settings.min_free_disk_space_for_temporary_data); }); From ba188f6a46b0b212b83e84e198cfe3b80658c398 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 11:52:30 +0300 Subject: [PATCH 58/85] Disable processors by default. --- dbms/src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index fb0e8cb96b2..c24392a5bd0 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -359,7 +359,7 @@ struct Settings : public SettingsCollection M(SettingBool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.", 0) \ M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql' and 'odbc' table functions.", 0) \ \ - M(SettingBool, experimental_use_processors, true, "Use processors pipeline.", 0) \ + M(SettingBool, experimental_use_processors, false, "Use processors pipeline.", 0) \ \ M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \ M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.", 0) \ From 5ed2a93748277667d16a3cbc80064622dde151a4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 12:03:08 +0300 Subject: [PATCH 59/85] Allow multiple edges for processors pipeline. --- dbms/src/Processors/Executors/PipelineExecutor.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index bc0de1fb81d..70cd2e2405f 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -64,13 +64,6 @@ bool PipelineExecutor::addEdges(UInt64 node) throwUnknownProcessor(to_proc, cur, true); UInt64 proc_num = it->second; - - for (auto & edge : edges) - { - if (edge.to == proc_num) - throw Exception("Multiple edges are not allowed for the same processors.", ErrorCodes::LOGICAL_ERROR); - } - auto & edge = edges.emplace_back(proc_num, is_backward, input_port_number, output_port_number, update_list); from_port.setUpdateInfo(&edge.update_info); From e3f0ea526ae47eaff69f2d82af7cb112c76f41bf Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 9 Jan 2020 12:52:40 +0300 Subject: [PATCH 60/85] update security changelog --- docs/en/security_changelog.md | 23 +++++++++++++++++++++++ docs/ru/security_changelog.md | 22 +++++++++++++++++++++- 2 files changed, 44 insertions(+), 1 deletion(-) diff --git a/docs/en/security_changelog.md b/docs/en/security_changelog.md index 0847300cc19..92b35868f94 100644 --- a/docs/en/security_changelog.md +++ b/docs/en/security_changelog.md @@ -1,6 +1,27 @@ +## Fixed in ClickHouse Release 19.14.3.3, 2019-09-10 + +### CVE-2019-15024 + +Аn attacker having write access to ZooKeeper and who is able to run a custom server available from the network where ClickHouse runs, can create a custom-built malicious server that will act as a ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from the malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. + +Credits: Eldar Zaitov of Yandex Information Security Team + +### CVE-2019-16535 + +Аn OOB read, OOB write and integer underflow in decompression algorithms can be used to achieve RCE or DoS via native protocol. + +Credits: Eldar Zaitov of Yandex Information Security Team + +### CVE-2019-16536 + +Stack overflow leading to DoS can be triggered by malicious authenticated client. + +Credits: Eldar Zaitov of Yandex Information Security Team + ## Fixed in ClickHouse Release 19.13.6.1, 2019-09-20 ### CVE-2019-18657 + Table function `url` had the vulnerability allowed the attacker to inject arbitrary HTTP headers in the request. Credits: [Nikita Tikhomirov](https://github.com/NSTikhomirov) @@ -24,6 +45,7 @@ Credits: Andrey Krasichkov and Evgeny Sidorov of Yandex Information Security Tea ## Fixed in ClickHouse Release 1.1.54388, 2018-06-28 ### CVE-2018-14668 + "remote" table function allowed arbitrary symbols in "user", "password" and "default_database" fields which led to Cross Protocol Request Forgery Attacks. Credits: Andrey Krasichkov of Yandex Information Security Team @@ -31,6 +53,7 @@ Credits: Andrey Krasichkov of Yandex Information Security Team ## Fixed in ClickHouse Release 1.1.54390, 2018-07-06 ### CVE-2018-14669 + ClickHouse MySQL client had "LOAD DATA LOCAL INFILE" functionality enabled that allowed a malicious MySQL database read arbitrary files from the connected ClickHouse server. Credits: Andrey Krasichkov and Evgeny Sidorov of Yandex Information Security Team diff --git a/docs/ru/security_changelog.md b/docs/ru/security_changelog.md index 17ae1eba19d..4e588386af2 100644 --- a/docs/ru/security_changelog.md +++ b/docs/ru/security_changelog.md @@ -1,3 +1,23 @@ +## Исправлено в релизе 19.14.3.3, 2019-09-10 + +### CVE-2019-15024 + +Злоумышленник с доступом на запись к ZooKeeper и возможностью запустить собственный сервер в сети доступной ClickHouse может создать вредоносный сервер, который будет вести себя как реплика ClickHouse и зарегистрируется в ZooKeeper. В процессе репликации вредоносный сервер может указать любой путь на файловой системе в который будут записаны данные. + +Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса + +### CVE-2019-16535 + +Интерфейс декомпрессии позволял совершать OOB чтения и записи данных в памяти, а также переполнение целочисленных переменных, что могло приводить к отказу в обслуживании. Также потенциально могло использоваьтся для удаленного выполнения кода. + +Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса + +### CVE-2019-16536 + +Аутентифицированный клиент злоумышленника имел возможность вызвать переполнение стека, что могло привести к отказу в обслуживании. + +Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса + ## Исправлено в релизе 19.13.6.1 от 20 сентября 2019 ### CVE-2019-18657 @@ -19,7 +39,7 @@ unixODBC позволял указать путь для подключения Обнаружено благодаря: Андрею Красичкову и Евгению Сидорову из Службы Информационной Безопасности Яндекса -## Исправлено в релизе 1.1.54388 от 28 июня 2018 +## Исправлено в релизе 1.1.54388 от 28 июня 2018 ### CVE-2018-14668 Табличная функция "remote" допускала произвольные символы в полях "user", "password" и "default_database", что позволяло производить атаки класса Cross Protocol Request Forgery. From 4a9e2399a757f8206292bba4e4f644145d95ef8e Mon Sep 17 00:00:00 2001 From: Alexander Burmak Date: Thu, 9 Jan 2020 11:02:07 +0300 Subject: [PATCH 61/85] Updated contrib/googletest to v1.10.0 --- contrib/googletest | 2 +- .../tests/gtest_compressionCodec.cpp | 44 +++++++++---------- .../src/Core/tests/gtest_DecimalFunctions.cpp | 4 +- .../tests/gtest_data_type_get_common_type.cpp | 5 ++- .../gtest_DateTime64_parsing_and_writing.cpp | 6 +-- dbms/src/IO/tests/gtest_bit_io.cpp | 13 +++--- 6 files changed, 38 insertions(+), 36 deletions(-) diff --git a/contrib/googletest b/contrib/googletest index d175c8bf823..703bd9caab5 160000 --- a/contrib/googletest +++ b/contrib/googletest @@ -1 +1 @@ -Subproject commit d175c8bf823e709d570772b038757fadf63bc632 +Subproject commit 703bd9caab50b139428cea1aaff9974ebee5742e diff --git a/dbms/src/Compression/tests/gtest_compressionCodec.cpp b/dbms/src/Compression/tests/gtest_compressionCodec.cpp index 95bef3b691e..37807632eff 100644 --- a/dbms/src/Compression/tests/gtest_compressionCodec.cpp +++ b/dbms/src/Compression/tests/gtest_compressionCodec.cpp @@ -860,7 +860,7 @@ INSTANTIATE_TEST_CASE_P(Simple, ::testing::Values( makeSeq(1, 2, 3, 5, 7, 11, 13, 17, 23, 29, 31, 37, 41, 43, 47, 53, 59, 61, 67, 71, 73, 79, 83, 89, 97) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(SmallSequences, @@ -877,7 +877,7 @@ INSTANTIATE_TEST_CASE_P(SmallSequences, + generatePyramidOfSequences(42, G(SequentialGenerator(1))) + generatePyramidOfSequences(42, G(SequentialGenerator(1))) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(Mixed, @@ -894,7 +894,7 @@ INSTANTIATE_TEST_CASE_P(Mixed, generateSeq(G(MinMaxGenerator()), 1, 5) + generateSeq(G(SequentialGenerator(1)), 1, 1001), generateSeq(G(MinMaxGenerator()), 1, 5) + generateSeq(G(SequentialGenerator(1)), 1, 1001) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(SameValueInt, @@ -911,7 +911,7 @@ INSTANTIATE_TEST_CASE_P(SameValueInt, generateSeq(G(SameValueGenerator(1000))), generateSeq(G(SameValueGenerator(1000))) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(SameNegativeValueInt, @@ -928,7 +928,7 @@ INSTANTIATE_TEST_CASE_P(SameNegativeValueInt, generateSeq(G(SameValueGenerator(-1000))), generateSeq(G(SameValueGenerator(-1000))) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(SameValueFloat, @@ -942,7 +942,7 @@ INSTANTIATE_TEST_CASE_P(SameValueFloat, generateSeq(G(SameValueGenerator(M_E))), generateSeq(G(SameValueGenerator(M_E))) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(SameNegativeValueFloat, @@ -956,7 +956,7 @@ INSTANTIATE_TEST_CASE_P(SameNegativeValueFloat, generateSeq(G(SameValueGenerator(-1 * M_E))), generateSeq(G(SameValueGenerator(-1 * M_E))) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(SequentialInt, @@ -973,7 +973,7 @@ INSTANTIATE_TEST_CASE_P(SequentialInt, generateSeq(G(SequentialGenerator(1))), generateSeq(G(SequentialGenerator(1))) ) - ), + ) ); // -1, -2, -3, ... etc for signed @@ -992,7 +992,7 @@ INSTANTIATE_TEST_CASE_P(SequentialReverseInt, generateSeq(G(SequentialGenerator(-1))), generateSeq(G(SequentialGenerator(-1))) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(SequentialFloat, @@ -1006,7 +1006,7 @@ INSTANTIATE_TEST_CASE_P(SequentialFloat, generateSeq(G(SequentialGenerator(M_E))), generateSeq(G(SequentialGenerator(M_E))) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(SequentialReverseFloat, @@ -1020,7 +1020,7 @@ INSTANTIATE_TEST_CASE_P(SequentialReverseFloat, generateSeq(G(SequentialGenerator(-1 * M_E))), generateSeq(G(SequentialGenerator(-1 * M_E))) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(MonotonicInt, @@ -1037,7 +1037,7 @@ INSTANTIATE_TEST_CASE_P(MonotonicInt, generateSeq(G(MonotonicGenerator(1, 5))), generateSeq(G(MonotonicGenerator(1, 5))) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(MonotonicReverseInt, @@ -1054,7 +1054,7 @@ INSTANTIATE_TEST_CASE_P(MonotonicReverseInt, generateSeq(G(MonotonicGenerator(-1, 5))), generateSeq(G(MonotonicGenerator(-1, 5))) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(MonotonicFloat, @@ -1067,7 +1067,7 @@ INSTANTIATE_TEST_CASE_P(MonotonicFloat, generateSeq(G(MonotonicGenerator(M_E, 5))), generateSeq(G(MonotonicGenerator(M_E, 5))) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(MonotonicReverseFloat, @@ -1080,7 +1080,7 @@ INSTANTIATE_TEST_CASE_P(MonotonicReverseFloat, generateSeq(G(MonotonicGenerator(-1 * M_E, 5))), generateSeq(G(MonotonicGenerator(-1 * M_E, 5))) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(RandomInt, @@ -1093,7 +1093,7 @@ INSTANTIATE_TEST_CASE_P(RandomInt, generateSeq(G(RandomGenerator(0, 0, 1000'000'000))), generateSeq(G(RandomGenerator(0, 0, 1000'000'000))) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(RandomishInt, @@ -1108,7 +1108,7 @@ INSTANTIATE_TEST_CASE_P(RandomishInt, generateSeq(G(RandomishGenerator)), generateSeq(G(RandomishGenerator)) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(RandomishFloat, @@ -1119,7 +1119,7 @@ INSTANTIATE_TEST_CASE_P(RandomishFloat, generateSeq(G(RandomishGenerator)), generateSeq(G(RandomishGenerator)) ) - ), + ) ); // Double delta overflow case, deltas are out of bounds for target type @@ -1136,7 +1136,7 @@ INSTANTIATE_TEST_CASE_P(OverflowInt, generateSeq(G(MinMaxGenerator())), generateSeq(G(MinMaxGenerator())) ) - ), + ) ); INSTANTIATE_TEST_CASE_P(OverflowFloat, @@ -1152,7 +1152,7 @@ INSTANTIATE_TEST_CASE_P(OverflowFloat, generateSeq(G(FFand0Generator())), generateSeq(G(FFand0Generator())) ) - ), + ) ); template @@ -1227,7 +1227,7 @@ INSTANTIATE_TEST_CASE_P(DoubleDelta, BIN_STR("\x94\xd4\x00\x00\x00\x98\x01\x00\x00\x08\x00\x33\x00\x00\x00\x2a\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x6b\x65\x5f\x50\x34\xff\x4f\xaf\xbc\xe3\x5d\xa3\xd3\xd9\xf6\x1f\xe2\x07\x7c\x47\x20\x67\x48\x07\x47\xff\x47\xf6\xfe\xf8\x00\x00\x70\x6b\xd0\x00\x02\x83\xd9\xfb\x9f\xdc\x1f\xfc\x20\x1e\x80\x00\x22\xc8\xf0\x00\x00\x66\x67\xa0\x00\x02\x00\x3d\x00\x00\x0f\xff\xe8\x00\x00\x7f\xee\xff\xdf\x00\x00\x70\x0d\x7a\x00\x02\x80\x7b\x9f\xf7\x9f\xfb\xc0\x00\x00\xff\xfe\x00\x00\x08\x00\xfc\x00\x00\x00\x04\x00\x06\xbe\x4f\xbf\xff\xd6\x0c\xff\x00\x00\x00\x01\x00\x00\x00\x03\xf8\x00\x00\x00\x08\x00\x00\x00\x0f\xc0\x00\x00\x00\x3f\xff\xff\xff\xfb\xff\xff\xff\xfb\xe0\x00\x00\x01\xc0\x00\x00\x06\x9f\x80\x00\x00\x0a\x00\x00\x00\x34\xf3\xff\xff\xff\xe7\x9f\xff\xff\xff\x7e\x00\x00\x00\x00\xff\xff\xff\xfd\xf0\x00\x00\x00\x07\xff\xff\xff\xf0") }, }) - ), + ) ); template @@ -1301,7 +1301,7 @@ INSTANTIATE_TEST_CASE_P(Gorilla, BIN_STR("\x95\x91\x00\x00\x00\x50\x01\x00\x00\x08\x00\x2a\x00\x00\x00\x00\xc2\xeb\x0b\x00\x00\x00\x00\xe3\x2b\xa0\xa6\x19\x85\x98\xdc\x45\x74\x74\x43\xc2\x57\x41\x4c\x6e\x42\x79\xd9\x8f\x88\xa5\x05\xf3\xf1\x94\xa3\x62\x1e\x02\xdf\x05\x10\xf1\x15\x97\x35\x2a\x50\x71\x0f\x09\x6c\x89\xf7\x65\x1d\x11\xb7\xcc\x7d\x0b\x70\xc1\x86\x88\x48\x47\x87\xb6\x32\x26\xa7\x86\x87\x88\xd3\x93\x3d\xfc\x28\x68\x85\x05\x0b\x13\xc6\x5f\xd4\x70\xe1\x5e\x76\xf1\x9f\xf3\x33\x2a\x14\x14\x5e\x40\xc1\x5c\x28\x3f\xec\x43\x03\x05\x11\x91\xe8\xeb\x8e\x0a\x0e\x27\x21\x55\xcb\x39\xbc\x6a\xff\x11\x5d\x81\xa0\xa6\x10") }, }) - ), + ) ); // These 'tests' try to measure performance of encoding and decoding and hence only make sence to be run locally, diff --git a/dbms/src/Core/tests/gtest_DecimalFunctions.cpp b/dbms/src/Core/tests/gtest_DecimalFunctions.cpp index d03be3ff3b8..84bbb987819 100644 --- a/dbms/src/Core/tests/gtest_DecimalFunctions.cpp +++ b/dbms/src/Core/tests/gtest_DecimalFunctions.cpp @@ -168,5 +168,5 @@ INSTANTIATE_TEST_CASE_P(Basic, 89 } } - } -),); + }) +); diff --git a/dbms/src/DataTypes/tests/gtest_data_type_get_common_type.cpp b/dbms/src/DataTypes/tests/gtest_data_type_get_common_type.cpp index 2ae1c335387..f71dca14e43 100644 --- a/dbms/src/DataTypes/tests/gtest_data_type_get_common_type.cpp +++ b/dbms/src/DataTypes/tests/gtest_data_type_get_common_type.cpp @@ -159,7 +159,7 @@ INSTANTIATE_TEST_CASE_P(data_type, {"Tuple(Int64,Int8) Tuple(UInt64)", nullptr}, {"Array(Int64) Array(String)", nullptr}, } - ), + ) ); INSTANTIATE_TEST_CASE_P(data_type, @@ -210,5 +210,6 @@ INSTANTIATE_TEST_CASE_P(data_type, {"Int8 String", nullptr}, {"Nothing", nullptr}, {"FixedString(16) FixedString(8) String", nullptr}, - }), + } + ) ); diff --git a/dbms/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp b/dbms/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp index 08ca5dc88ee..c1a6068a17f 100644 --- a/dbms/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp +++ b/dbms/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp @@ -130,7 +130,7 @@ INSTANTIATE_TEST_CASE_P(Basic, 1568650817'1ULL, 1 } - }), + }) ); INSTANTIATE_TEST_CASE_P(BestEffort, @@ -142,7 +142,7 @@ INSTANTIATE_TEST_CASE_P(BestEffort, 1568650817'123456ULL, 6 } - }), + }) ); @@ -181,6 +181,6 @@ INSTANTIATE_TEST_CASE_P(Basic, 1568650817'001ULL, 3 } - }), + }) ); diff --git a/dbms/src/IO/tests/gtest_bit_io.cpp b/dbms/src/IO/tests/gtest_bit_io.cpp index 994e08214cc..b6b4cf998bd 100644 --- a/dbms/src/IO/tests/gtest_bit_io.cpp +++ b/dbms/src/IO/tests/gtest_bit_io.cpp @@ -221,7 +221,7 @@ INSTANTIATE_TEST_CASE_P(Simple, "10101001 10111010 11101111 10101111 10111010 11101011 10101001 00000000 " // 256 "10101111 10111010 11101011 10101001 00001111 11110000 00001110 11111111 " // 320 } - }), + }) ); TestCaseParameter primes_case(UInt8 repeat_times, UInt64 pattern) @@ -242,11 +242,12 @@ TestCaseParameter primes_case(UInt8 repeat_times, UInt64 pattern) } INSTANTIATE_TEST_CASE_P(Primes, - BitIO, - ::testing::Values( - primes_case(11, 0xFFFFFFFFFFFFFFFFULL), - primes_case(11, BIT_PATTERN) -),); + BitIO, + ::testing::Values( + primes_case(11, 0xFFFFFFFFFFFFFFFFULL), + primes_case(11, BIT_PATTERN) + ) +); TEST(BitHelpers, maskLowBits) { From c637396791f40c4a9d6d914534ff74eaf0910fd7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 13:37:27 +0300 Subject: [PATCH 62/85] Added DelayedPortsProcessor. --- dbms/src/Processors/DelayedPortsProcessor.cpp | 95 +++++++++++++++++++ dbms/src/Processors/DelayedPortsProcessor.h | 37 ++++++++ 2 files changed, 132 insertions(+) create mode 100644 dbms/src/Processors/DelayedPortsProcessor.cpp create mode 100644 dbms/src/Processors/DelayedPortsProcessor.h diff --git a/dbms/src/Processors/DelayedPortsProcessor.cpp b/dbms/src/Processors/DelayedPortsProcessor.cpp new file mode 100644 index 00000000000..d5a033dc7df --- /dev/null +++ b/dbms/src/Processors/DelayedPortsProcessor.cpp @@ -0,0 +1,95 @@ +#include + +namespace DB +{ + +DelayedPortsProcessor::DelayedPortsProcessor(const Block & header, size_t num_ports, const PortNumbers & delayed_ports) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)) + , num_delayed(delayed_ports.size()) +{ + port_pairs.resize(num_ports); + + auto input_it = inputs.begin(); + auto output_it = outputs.begin(); + for (size_t i = 0; i < num_ports; ++i) + { + port_pairs[i].input_port = &*input_it; + port_pairs[i].output_port = &*output_it; + ++input_it; + ++output_it; + } + + for (auto & delayed : delayed_ports) + port_pairs[delayed].is_delayed = true; +} + +bool DelayedPortsProcessor::processPair(PortsPair & pair) +{ + auto finish = [&]() + { + if (!pair.is_finished) + { + pair.input_port->close(); + + pair.is_finished = true; + ++num_finished; + } + }; + + if (pair.output_port->isFinished()) + { + finish(); + return false; + } + + if (pair.input_port->isFinished()) + { + finish(); + return false; + } + + if (!pair.output_port->canPush()) + return false; + + pair.input_port->setNeeded(); + if (pair.input_port->hasData()) + pair.output_port->pushData(pair.input_port->pullData()); + + return true; +} + +IProcessor::Status DelayedPortsProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +{ + bool skip_delayed = (num_finished + num_delayed) < port_pairs.size(); + bool need_data = false; + + for (auto & output_number : updated_outputs) + { + if (!skip_delayed || !port_pairs[output_number].is_delayed) + need_data = processPair(port_pairs[output_number]) || need_data; + } + + for (auto & input_number : updated_inputs) + { + if (!skip_delayed || !port_pairs[input_number].is_delayed) + need_data = processPair(port_pairs[input_number]) || need_data; + } + + /// In case if main streams are finished at current iteration, start processing delayed streams. + if (skip_delayed && (num_finished + num_delayed) >= port_pairs.size()) + { + for (auto & pair : port_pairs) + if (pair.is_delayed) + need_data = processPair(pair) || need_data; + } + + if (num_finished == port_pairs.size()) + return Status::Finished; + + if (need_data) + return Status::NeedData; + + return Status::PortFull; +} + +} diff --git a/dbms/src/Processors/DelayedPortsProcessor.h b/dbms/src/Processors/DelayedPortsProcessor.h new file mode 100644 index 00000000000..44dd632f8a8 --- /dev/null +++ b/dbms/src/Processors/DelayedPortsProcessor.h @@ -0,0 +1,37 @@ +#pragma once +#include + +namespace DB +{ + +/// Processor with N inputs and N outputs. Only moves data from i-th input to i-th output as is. +/// Some ports are delayed. Delayed ports are processed after other outputs are all finished. +/// Data between ports is not mixed. It is important because this processor can be used before MergingSortedTransform. +/// Delayed ports are appeared after joins, when some non-matched data need to be processed at the end. +class DelayedPortsProcessor : public IProcessor +{ +public: + DelayedPortsProcessor(const Block & header, size_t num_ports, const PortNumbers & delayed_ports); + + String getName() const override { return "DelayedPorts"; } + + Status prepare(const PortNumbers &, const PortNumbers &) override; + +private: + + struct PortsPair + { + InputPort * input_port = nullptr; + OutputPort * output_port = nullptr; + bool is_delayed = false; + bool is_finished = false; + }; + + std::vector port_pairs; + size_t num_delayed; + size_t num_finished = 0; + + bool processPair(PortsPair & pair); +}; + +} From 1d14d8b4a7264960bba7ec5ec275187a1706a83d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 9 Jan 2020 14:38:26 +0300 Subject: [PATCH 63/85] fix mutation size limit --- dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 5 ++++- dbms/src/Storages/StorageMergeTree.cpp | 5 +---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 8975535f31b..05db73ce215 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -169,9 +169,12 @@ UInt64 MergeTreeDataMergerMutator::getMaxSourcePartSizeForMutation() const auto data_settings = data.getSettings(); size_t busy_threads_in_pool = CurrentMetrics::values[CurrentMetrics::BackgroundPoolTask].load(std::memory_order_relaxed); + /// DataPart can be store only at one disk. Get Max of free space at all disks + UInt64 disk_space = data.storage_policy->getMaxUnreservedFreeSpace(); + /// Allow mutations only if there are enough threads, leave free threads for merges else if (background_pool_size - busy_threads_in_pool >= data_settings->number_of_free_entries_in_pool_to_execute_mutation) - return static_cast(data.storage_policy->getMaxUnreservedFreeSpace() / DISK_USAGE_COEFFICIENT_TO_RESERVE); + return static_cast(disk_space / DISK_USAGE_COEFFICIENT_TO_RESERVE); return 0; } diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 9a0583a464a..e2766f21494 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -697,9 +697,6 @@ bool StorageMergeTree::tryMutatePart() /// You must call destructor with unlocked `currently_processing_in_background_mutex`. std::optional tagger; { - /// DataPart can be store only at one disk. Get Max of free space at all disks - UInt64 disk_space = storage_policy->getMaxUnreservedFreeSpace(); - std::lock_guard lock(currently_processing_in_background_mutex); if (current_mutations_by_version.empty()) @@ -715,7 +712,7 @@ bool StorageMergeTree::tryMutatePart() if (mutations_begin_it == mutations_end_it) continue; - if (merger_mutator.getMaxSourcePartSizeForMutation() > disk_space) + if (merger_mutator.getMaxSourcePartSizeForMutation() < part->bytes_on_disk) continue; size_t current_ast_elements = 0; From 39b91584fd385ab62a185c54ecbb2260b5209ce5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 14:52:34 +0300 Subject: [PATCH 64/85] Simplifu delayed stream in QueryPipeline. --- .../Interpreters/InterpreterSelectQuery.cpp | 27 ++++++--------- dbms/src/Processors/QueryPipeline.cpp | 34 +++---------------- dbms/src/Processors/QueryPipeline.h | 9 ++--- 3 files changed, 17 insertions(+), 53 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index f5971d7edbf..b764eed3fc0 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1180,7 +1180,6 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (expressions.second_stage) { bool need_second_distinct_pass = false; - bool need_merge_streams = false; if (expressions.need_aggregate) { @@ -1241,13 +1240,11 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS executePreLimit(pipeline); } - if (need_second_distinct_pass - || query.limitLength() - || query.limitBy() - || pipeline.hasDelayedStream()) - { - need_merge_streams = true; - } + bool need_merge_streams = need_second_distinct_pass || query.limitLength() || query.limitBy(); + + if constexpr (!pipeline_with_processors) + if (pipeline.hasDelayedStream()) + need_merge_streams = true; if (need_merge_streams) { @@ -1933,7 +1930,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const * 1. Parallel aggregation is done, and the results should be merged in parallel. * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. */ - bool allow_to_use_two_level_group_by = pipeline.getNumMainStreams() > 1 || settings.max_bytes_before_external_group_by != 0; + bool allow_to_use_two_level_group_by = pipeline.getNumStreams() > 1 || settings.max_bytes_before_external_group_by != 0; Aggregator::Params params(header_before_aggregation, keys, aggregates, overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode, @@ -1947,12 +1944,12 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const pipeline.dropTotalsIfHas(); /// If there are several sources, then we perform parallel aggregation - if (pipeline.getNumMainStreams() > 1) + if (pipeline.getNumStreams() > 1) { /// Add resize transform to uniformly distribute data between aggregating streams. - pipeline.resize(pipeline.getNumMainStreams(), true); + pipeline.resize(pipeline.getNumStreams(), true); - auto many_data = std::make_shared(pipeline.getNumMainStreams()); + auto many_data = std::make_shared(pipeline.getNumStreams()); auto merge_threads = settings.aggregation_memory_efficient_merge_threads ? static_cast(settings.aggregation_memory_efficient_merge_threads) : static_cast(settings.max_threads); @@ -2807,11 +2804,7 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, SubqueriesForSets & subqueries_for_sets) { if (query_info.input_sorting_info) - { - if (pipeline.hasDelayedStream()) - throw Exception("Using read in order optimization, but has delayed stream in pipeline", ErrorCodes::LOGICAL_ERROR); executeMergeSorted(pipeline, query_info.input_sorting_info->order_key_prefix_descr, 0); - } const Settings & settings = context->getSettingsRef(); @@ -2828,7 +2821,7 @@ void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline, Block header) { /// Unify streams in case they have different headers. - /// TODO: remove previos addition of _dummy column. + /// TODO: remove previous addition of _dummy column. if (header.columns() > 1 && header.has("_dummy")) header.erase("_dummy"); diff --git a/dbms/src/Processors/QueryPipeline.cpp b/dbms/src/Processors/QueryPipeline.cpp index 13e91ac718d..25abeb6c6d3 100644 --- a/dbms/src/Processors/QueryPipeline.cpp +++ b/dbms/src/Processors/QueryPipeline.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB { @@ -165,7 +166,6 @@ void QueryPipeline::addSimpleTransformImpl(const TProcessorGetter & getter) for (size_t stream_num = 0; stream_num < streams.size(); ++stream_num) add_transform(streams[stream_num], StreamType::Main, stream_num); - add_transform(delayed_stream_port, StreamType::Main); add_transform(totals_having_port, StreamType::Totals); add_transform(extremes_port, StreamType::Extremes); @@ -185,7 +185,6 @@ void QueryPipeline::addSimpleTransform(const ProcessorGetterWithStreamKind & get void QueryPipeline::addPipe(Processors pipe) { checkInitialized(); - concatDelayedStream(); if (pipe.empty()) throw Exception("Can't add empty processors list to QueryPipeline.", ErrorCodes::LOGICAL_ERROR); @@ -224,41 +223,20 @@ void QueryPipeline::addDelayedStream(ProcessorPtr source) { checkInitialized(); - if (delayed_stream_port) - throw Exception("QueryPipeline already has stream with non joined data.", ErrorCodes::LOGICAL_ERROR); - checkSource(source, false); assertBlocksHaveEqualStructure(current_header, source->getOutputs().front().getHeader(), "QueryPipeline"); - delayed_stream_port = &source->getOutputs().front(); + IProcessor::PortNumbers delayed_streams = { streams.size() }; + streams.emplace_back(&source->getOutputs().front()); processors.emplace_back(std::move(source)); -} -void QueryPipeline::concatDelayedStream() -{ - if (!delayed_stream_port) - return; - - auto resize = std::make_shared(current_header, getNumMainStreams(), 1); - auto stream = streams.begin(); - for (auto & input : resize->getInputs()) - connect(**(stream++), input); - - auto concat = std::make_shared(current_header, 2); - connect(resize->getOutputs().front(), concat->getInputs().front()); - connect(*delayed_stream_port, concat->getInputs().back()); - - streams = { &concat->getOutputs().front() }; - processors.emplace_back(std::move(resize)); - processors.emplace_back(std::move(concat)); - - delayed_stream_port = nullptr; + auto processor = std::make_shared(current_header, streams.size(), delayed_streams); + addPipe({ std::move(processor) }); } void QueryPipeline::resize(size_t num_streams, bool force) { checkInitialized(); - concatDelayedStream(); if (!force && num_streams == getNumStreams()) return; @@ -443,7 +421,6 @@ void QueryPipeline::unitePipelines( std::vector && pipelines, const Block & common_header, const Context & context) { checkInitialized(); - concatDelayedStream(); addSimpleTransform([&](const Block & header) { @@ -456,7 +433,6 @@ void QueryPipeline::unitePipelines( for (auto & pipeline : pipelines) { pipeline.checkInitialized(); - pipeline.concatDelayedStream(); pipeline.addSimpleTransform([&](const Block & header) { diff --git a/dbms/src/Processors/QueryPipeline.h b/dbms/src/Processors/QueryPipeline.h index c27e570018f..29ebaf22955 100644 --- a/dbms/src/Processors/QueryPipeline.h +++ b/dbms/src/Processors/QueryPipeline.h @@ -57,7 +57,7 @@ public: /// Will read from this stream after all data was read from other streams. void addDelayedStream(ProcessorPtr source); - bool hasDelayedStream() const { return delayed_stream_port; } + /// Check if resize transform was used. (In that case another distinct transform will be added). bool hasMixedStreams() const { return has_resize || hasMoreThanOneStream(); } @@ -69,8 +69,7 @@ public: PipelineExecutorPtr execute(); - size_t getNumStreams() const { return streams.size() + (hasDelayedStream() ? 1 : 0); } - size_t getNumMainStreams() const { return streams.size(); } + size_t getNumStreams() const { return streams.size(); } bool hasMoreThanOneStream() const { return getNumStreams() > 1; } bool hasTotals() const { return totals_having_port != nullptr; } @@ -103,9 +102,6 @@ private: OutputPort * totals_having_port = nullptr; OutputPort * extremes_port = nullptr; - /// Special port for delayed stream. - OutputPort * delayed_stream_port = nullptr; - /// If resize processor was added to pipeline. bool has_resize = false; @@ -126,7 +122,6 @@ private: void checkInitialized(); void checkSource(const ProcessorPtr & source, bool can_have_totals); - void concatDelayedStream(); template void addSimpleTransformImpl(const TProcessorGetter & getter); From ea6fa1db8f45e12c6e9c375364fc9708c2fc4cf3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 14:59:53 +0300 Subject: [PATCH 65/85] Fix string size for GatherUtils StringSource. --- dbms/src/Functions/GatherUtils/Sources.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/GatherUtils/Sources.h b/dbms/src/Functions/GatherUtils/Sources.h index c21a6fc523c..7c881bba0c5 100644 --- a/dbms/src/Functions/GatherUtils/Sources.h +++ b/dbms/src/Functions/GatherUtils/Sources.h @@ -238,7 +238,7 @@ struct StringSource size_t getElementSize() const { - return offsets[row_num] - prev_offset; + return offsets[row_num] - prev_offset - 1; } Slice getWhole() const From 856f72f3f7511a9f003570dcdd10bd5e3b7675dd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 15:13:36 +0300 Subject: [PATCH 66/85] Added test for substring with negative size. --- .../01060_substring_negative_size.reference | 28 +++++++++++++++ .../01060_substring_negative_size.sql | 36 +++++++++++++++++++ 2 files changed, 64 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01060_substring_negative_size.reference create mode 100644 dbms/tests/queries/0_stateless/01060_substring_negative_size.sql diff --git a/dbms/tests/queries/0_stateless/01060_substring_negative_size.reference b/dbms/tests/queries/0_stateless/01060_substring_negative_size.reference new file mode 100644 index 00000000000..0d787616072 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01060_substring_negative_size.reference @@ -0,0 +1,28 @@ +bcdef +bcdef +bcdef +bcdef +- +bcdef +bcdef +bcdef +bcdef +- +bcdef +23456 +bcdef +3456 +bcdef +2345 +bcdef +345 +- +bcdef +23456 +bcdef +3456 +bcdef +2345 +bcdef +345 + diff --git a/dbms/tests/queries/0_stateless/01060_substring_negative_size.sql b/dbms/tests/queries/0_stateless/01060_substring_negative_size.sql new file mode 100644 index 00000000000..75523ff8aac --- /dev/null +++ b/dbms/tests/queries/0_stateless/01060_substring_negative_size.sql @@ -0,0 +1,36 @@ +select substring('abcdefgh', 2, -2); +select substring('abcdefgh', materialize(2), -2); +select substring('abcdefgh', 2, materialize(-2)); +select substring('abcdefgh', materialize(2), materialize(-2)); + +select '-'; + +select substring(cast('abcdefgh' as FixedString(8)), 2, -2); +select substring(cast('abcdefgh' as FixedString(8)), materialize(2), -2); +select substring(cast('abcdefgh' as FixedString(8)), 2, materialize(-2)); +select substring(cast('abcdefgh' as FixedString(8)), materialize(2), materialize(-2)); + +select '-'; + +drop table t is exists; +create table t (s String, l Int8, r Int8) engine = Memory; +insert into t values ('abcdefgh', 2, -2), ('12345678', 3, -3); + +select substring(s, 2, -2) from t; +select substring(s, l, -2) from t; +select substring(s, 2, r) from t; +select substring(s, l, r) from t; + +select '-'; + +drop table t is exists; +create table t (s FixedString(8), l Int8, r Int8) engine = Memory; +insert into t values ('abcdefgh', 2, -2), ('12345678', 3, -3); + +select substring(s, 2, -2) from t; +select substring(s, l, -2) from t; +select substring(s, 2, r) from t; +select substring(s, l, r) from t; + +drop table t is exists; + From 84bc2bd104b0d8078b510cc0fd2d311b83b1d2f4 Mon Sep 17 00:00:00 2001 From: Alexander Burmak Date: Thu, 9 Jan 2020 15:21:49 +0300 Subject: [PATCH 67/85] Fixed release build --- .../Compression/tests/gtest_compressionCodec.cpp | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/dbms/src/Compression/tests/gtest_compressionCodec.cpp b/dbms/src/Compression/tests/gtest_compressionCodec.cpp index 37807632eff..762ed7d29a1 100644 --- a/dbms/src/Compression/tests/gtest_compressionCodec.cpp +++ b/dbms/src/Compression/tests/gtest_compressionCodec.cpp @@ -301,10 +301,6 @@ struct Codec : codec_statement(std::move(codec_statement_)), expected_compression_ratio(expected_compression_ratio_) {} - - Codec() - : Codec(std::string()) - {} }; @@ -314,23 +310,12 @@ struct CodecTestSequence std::vector serialized_data; DataTypePtr data_type; - CodecTestSequence() - : name(), - serialized_data(), - data_type() - {} - CodecTestSequence(std::string name_, std::vector serialized_data_, DataTypePtr data_type_) : name(name_), serialized_data(serialized_data_), data_type(data_type_) {} - CodecTestSequence(const CodecTestSequence &) = default; - CodecTestSequence & operator=(const CodecTestSequence &) = default; - CodecTestSequence(CodecTestSequence &&) = default; - CodecTestSequence & operator=(CodecTestSequence &&) = default; - CodecTestSequence & append(const CodecTestSequence & other) { assert(data_type->equals(*other.data_type)); From 0b2d238af9a490404216c92f6ac072afcabb3c3d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 15:24:56 +0300 Subject: [PATCH 68/85] Fix DelayedPortsProcessor. --- dbms/src/Processors/DelayedPortsProcessor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/DelayedPortsProcessor.cpp b/dbms/src/Processors/DelayedPortsProcessor.cpp index d5a033dc7df..672f2645c16 100644 --- a/dbms/src/Processors/DelayedPortsProcessor.cpp +++ b/dbms/src/Processors/DelayedPortsProcessor.cpp @@ -29,8 +29,6 @@ bool DelayedPortsProcessor::processPair(PortsPair & pair) { if (!pair.is_finished) { - pair.input_port->close(); - pair.is_finished = true; ++num_finished; } @@ -38,12 +36,14 @@ bool DelayedPortsProcessor::processPair(PortsPair & pair) if (pair.output_port->isFinished()) { + pair.input_port->close(); finish(); return false; } if (pair.input_port->isFinished()) { + pair.output_port->finish(); finish(); return false; } From bc7df87c260687e0c1a3deb5a7c5f0377a415650 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 15:52:17 +0300 Subject: [PATCH 69/85] Fix test. --- .../0_stateless/01060_substring_negative_size.reference | 1 - .../queries/0_stateless/01060_substring_negative_size.sql | 6 +++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/dbms/tests/queries/0_stateless/01060_substring_negative_size.reference b/dbms/tests/queries/0_stateless/01060_substring_negative_size.reference index 0d787616072..b25696dc7d6 100644 --- a/dbms/tests/queries/0_stateless/01060_substring_negative_size.reference +++ b/dbms/tests/queries/0_stateless/01060_substring_negative_size.reference @@ -25,4 +25,3 @@ bcdef 2345 bcdef 345 - diff --git a/dbms/tests/queries/0_stateless/01060_substring_negative_size.sql b/dbms/tests/queries/0_stateless/01060_substring_negative_size.sql index 75523ff8aac..23cab14a6e0 100644 --- a/dbms/tests/queries/0_stateless/01060_substring_negative_size.sql +++ b/dbms/tests/queries/0_stateless/01060_substring_negative_size.sql @@ -12,7 +12,7 @@ select substring(cast('abcdefgh' as FixedString(8)), materialize(2), materialize select '-'; -drop table t is exists; +drop table if exists t; create table t (s String, l Int8, r Int8) engine = Memory; insert into t values ('abcdefgh', 2, -2), ('12345678', 3, -3); @@ -23,7 +23,7 @@ select substring(s, l, r) from t; select '-'; -drop table t is exists; +drop table if exists t; create table t (s FixedString(8), l Int8, r Int8) engine = Memory; insert into t values ('abcdefgh', 2, -2), ('12345678', 3, -3); @@ -32,5 +32,5 @@ select substring(s, l, -2) from t; select substring(s, 2, r) from t; select substring(s, l, r) from t; -drop table t is exists; +drop table if exists t; From 6d3828b8f2d7e0e1cb0c9463369bf664e1d9f0d8 Mon Sep 17 00:00:00 2001 From: Alexander Burmak Date: Thu, 9 Jan 2020 17:08:31 +0300 Subject: [PATCH 70/85] Revert "Fixed release build" This reverts commit 84bc2bd104b0d8078b510cc0fd2d311b83b1d2f4. --- .../Compression/tests/gtest_compressionCodec.cpp | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/dbms/src/Compression/tests/gtest_compressionCodec.cpp b/dbms/src/Compression/tests/gtest_compressionCodec.cpp index 762ed7d29a1..37807632eff 100644 --- a/dbms/src/Compression/tests/gtest_compressionCodec.cpp +++ b/dbms/src/Compression/tests/gtest_compressionCodec.cpp @@ -301,6 +301,10 @@ struct Codec : codec_statement(std::move(codec_statement_)), expected_compression_ratio(expected_compression_ratio_) {} + + Codec() + : Codec(std::string()) + {} }; @@ -310,12 +314,23 @@ struct CodecTestSequence std::vector serialized_data; DataTypePtr data_type; + CodecTestSequence() + : name(), + serialized_data(), + data_type() + {} + CodecTestSequence(std::string name_, std::vector serialized_data_, DataTypePtr data_type_) : name(name_), serialized_data(serialized_data_), data_type(data_type_) {} + CodecTestSequence(const CodecTestSequence &) = default; + CodecTestSequence & operator=(const CodecTestSequence &) = default; + CodecTestSequence(CodecTestSequence &&) = default; + CodecTestSequence & operator=(CodecTestSequence &&) = default; + CodecTestSequence & append(const CodecTestSequence & other) { assert(data_type->equals(*other.data_type)); From d23f9671488994e8c20d881795cce331963a6fd2 Mon Sep 17 00:00:00 2001 From: Alexander Burmak Date: Thu, 9 Jan 2020 17:18:00 +0300 Subject: [PATCH 71/85] Fixed release build - v2 --- .../Compression/tests/gtest_compressionCodec.cpp | 15 +-------------- 1 file changed, 1 insertion(+), 14 deletions(-) diff --git a/dbms/src/Compression/tests/gtest_compressionCodec.cpp b/dbms/src/Compression/tests/gtest_compressionCodec.cpp index 37807632eff..b2b4f6918e6 100644 --- a/dbms/src/Compression/tests/gtest_compressionCodec.cpp +++ b/dbms/src/Compression/tests/gtest_compressionCodec.cpp @@ -301,10 +301,6 @@ struct Codec : codec_statement(std::move(codec_statement_)), expected_compression_ratio(expected_compression_ratio_) {} - - Codec() - : Codec(std::string()) - {} }; @@ -314,11 +310,7 @@ struct CodecTestSequence std::vector serialized_data; DataTypePtr data_type; - CodecTestSequence() - : name(), - serialized_data(), - data_type() - {} + CodecTestSequence() = default; CodecTestSequence(std::string name_, std::vector serialized_data_, DataTypePtr data_type_) : name(name_), @@ -326,11 +318,6 @@ struct CodecTestSequence data_type(data_type_) {} - CodecTestSequence(const CodecTestSequence &) = default; - CodecTestSequence & operator=(const CodecTestSequence &) = default; - CodecTestSequence(CodecTestSequence &&) = default; - CodecTestSequence & operator=(CodecTestSequence &&) = default; - CodecTestSequence & append(const CodecTestSequence & other) { assert(data_type->equals(*other.data_type)); From 5acbe8e5bd0c3d15aed9d6cdf54e2805ffb1c66c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 9 Jan 2020 18:15:42 +0300 Subject: [PATCH 72/85] Update security_changelog.md --- docs/ru/security_changelog.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/security_changelog.md b/docs/ru/security_changelog.md index 4e588386af2..66d76849d15 100644 --- a/docs/ru/security_changelog.md +++ b/docs/ru/security_changelog.md @@ -2,7 +2,7 @@ ### CVE-2019-15024 -Злоумышленник с доступом на запись к ZooKeeper и возможностью запустить собственный сервер в сети доступной ClickHouse может создать вредоносный сервер, который будет вести себя как реплика ClickHouse и зарегистрируется в ZooKeeper. В процессе репликации вредоносный сервер может указать любой путь на файловой системе в который будут записаны данные. +Злоумышленник с доступом на запись к ZooKeeper и возможностью запустить собственный сервер в сети доступной ClickHouse может создать вредоносный сервер, который будет вести себя как реплика ClickHouse и зарегистрируется в ZooKeeper. В процессе репликации вредоносный сервер может указать любой путь на файловой системе в который будут записаны данные. Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса From 334b631759a685bf9ba34248c8b8370b9bba84ed Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 9 Jan 2020 18:16:15 +0300 Subject: [PATCH 73/85] Update security_changelog.md --- docs/ru/security_changelog.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/security_changelog.md b/docs/ru/security_changelog.md index 66d76849d15..db742b5f990 100644 --- a/docs/ru/security_changelog.md +++ b/docs/ru/security_changelog.md @@ -4,19 +4,19 @@ Злоумышленник с доступом на запись к ZooKeeper и возможностью запустить собственный сервер в сети доступной ClickHouse может создать вредоносный сервер, который будет вести себя как реплика ClickHouse и зарегистрируется в ZooKeeper. В процессе репликации вредоносный сервер может указать любой путь на файловой системе в который будут записаны данные. -Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса +Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса ### CVE-2019-16535 Интерфейс декомпрессии позволял совершать OOB чтения и записи данных в памяти, а также переполнение целочисленных переменных, что могло приводить к отказу в обслуживании. Также потенциально могло использоваьтся для удаленного выполнения кода. -Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса +Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса ### CVE-2019-16536 Аутентифицированный клиент злоумышленника имел возможность вызвать переполнение стека, что могло привести к отказу в обслуживании. -Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса +Обнаружено благодаря: Эльдару Заитову из Службы Информационной Безопасности Яндекса ## Исправлено в релизе 19.13.6.1 от 20 сентября 2019 From 2c73233e8263c6c8c0675e0680c9834c1545acb4 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 9 Jan 2020 18:35:45 +0300 Subject: [PATCH 74/85] Report a more verbose error when unable to select parts to merge. --- dbms/src/Storages/StorageMergeTree.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 9a0583a464a..865058c2f79 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -612,7 +612,13 @@ bool StorageMergeTree::merge( if (!selected) { if (out_disable_reason) - *out_disable_reason = "Cannot select parts for optimization"; + { + if (!out_disable_reason->empty()) + { + *out_disable_reason += ". "; + } + *out_disable_reason += "Cannot select parts for optimization"; + } return false; } From 9069a9d4f115ea0c7311fe927fe776ad454cefcc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Jan 2020 19:27:49 +0300 Subject: [PATCH 75/85] Fix test. --- .../01033_substr_negative_size_arg.reference | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/dbms/tests/queries/0_stateless/01033_substr_negative_size_arg.reference b/dbms/tests/queries/0_stateless/01033_substr_negative_size_arg.reference index 98c07557034..db3a106ac7f 100644 --- a/dbms/tests/queries/0_stateless/01033_substr_negative_size_arg.reference +++ b/dbms/tests/queries/0_stateless/01033_substr_negative_size_arg.reference @@ -1,8 +1,8 @@ -lickhous -lickhous -lickhous -lickhous -lickhous -lickhous -lickhous -lickhous +lickhou +lickhou +lickhou +lickhou +lickhou +lickhou +lickhou +lickhou From e594bc10618b6a94bb5222844ff88be358cd129a Mon Sep 17 00:00:00 2001 From: Sergei Semin Date: Thu, 9 Jan 2020 19:39:27 +0300 Subject: [PATCH 76/85] fix typo in tutorial.md replace ant with want (forgotten w) --- docs/en/getting_started/tutorial.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting_started/tutorial.md b/docs/en/getting_started/tutorial.md index acdd9074beb..bffee808122 100644 --- a/docs/en/getting_started/tutorial.md +++ b/docs/en/getting_started/tutorial.md @@ -444,7 +444,7 @@ SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192 ``` -You can execute those queries using interactive mode of `clickhouse-client` (just launch it in terminal without specifying a query in advance) or try some [alternative interface](../interfaces/index.md) if you ant. +You can execute those queries using interactive mode of `clickhouse-client` (just launch it in terminal without specifying a query in advance) or try some [alternative interface](../interfaces/index.md) if you want. As we can see, `hits_v1` uses the [basic MergeTree engine](../operations/table_engines/mergetree.md), while the `visits_v1` uses the [Collapsing](../operations/table_engines/collapsingmergetree.md) variant. From 38bcc5996b83ad84d438ad87ebeaba8e23cf0a3a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 9 Jan 2020 19:42:20 +0300 Subject: [PATCH 77/85] Update apply_catboost_model.md --- docs/en/guides/apply_catboost_model.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/guides/apply_catboost_model.md b/docs/en/guides/apply_catboost_model.md index 5e868bffe0d..06863bb48f9 100644 --- a/docs/en/guides/apply_catboost_model.md +++ b/docs/en/guides/apply_catboost_model.md @@ -74,7 +74,7 @@ $ clickhouse client ROLE_FAMILY UInt32, ROLE_CODE UInt32 ) -ENGINE = MergeTree(date, date, 8192) +ENGINE = MergeTree ORDER BY date ``` **3.** Exit from ClickHouse console client: From 2c6b00ab2854ddf2962867b18f21bb1de1b35807 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 9 Jan 2020 19:42:37 +0300 Subject: [PATCH 78/85] Update apply_catboost_model.md --- docs/ru/guides/apply_catboost_model.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/guides/apply_catboost_model.md b/docs/ru/guides/apply_catboost_model.md index 65cb3f58508..69aa0faccb2 100644 --- a/docs/ru/guides/apply_catboost_model.md +++ b/docs/ru/guides/apply_catboost_model.md @@ -74,7 +74,7 @@ $ clickhouse client ROLE_FAMILY UInt32, ROLE_CODE UInt32 ) -ENGINE = MergeTree(date, date, 8192) +ENGINE = MergeTree ORDER BY date ``` **3.** Выйдите из клиента ClickHouse: From 6e03cc2293e5cdeaa1d02c2d955473da0d6a41a5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 9 Jan 2020 20:02:55 +0300 Subject: [PATCH 79/85] Update CODEOWNERS --- .github/CODEOWNERS | 2 -- 1 file changed, 2 deletions(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 257040c68b7..8e502c0b36f 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -1,4 +1,2 @@ -dbms/* @ClickHouse/core-assigner -utils/* @ClickHouse/core-assigner docs/* @ClickHouse/docs docs/zh/* @ClickHouse/docs-zh From 3e76a3bd6738ec5a7f429f93bd9f4260699f5c24 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 10 Jan 2020 07:00:37 +0800 Subject: [PATCH 80/85] Add upcoming SF meetup --- README.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/README.md b/README.md index a545c91886f..21498a22912 100644 --- a/README.md +++ b/README.md @@ -11,3 +11,7 @@ ClickHouse is an open-source column-oriented database management system that all * [Blog](https://clickhouse.yandex/blog/en/) contains various ClickHouse-related articles, as well as announces and reports about events. * [Contacts](https://clickhouse.yandex/#contacts) can help to get your questions answered if there are any. * You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person. + +## Upcoming Events + +* [ClickHouse Meetup in San Francisco](https://www.eventbrite.com/e/clickhouse-february-meetup-registration-88496227599) on February 5. From 1de3261ec4186c25cf2dbf4d3e4fa4c6971f99bb Mon Sep 17 00:00:00 2001 From: meo Date: Fri, 10 Jan 2020 00:53:07 +0100 Subject: [PATCH 81/85] Catalina support support was only introduced in Mac OS 10.15: minimum version must be raised to compile on Catalina --- cmake/darwin/default_libs.cmake | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/cmake/darwin/default_libs.cmake b/cmake/darwin/default_libs.cmake index 6010ea0f5de..7b57e63f4ee 100644 --- a/cmake/darwin/default_libs.cmake +++ b/cmake/darwin/default_libs.cmake @@ -13,12 +13,12 @@ set(CMAKE_C_STANDARD_LIBRARIES ${DEFAULT_LIBS}) # Minimal supported SDK version -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mmacosx-version-min=10.14") -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mmacosx-version-min=10.14") -set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} -mmacosx-version-min=10.14") +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -mmacosx-version-min=10.15") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -mmacosx-version-min=10.15") +set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} -mmacosx-version-min=10.15") -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -mmacosx-version-min=10.14") -set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -mmacosx-version-min=10.14") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -mmacosx-version-min=10.15") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -mmacosx-version-min=10.15") # Global libraries From 01358358556dc2354a39a715f6139724daf05c4f Mon Sep 17 00:00:00 2001 From: Alexander Burmak Date: Fri, 10 Jan 2020 10:03:34 +0300 Subject: [PATCH 82/85] Additional release build fix --- .../tests/gtest_compressionCodec.cpp | 37 +++++++++---------- 1 file changed, 17 insertions(+), 20 deletions(-) diff --git a/dbms/src/Compression/tests/gtest_compressionCodec.cpp b/dbms/src/Compression/tests/gtest_compressionCodec.cpp index b2b4f6918e6..f15cb881798 100644 --- a/dbms/src/Compression/tests/gtest_compressionCodec.cpp +++ b/dbms/src/Compression/tests/gtest_compressionCodec.cpp @@ -310,8 +310,6 @@ struct CodecTestSequence std::vector serialized_data; DataTypePtr data_type; - CodecTestSequence() = default; - CodecTestSequence(std::string name_, std::vector serialized_data_, DataTypePtr data_type_) : name(name_), serialized_data(serialized_data_), @@ -806,24 +804,6 @@ std::vector generatePyramidOfSequences(const size_t sequences return sequences; }; -// Just as if all sequences from generatePyramidOfSequences were appended to one-by-one to the first one. -template -CodecTestSequence generatePyramidSequence(const size_t sequences_count, Generator && generator, const char* generator_name) -{ - CodecTestSequence sequence; - sequence.data_type = makeDataType(); - sequence.serialized_data.reserve(sequences_count * sequences_count * sizeof(T)); - - for (size_t i = 1; i < sequences_count; ++i) - { - std::string name = generator_name + std::string(" from 0 to ") + std::to_string(i); - sequence.append(generateSeq(std::forward(generator), name.c_str(), 0, i)); - } - - return sequence; -}; - - // helper macro to produce human-friendly sequence name from generator #define G(generator) generator, #generator @@ -1295,6 +1275,23 @@ INSTANTIATE_TEST_CASE_P(Gorilla, // also they require pretty big data to run agains and generating this data slows down startup of unit test process. // So un-comment only at your discretion. +// Just as if all sequences from generatePyramidOfSequences were appended to one-by-one to the first one. +//template +//CodecTestSequence generatePyramidSequence(const size_t sequences_count, Generator && generator, const char* generator_name) +//{ +// CodecTestSequence sequence; +// sequence.data_type = makeDataType(); +// sequence.serialized_data.reserve(sequences_count * sequences_count * sizeof(T)); +// +// for (size_t i = 1; i < sequences_count; ++i) +// { +// std::string name = generator_name + std::string(" from 0 to ") + std::to_string(i); +// sequence.append(generateSeq(std::forward(generator), name.c_str(), 0, i)); +// } +// +// return sequence; +//}; + //INSTANTIATE_TEST_CASE_P(DoubleDelta, // CodecTest_Performance, // ::testing::Combine( From d7bfb5c7b05332c4f89a39e60d8a4cf77aa7d40f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 10 Jan 2020 11:05:08 +0300 Subject: [PATCH 83/85] refactor changelog --- CHANGELOG.md | 838 +++++++++++++++++++++++++++------------------------ 1 file changed, 437 insertions(+), 401 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a6757c38898..f456a56f1be 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,8 @@ -## ClickHouse release v19.17.6.36, 2019-12-27 +## ClickHouse release v19.17 -### Bug Fix +### ClickHouse release v19.17.6.36, 2019-12-27 + +#### Bug Fix * Fixed potential buffer overflow in decompress. Malicious user can pass fabricated compressed data that could cause read after buffer. This issue was found by Eldar Zaitov from Yandex information security team. [#8404](https://github.com/ClickHouse/ClickHouse/pull/8404) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed possible server crash (`std::terminate`) when the server cannot send or write data in JSON or XML format with values of String data type (that require UTF-8 validation) or when compressing result data with Brotli algorithm or in some other rare cases. [#8384](https://github.com/ClickHouse/ClickHouse/pull/8384) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed dictionaries with source from a clickhouse `VIEW`, now reading such dictionaries doesn't cause the error `There is no query`. [#8351](https://github.com/ClickHouse/ClickHouse/pull/8351) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) @@ -32,13 +34,12 @@ next request would interpret this info as the beginning of the next query causin * Now an exception will be thrown in case of using WITH TIES alongside LIMIT BY. And now it's possible to use TOP with LIMIT BY. [#7637](https://github.com/ClickHouse/ClickHouse/pull/7637) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) * Fix dictionary reload if it has `invalidate_query`, which stopped updates and some exception on previous update tries. [#8029](https://github.com/ClickHouse/ClickHouse/pull/8029) ([alesapin](https://github.com/alesapin)) +### ClickHouse release v19.17.4.11, 2019-11-22 -## ClickHouse release v19.17.4.11, 2019-11-22 - -### Backward Incompatible Change +#### Backward Incompatible Change * Using column instead of AST to store scalar subquery results for better performance. Setting `enable_scalar_subquery_optimization` was added in 19.17 and it was enabled by default. It leads to errors like [this](https://github.com/ClickHouse/ClickHouse/issues/7851) during upgrade to 19.17.2 or 19.17.3 from previous versions. This setting was disabled by default in 19.17.4, to make possible upgrading from 19.16 and older versions without errors. [#7392](https://github.com/ClickHouse/ClickHouse/pull/7392) ([Amos Bird](https://github.com/amosbird)) -### New Feature +#### New Feature * Add the ability to create dictionaries with DDL queries. [#7360](https://github.com/ClickHouse/ClickHouse/pull/7360) ([alesapin](https://github.com/alesapin)) * Make `bloom_filter` type of index supporting `LowCardinality` and `Nullable` [#7363](https://github.com/ClickHouse/ClickHouse/issues/7363) [#7561](https://github.com/ClickHouse/ClickHouse/pull/7561) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) * Add function `isValidJSON` to check that passed string is a valid json. [#5910](https://github.com/ClickHouse/ClickHouse/issues/5910) [#7293](https://github.com/ClickHouse/ClickHouse/pull/7293) ([Vdimir](https://github.com/Vdimir)) @@ -51,10 +52,10 @@ next request would interpret this info as the beginning of the next query causin * Implemented `javaHashUTF16LE()` function [#7651](https://github.com/ClickHouse/ClickHouse/pull/7651) ([achimbab](https://github.com/achimbab)) * Add `_shard_num` virtual column for the Distributed engine [#7624](https://github.com/ClickHouse/ClickHouse/pull/7624) ([Azat Khuzhin](https://github.com/azat)) -### Experimental Feature +#### Experimental Feature * Support for processors (new query execution pipeline) in `MergeTree`. [#7181](https://github.com/ClickHouse/ClickHouse/pull/7181) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -### Bug Fix +#### Bug Fix * Fix incorrect float parsing in `Values` [#7817](https://github.com/ClickHouse/ClickHouse/issues/7817) [#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([tavplubix](https://github.com/tavplubix)) * Fix rare deadlock which can happen when trace_log is enabled. [#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov)) * Prevent message duplication when producing Kafka table has any MVs selecting from it [#7265](https://github.com/ClickHouse/ClickHouse/pull/7265) ([Ivan](https://github.com/abyss7)) @@ -78,7 +79,7 @@ next request would interpret this info as the beginning of the next query causin * Fixed exception in case of using 1 argument while defining S3, URL and HDFS storages. [#7618](https://github.com/ClickHouse/ClickHouse/pull/7618) ([Vladimir Chebotarev](https://github.com/excitoon)) * Fix scope of the InterpreterSelectQuery for views with query [#7601](https://github.com/ClickHouse/ClickHouse/pull/7601) ([Azat Khuzhin](https://github.com/azat)) -### Improvement +#### Improvement * `Nullable` columns recognized and NULL-values handled correctly by ODBC-bridge [#7402](https://github.com/ClickHouse/ClickHouse/pull/7402) ([Vasily Nemkov](https://github.com/Enmk)) * Write current batch for distributed send atomically [#7600](https://github.com/ClickHouse/ClickHouse/pull/7600) ([Azat Khuzhin](https://github.com/azat)) * Throw an exception if we cannot detect table for column name in query. [#7358](https://github.com/ClickHouse/ClickHouse/pull/7358) ([Artem Zuikov](https://github.com/4ertus2)) @@ -90,14 +91,14 @@ next request would interpret this info as the beginning of the next query causin * Better Null format for tcp handler, so that it's possible to use `select ignore() from table format Null` for perf measure via clickhouse-client [#7606](https://github.com/ClickHouse/ClickHouse/pull/7606) ([Amos Bird](https://github.com/amosbird)) * Queries like `CREATE TABLE ... AS (SELECT (1, 2))` are parsed correctly [#7542](https://github.com/ClickHouse/ClickHouse/pull/7542) ([hcz](https://github.com/hczhcz)) -### Performance Improvement +#### Performance Improvement * The performance of aggregation over short string keys is improved. [#6243](https://github.com/ClickHouse/ClickHouse/pull/6243) ([Alexander Kuzmenkov](https://github.com/akuzm), [Amos Bird](https://github.com/amosbird)) * Run another pass of syntax/expression analysis to get potential optimizations after constant predicates are folded. [#7497](https://github.com/ClickHouse/ClickHouse/pull/7497) ([Amos Bird](https://github.com/amosbird)) * Use storage meta info to evaluate trivial `SELECT count() FROM table;` [#7510](https://github.com/ClickHouse/ClickHouse/pull/7510) ([Amos Bird](https://github.com/amosbird), [alexey-milovidov](https://github.com/alexey-milovidov)) * Vectorize processing `arrayReduce` similar to Aggregator `addBatch`. [#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([Amos Bird](https://github.com/amosbird)) * Minor improvements in performance of `Kafka` consumption [#7475](https://github.com/ClickHouse/ClickHouse/pull/7475) ([Ivan](https://github.com/abyss7)) -### Build/Testing/Packaging Improvement +#### Build/Testing/Packaging Improvement * Add support for cross-compiling to the CPU architecture AARCH64. Refactor packager script. [#7370](https://github.com/ClickHouse/ClickHouse/pull/7370) [#7539](https://github.com/ClickHouse/ClickHouse/pull/7539) ([Ivan](https://github.com/abyss7)) * Unpack darwin-x86_64 and linux-aarch64 toolchains into mounted Docker volume when building packages [#7534](https://github.com/ClickHouse/ClickHouse/pull/7534) ([Ivan](https://github.com/abyss7)) * Update Docker Image for Binary Packager [#7474](https://github.com/ClickHouse/ClickHouse/pull/7474) ([Ivan](https://github.com/abyss7)) @@ -108,13 +109,14 @@ next request would interpret this info as the beginning of the next query causin * Remove hardcoded paths in `unwind` target [#7460](https://github.com/ClickHouse/ClickHouse/pull/7460) ([Konstantin Podshumok](https://github.com/podshumok)) * Allow to use mysql format without ssl [#7524](https://github.com/ClickHouse/ClickHouse/pull/7524) ([proller](https://github.com/proller)) -### Other +#### Other * Added ANTLR4 grammar for ClickHouse SQL dialect [#7595](https://github.com/ClickHouse/ClickHouse/issues/7595) [#7596](https://github.com/ClickHouse/ClickHouse/pull/7596) ([alexey-milovidov](https://github.com/alexey-milovidov)) +## ClickHouse release v19.16 -## ClickHouse release v19.16.2.2, 2019-10-30 +### ClickHouse release v19.16.2.2, 2019-10-30 -### Backward Incompatible Change +#### Backward Incompatible Change * Add missing arity validation for count/counIf. [#7095](https://github.com/ClickHouse/ClickHouse/issues/7095) [#7298](https://github.com/ClickHouse/ClickHouse/pull/7298) ([Vdimir](https://github.com/Vdimir)) @@ -125,7 +127,7 @@ Zuikov](https://github.com/4ertus2)) [#7118](https://github.com/ClickHouse/ClickHouse/pull/7118) ([tavplubix](https://github.com/tavplubix)) -### New Feature +#### New Feature * Introduce uniqCombined64() to calculate cardinality greater than UINT_MAX. [#7213](https://github.com/ClickHouse/ClickHouse/pull/7213), [#7222](https://github.com/ClickHouse/ClickHouse/pull/7222) ([Azat @@ -166,7 +168,7 @@ Yu](https://github.com/yuzhichang)) * Support Redis as source of external dictionary. [#4361](https://github.com/ClickHouse/ClickHouse/pull/4361) [#6962](https://github.com/ClickHouse/ClickHouse/pull/6962) ([comunodi](https://github.com/comunodi), [Anton Popov](https://github.com/CurtizJ)) -### Bug Fix +#### Bug Fix * Fix wrong query result if it has `WHERE IN (SELECT ...)` section and `optimize_read_in_order` is used. [#7371](https://github.com/ClickHouse/ClickHouse/pull/7371) ([Anton Popov](https://github.com/CurtizJ)) @@ -207,7 +209,7 @@ Kochetov](https://github.com/KochetovNicolai)) [#7271](https://github.com/ClickHouse/ClickHouse/pull/7271) ([vzakaznikov](https://github.com/vzakaznikov)) -### Improvement +#### Improvement * Add a message in case of queue_wait_max_ms wait takes place. [#7390](https://github.com/ClickHouse/ClickHouse/pull/7390) ([Azat Khuzhin](https://github.com/azat)) @@ -252,7 +254,7 @@ Khuzhin](https://github.com/azat)) memory). Load data back when needed. [#7186](https://github.com/ClickHouse/ClickHouse/pull/7186) ([Artem Zuikov](https://github.com/4ertus2)) -### Performance Improvement +#### Performance Improvement * Speed up joinGet with const arguments by avoiding data duplication. [#7359](https://github.com/ClickHouse/ClickHouse/pull/7359) ([Amos Bird](https://github.com/amosbird)) @@ -262,7 +264,7 @@ Bird](https://github.com/amosbird)) [#6781](https://github.com/ClickHouse/ClickHouse/pull/6781) ([tavplubix](https://github.com/tavplubix)) -### Build/Testing/Packaging Improvement +#### Build/Testing/Packaging Improvement * Disable some contribs for cross-compilation to Mac OS. [#7101](https://github.com/ClickHouse/ClickHouse/pull/7101) ([Ivan](https://github.com/abyss7)) * Add missing linking with PocoXML for clickhouse_common_io. @@ -330,7 +332,7 @@ Bird](https://github.com/amosbird)) [#7063](https://github.com/ClickHouse/ClickHouse/pull/7063) ([proller](https://github.com/proller)) -### Code cleanup +#### Code cleanup * Generalize configuration repository to prepare for DDL for Dictionaries. [#7155](https://github.com/ClickHouse/ClickHouse/pull/7155) ([alesapin](https://github.com/alesapin)) * Parser for dictionaries DDL without any semantic. @@ -364,9 +366,11 @@ fix comments to make obvious that it may throw. [#7350](https://github.com/ClickHouse/ClickHouse/pull/7350) ([tavplubix](https://github.com/tavplubix)) -## ClickHouse release 19.15.4.10, 2019-10-31 +## ClickHouse release 19.15 -### Bug Fix +### ClickHouse release 19.15.4.10, 2019-10-31 + +#### Bug Fix * Added handling of SQL_TINYINT and SQL_BIGINT, and fix handling of SQL_FLOAT data source types in ODBC Bridge. [#7491](https://github.com/ClickHouse/ClickHouse/pull/7491) ([Denis Glazachev](https://github.com/traceon)) * Allowed to have some parts on destination disk or volume in MOVE PARTITION. @@ -391,9 +395,9 @@ fix comments to make obvious that it may throw. [#7158](https://github.com/ClickHouse/ClickHouse/pull/7158) ([Azat Khuzhin](https://github.com/azat)) * Added example config with macros for tests ([alexey-milovidov](https://github.com/alexey-milovidov)) -## ClickHouse release 19.15.3.6, 2019-10-09 +### ClickHouse release 19.15.3.6, 2019-10-09 -### Bug Fix +#### Bug Fix * Fixed bad_variant in hashed dictionary. ([alesapin](https://github.com/alesapin)) * Fixed up bug with segmentation fault in ATTACH PART query. @@ -405,9 +409,9 @@ fix comments to make obvious that it may throw. * Serialize NULL values correctly in min/max indexes of MergeTree parts. [#7234](https://github.com/ClickHouse/ClickHouse/pull/7234) ([Alexander Kuzmenkov](https://github.com/akuzm)) -## ClickHouse release 19.15.2.2, 2019-10-01 +### ClickHouse release 19.15.2.2, 2019-10-01 -### New Feature +#### New Feature * Tiered storage: support to use multiple storage volumes for tables with MergeTree engine. It's possible to store fresh data on SSD and automatically move old data to HDD. ([example](https://clickhouse.github.io/clickhouse-presentations/meetup30/new_features/#12)). [#4918](https://github.com/ClickHouse/ClickHouse/pull/4918) ([Igr](https://github.com/ObjatieGroba)) [#6489](https://github.com/ClickHouse/ClickHouse/pull/6489) ([alesapin](https://github.com/alesapin)) * Add table function `input` for reading incoming data in `INSERT SELECT` query. [#5450](https://github.com/ClickHouse/ClickHouse/pull/5450) ([palasonic1](https://github.com/palasonic1)) [#6832](https://github.com/ClickHouse/ClickHouse/pull/6832) ([Anton Popov](https://github.com/CurtizJ)) * Add a `sparse_hashed` dictionary layout, that is functionally equivalent to the `hashed` layout, but is more memory efficient. It uses about twice as less memory at the cost of slower value retrieval. [#6894](https://github.com/ClickHouse/ClickHouse/pull/6894) ([Azat Khuzhin](https://github.com/azat)) @@ -417,11 +421,11 @@ fix comments to make obvious that it may throw. * Add `bitmapMin` and `bitmapMax` functions. [#6970](https://github.com/ClickHouse/ClickHouse/pull/6970) ([Zhichang Yu](https://github.com/yuzhichang)) * Add function `repeat` related to [issue-6648](https://github.com/yandex/ClickHouse/issues/6648) [#6999](https://github.com/ClickHouse/ClickHouse/pull/6999) ([flynn](https://github.com/ucasFL)) -### Experimental Feature +#### Experimental Feature * Implement (in memory) Merge Join variant that does not change current pipeline. Result is partially sorted by merge key. Set `partial_merge_join = 1` to use this feature. The Merge Join is still in development. [#6940](https://github.com/ClickHouse/ClickHouse/pull/6940) ([Artem Zuikov](https://github.com/4ertus2)) * Add `S3` engine and table function. It is still in development (no authentication support yet). [#5596](https://github.com/ClickHouse/ClickHouse/pull/5596) ([Vladimir Chebotarev](https://github.com/excitoon)) -### Improvement +#### Improvement * Every message read from Kafka is inserted atomically. This resolves almost all known issues with Kafka engine. [#6950](https://github.com/ClickHouse/ClickHouse/pull/6950) ([Ivan](https://github.com/abyss7)) * Improvements for failover of Distributed queries. Shorten recovery time, also it is now configurable and can be seen in `system.clusters`. [#6399](https://github.com/ClickHouse/ClickHouse/pull/6399) ([Vasily Nemkov](https://github.com/Enmk)) * Support numeric values for Enums directly in `IN` section. #6766 [#6941](https://github.com/ClickHouse/ClickHouse/pull/6941) ([dimarub2000](https://github.com/dimarub2000)) @@ -432,7 +436,7 @@ fix comments to make obvious that it may throw. * Add automatically cast type `T` to `LowCardinality(T)` while inserting data in column of type `LowCardinality(T)` in Native format via HTTP. [#6891](https://github.com/ClickHouse/ClickHouse/pull/6891) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) * Add ability to use function `hex` without using `reinterpretAsString` for `Float32`, `Float64`. [#7024](https://github.com/ClickHouse/ClickHouse/pull/7024) ([Mikhail Korotov](https://github.com/millb)) -### Build/Testing/Packaging Improvement +#### Build/Testing/Packaging Improvement * Add gdb-index to clickhouse binary with debug info. It will speed up startup time of `gdb`. [#6947](https://github.com/ClickHouse/ClickHouse/pull/6947) ([alesapin](https://github.com/alesapin)) * Speed up deb packaging with patched dpkg-deb which uses `pigz`. [#6960](https://github.com/ClickHouse/ClickHouse/pull/6960) ([alesapin](https://github.com/alesapin)) * Set `enable_fuzzing = 1` to enable libfuzzer instrumentation of all the project code. [#7042](https://github.com/ClickHouse/ClickHouse/pull/7042) ([kyprizel](https://github.com/kyprizel)) @@ -440,7 +444,7 @@ fix comments to make obvious that it may throw. * Add build with MemorySanitizer to CI. [#7066](https://github.com/ClickHouse/ClickHouse/pull/7066) ([Alexander Kuzmenkov](https://github.com/akuzm)) * Replace `libsparsehash` with `sparsehash-c11` [#6965](https://github.com/ClickHouse/ClickHouse/pull/6965) ([Azat Khuzhin](https://github.com/azat)) -### Bug Fix +#### Bug Fix * Fixed performance degradation of index analysis on complex keys on large tables. This fixes #6924. [#7075](https://github.com/ClickHouse/ClickHouse/pull/7075) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix logical error causing segfaults when selecting from Kafka empty topic. [#6909](https://github.com/ClickHouse/ClickHouse/pull/6909) ([Ivan](https://github.com/abyss7)) * Fix too early MySQL connection close in `MySQLBlockInputStream.cpp`. [#6882](https://github.com/ClickHouse/ClickHouse/pull/6882) ([Clément Rodriguez](https://github.com/clemrodriguez)) @@ -451,28 +455,29 @@ fix comments to make obvious that it may throw. * Fix `Unknown identifier` error in ORDER BY and GROUP BY with multiple JOINs [#7022](https://github.com/ClickHouse/ClickHouse/pull/7022) ([Artem Zuikov](https://github.com/4ertus2)) * Fixed `MSan` warning while executing function with `LowCardinality` argument. [#7062](https://github.com/ClickHouse/ClickHouse/pull/7062) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -### Backward Incompatible Change +#### Backward Incompatible Change * Changed serialization format of bitmap* aggregate function states to improve performance. Serialized states of bitmap* from previous versions cannot be read. [#6908](https://github.com/ClickHouse/ClickHouse/pull/6908) ([Zhichang Yu](https://github.com/yuzhichang)) -## ClickHouse release 19.14.7.15, 2019-10-02 +## ClickHouse release 19.14 +### ClickHouse release 19.14.7.15, 2019-10-02 -### Bug Fix +#### Bug Fix * This release also contains all bug fixes from 19.11.12.69. * Fixed compatibility for distributed queries between 19.14 and earlier versions. This fixes [#7068](https://github.com/ClickHouse/ClickHouse/issues/7068). [#7069](https://github.com/ClickHouse/ClickHouse/pull/7069) ([alexey-milovidov](https://github.com/alexey-milovidov)) -## ClickHouse release 19.14.6.12, 2019-09-19 +### ClickHouse release 19.14.6.12, 2019-09-19 -### Bug Fix +#### Bug Fix * Fix for function `АrrayEnumerateUniqRanked` with empty arrays in params. [#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([proller](https://github.com/proller)) * Fixed subquery name in queries with `ARRAY JOIN` and `GLOBAL IN subquery` with alias. Use subquery alias for external table name if it is specified. [#6934](https://github.com/ClickHouse/ClickHouse/pull/6934) ([Ivan](https://github.com/abyss7)) -### Build/Testing/Packaging Improvement +#### Build/Testing/Packaging Improvement * Fix [flapping](https://clickhouse-test-reports.s3.yandex.net/6944/aab95fd5175a513413c7395a73a82044bdafb906/functional_stateless_tests_(debug).html) test `00715_fetch_merged_or_mutated_part_zookeeper` by rewriting it to a shell scripts because it needs to wait for mutations to apply. [#6977](https://github.com/ClickHouse/ClickHouse/pull/6977) ([Alexander Kazakov](https://github.com/Akazz)) * Fixed UBSan and MemSan failure in function `groupUniqArray` with emtpy array argument. It was caused by placing of empty `PaddedPODArray` into hash table zero cell because constructor for zero cell value was not called. [#6937](https://github.com/ClickHouse/ClickHouse/pull/6937) ([Amos Bird](https://github.com/amosbird)) -## ClickHouse release 19.14.3.3, 2019-09-10 +### ClickHouse release 19.14.3.3, 2019-09-10 -### New Feature +#### New Feature * `WITH FILL` modifier for `ORDER BY`. (continuation of [#5069](https://github.com/ClickHouse/ClickHouse/issues/5069)) [#6610](https://github.com/ClickHouse/ClickHouse/pull/6610) ([Anton Popov](https://github.com/CurtizJ)) * `WITH TIES` modifier for `LIMIT`. (continuation of [#5069](https://github.com/ClickHouse/ClickHouse/issues/5069)) [#6610](https://github.com/ClickHouse/ClickHouse/pull/6610) ([Anton Popov](https://github.com/CurtizJ)) * Parse unquoted `NULL` literal as NULL (if setting `format_csv_unquoted_null_literal_as_null=1`). Initialize null fields with default values if data type of this field is not nullable (if setting `input_format_null_as_default=1`). [#5990](https://github.com/ClickHouse/ClickHouse/issues/5990) [#6055](https://github.com/ClickHouse/ClickHouse/pull/6055) ([tavplubix](https://github.com/tavplubix)) @@ -498,11 +503,11 @@ fix comments to make obvious that it may throw. * Added support for `_partition` and `_timestamp` virtual columns to Kafka engine. [#6400](https://github.com/ClickHouse/ClickHouse/pull/6400) ([Ivan](https://github.com/abyss7)) * Possibility to remove sensitive data from `query_log`, server logs, process list with regexp-based rules. [#5710](https://github.com/ClickHouse/ClickHouse/pull/5710) ([filimonov](https://github.com/filimonov)) -### Experimental Feature +#### Experimental Feature * Input and output data format `Template`. It allows to specify custom format string for input and output. [#4354](https://github.com/ClickHouse/ClickHouse/issues/4354) [#6727](https://github.com/ClickHouse/ClickHouse/pull/6727) ([tavplubix](https://github.com/tavplubix)) * Implementation of `LIVE VIEW` tables that were originally proposed in [#2898](https://github.com/ClickHouse/ClickHouse/pull/2898), prepared in [#3925](https://github.com/ClickHouse/ClickHouse/issues/3925), and then updated in [#5541](https://github.com/ClickHouse/ClickHouse/issues/5541). See [#5541](https://github.com/ClickHouse/ClickHouse/issues/5541) for detailed description. [#5541](https://github.com/ClickHouse/ClickHouse/issues/5541) ([vzakaznikov](https://github.com/vzakaznikov)) [#6425](https://github.com/ClickHouse/ClickHouse/pull/6425) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [#6656](https://github.com/ClickHouse/ClickHouse/pull/6656) ([vzakaznikov](https://github.com/vzakaznikov)) Note that `LIVE VIEW` feature may be removed in next versions. -### Bug Fix +#### Bug Fix * This release also contains all bug fixes from 19.13 and 19.11. * Fix segmentation fault when the table has skip indices and vertical merge happens. [#6723](https://github.com/ClickHouse/ClickHouse/pull/6723) ([alesapin](https://github.com/alesapin)) * Fix per-column TTL with non-trivial column defaults. Previously in case of force TTL merge with `OPTIMIZE ... FINAL` query, expired values was replaced by type defaults instead of user-specified column defaults. [#6796](https://github.com/ClickHouse/ClickHouse/pull/6796) ([Anton Popov](https://github.com/CurtizJ)) @@ -540,7 +545,7 @@ fix comments to make obvious that it may throw. * Fix bug with writing secondary indices marks with adaptive granularity. [#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alesapin](https://github.com/alesapin)) * Fix initialization order while server startup. Since `StorageMergeTree::background_task_handle` is initialized in `startup()` the `MergeTreeBlockOutputStream::write()` may try to use it before initialization. Just check if it is initialized. [#6080](https://github.com/ClickHouse/ClickHouse/pull/6080) ([Ivan](https://github.com/abyss7)) * Clearing the data buffer from the previous read operation that was completed with an error. [#6026](https://github.com/ClickHouse/ClickHouse/pull/6026) ([Nikolay](https://github.com/bopohaa)) -* Fix bug with enabling adaptive granularity when creating a new replica for Replicated*MergeTree table. [#6394](https://github.com/ClickHouse/ClickHouse/issues/6394) [#6452](https://github.com/ClickHouse/ClickHouse/pull/6452) ([alesapin](https://github.com/alesapin)) +* Fix bug with enabling adaptive granularity when creating a new replica for Replicated\*MergeTree table. [#6394](https://github.com/ClickHouse/ClickHouse/issues/6394) [#6452](https://github.com/ClickHouse/ClickHouse/pull/6452) ([alesapin](https://github.com/alesapin)) * Fixed possible crash during server startup in case of exception happened in `libunwind` during exception at access to uninitialized `ThreadStatus` structure. [#6456](https://github.com/ClickHouse/ClickHouse/pull/6456) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) * Fix crash in `yandexConsistentHash` function. Found by fuzz test. [#6304](https://github.com/ClickHouse/ClickHouse/issues/6304) [#6305](https://github.com/ClickHouse/ClickHouse/pull/6305) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed the possibility of hanging queries when server is overloaded and global thread pool becomes near full. This have higher chance to happen on clusters with large number of shards (hundreds), because distributed queries allocate a thread per connection to each shard. For example, this issue may reproduce if a cluster of 330 shards is processing 30 concurrent distributed queries. This issue affects all versions starting from 19.2. [#6301](https://github.com/ClickHouse/ClickHouse/pull/6301) ([alexey-milovidov](https://github.com/alexey-milovidov)) @@ -560,11 +565,11 @@ fix comments to make obvious that it may throw. * Typo in the error message ( is -> are ). [#6839](https://github.com/ClickHouse/ClickHouse/pull/6839) ([Denis Zhuravlev](https://github.com/den-crane)) * Fixed error while parsing of columns list from string if type contained a comma (this issue was relevant for `File`, `URL`, `HDFS` storages) [#6217](https://github.com/ClickHouse/ClickHouse/issues/6217). [#6209](https://github.com/ClickHouse/ClickHouse/pull/6209) ([dimarub2000](https://github.com/dimarub2000)) -### Security Fix +#### Security Fix * This release also contains all bug security fixes from 19.13 and 19.11. * Fixed the possibility of a fabricated query to cause server crash due to stack overflow in SQL parser. Fixed the possibility of stack overflow in Merge and Distributed tables, materialized views and conditions for row-level security that involve subqueries. [#6433](https://github.com/ClickHouse/ClickHouse/pull/6433) ([alexey-milovidov](https://github.com/alexey-milovidov)) -### Improvement +#### Improvement * Correct implementation of ternary logic for `AND/OR`. [#6048](https://github.com/ClickHouse/ClickHouse/pull/6048) ([Alexander Kazakov](https://github.com/Akazz)) * Now values and rows with expired TTL will be removed after `OPTIMIZE ... FINAL` query from old parts without TTL infos or with outdated TTL infos, e.g. after `ALTER ... MODIFY TTL` query. Added queries `SYSTEM STOP/START TTL MERGES` to disallow/allow assign merges with TTL and filter expired values in all merges. [#6274](https://github.com/ClickHouse/ClickHouse/pull/6274) ([Anton Popov](https://github.com/CurtizJ)) * Possibility to change the location of ClickHouse history file for client using `CLICKHOUSE_HISTORY_FILE` env. [#6840](https://github.com/ClickHouse/ClickHouse/pull/6840) ([filimonov](https://github.com/filimonov)) @@ -625,7 +630,7 @@ fix comments to make obvious that it may throw. * `MergeTree` now has an additional option `ttl_only_drop_parts` (disabled by default) to avoid partial pruning of parts, so that they dropped completely when all the rows in a part are expired. [#6191](https://github.com/ClickHouse/ClickHouse/pull/6191) ([Sergi Vladykin](https://github.com/svladykin)) * Type checks for set index functions. Throw exception if function got a wrong type. This fixes fuzz test with UBSan. [#6511](https://github.com/ClickHouse/ClickHouse/pull/6511) ([Nikita Vasilev](https://github.com/nikvas0)) -### Performance Improvement +#### Performance Improvement * Optimize queries with `ORDER BY expressions` clause, where `expressions` have coinciding prefix with sorting key in `MergeTree` tables. This optimization is controlled by `optimize_read_in_order` setting. [#6054](https://github.com/ClickHouse/ClickHouse/pull/6054) [#6629](https://github.com/ClickHouse/ClickHouse/pull/6629) ([Anton Popov](https://github.com/CurtizJ)) * Allow to use multiple threads during parts loading and removal. [#6372](https://github.com/ClickHouse/ClickHouse/issues/6372) [#6074](https://github.com/ClickHouse/ClickHouse/issues/6074) [#6438](https://github.com/ClickHouse/ClickHouse/pull/6438) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Implemented batch variant of updating aggregate function states. It may lead to performance benefits. [#6435](https://github.com/ClickHouse/ClickHouse/pull/6435) ([alexey-milovidov](https://github.com/alexey-milovidov)) @@ -635,7 +640,7 @@ fix comments to make obvious that it may throw. * Pre-fault pages when allocating memory with `mmap()`. [#6667](https://github.com/ClickHouse/ClickHouse/pull/6667) ([akuzm](https://github.com/akuzm)) * Fix performance bug in `Decimal` comparison. [#6380](https://github.com/ClickHouse/ClickHouse/pull/6380) ([Artem Zuikov](https://github.com/4ertus2)) -### Build/Testing/Packaging Improvement +#### Build/Testing/Packaging Improvement * Remove Compiler (runtime template instantiation) because we've win over it's performance. [#6646](https://github.com/ClickHouse/ClickHouse/pull/6646) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Added performance test to show degradation of performance in gcc-9 in more isolated way. [#6302](https://github.com/ClickHouse/ClickHouse/pull/6302) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Added table function `numbers_mt`, which is multithreaded version of `numbers`. Updated performance tests with hash functions. [#6554](https://github.com/ClickHouse/ClickHouse/pull/6554) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) @@ -712,19 +717,19 @@ fix comments to make obvious that it may throw. * Fix "splitted" build. [#6618](https://github.com/ClickHouse/ClickHouse/pull/6618) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Other build fixes: [#6186](https://github.com/ClickHouse/ClickHouse/pull/6186) ([Amos Bird](https://github.com/amosbird)) [#6486](https://github.com/ClickHouse/ClickHouse/pull/6486) [#6348](https://github.com/ClickHouse/ClickHouse/pull/6348) ([vxider](https://github.com/Vxider)) [#6744](https://github.com/ClickHouse/ClickHouse/pull/6744) ([Ivan](https://github.com/abyss7)) [#6016](https://github.com/ClickHouse/ClickHouse/pull/6016) [#6421](https://github.com/ClickHouse/ClickHouse/pull/6421) [#6491](https://github.com/ClickHouse/ClickHouse/pull/6491) ([proller](https://github.com/proller)) -### Backward Incompatible Change +#### Backward Incompatible Change * Removed rarely used table function `catBoostPool` and storage `CatBoostPool`. If you have used this table function, please write email to `clickhouse-feedback@yandex-team.com`. Note that CatBoost integration remains and will be supported. [#6279](https://github.com/ClickHouse/ClickHouse/pull/6279) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Disable `ANY RIGHT JOIN` and `ANY FULL JOIN` by default. Set `any_join_distinct_right_table_keys` setting to enable them. [#5126](https://github.com/ClickHouse/ClickHouse/issues/5126) [#6351](https://github.com/ClickHouse/ClickHouse/pull/6351) ([Artem Zuikov](https://github.com/4ertus2)) -## ClickHouse release 19.13.6.51, 2019-10-02 +## ClickHouse release 19.13 +### ClickHouse release 19.13.6.51, 2019-10-02 -### Bug Fix +#### Bug Fix * This release also contains all bug fixes from 19.11.12.69. +### ClickHouse release 19.13.5.44, 2019-09-20 -## ClickHouse release 19.13.5.44, 2019-09-20 - -### Bug Fix +#### Bug Fix * This release also contains all bug fixes from 19.14.6.12. * Fixed possible inconsistent state of table while executing `DROP` query for replicated table while zookeeper is not accessible. [#6045](https://github.com/ClickHouse/ClickHouse/issues/6045) [#6413](https://github.com/ClickHouse/ClickHouse/pull/6413) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) * Fix for data race in StorageMerge [#6717](https://github.com/ClickHouse/ClickHouse/pull/6717) ([alexey-milovidov](https://github.com/alexey-milovidov)) @@ -736,9 +741,9 @@ fix comments to make obvious that it may throw. * Fixed parsing of `AggregateFunction` values embedded in query. [#6575](https://github.com/ClickHouse/ClickHouse/issues/6575) [#6773](https://github.com/ClickHouse/ClickHouse/pull/6773) ([Zhichang Yu](https://github.com/yuzhichang)) * Fixed wrong behaviour of `trim` functions family. [#6647](https://github.com/ClickHouse/ClickHouse/pull/6647) ([alexey-milovidov](https://github.com/alexey-milovidov)) -## ClickHouse release 19.13.4.32, 2019-09-10 +### ClickHouse release 19.13.4.32, 2019-09-10 -### Bug Fix +#### Bug Fix * This release also contains all bug security fixes from 19.11.9.52 and 19.11.10.54. * Fixed data race in `system.parts` table and `ALTER` query. [#6245](https://github.com/ClickHouse/ClickHouse/issues/6245) [#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed mismatched header in streams happened in case of reading from empty distributed table with sample and prewhere. [#6167](https://github.com/ClickHouse/ClickHouse/issues/6167) ([Lixiang Qian](https://github.com/fancyqlx)) [#6823](https://github.com/ClickHouse/ClickHouse/pull/6823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) @@ -749,29 +754,83 @@ fix comments to make obvious that it may throw. * Query transformation for `MySQL`, `ODBC`, `JDBC` table functions now works properly for `SELECT WHERE` queries with multiple `AND` expressions. [#6381](https://github.com/ClickHouse/ClickHouse/issues/6381) [#6676](https://github.com/ClickHouse/ClickHouse/pull/6676) ([dimarub2000](https://github.com/dimarub2000)) * Added previous declaration checks for MySQL 8 integration. [#6569](https://github.com/ClickHouse/ClickHouse/pull/6569) ([Rafael David Tinoco](https://github.com/rafaeldtinoco)) -### Security Fix +#### Security Fix * Fix two vulnerabilities in codecs in decompression phase (malicious user can fabricate compressed data that will lead to buffer overflow in decompression). [#6670](https://github.com/ClickHouse/ClickHouse/pull/6670) ([Artem Zuikov](https://github.com/4ertus2)) -## ClickHouse release 19.11.13.74, 2019-11-01 -### Bug Fix +### ClickHouse release 19.13.3.26, 2019-08-22 + +#### Bug Fix +* Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) +* Fix NPE when using IN clause with a subquery with a tuple. [#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) +* Fixed an issue that if a stale replica becomes alive, it may still have data parts that were removed by DROP PARTITION. [#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) +* Fixed issue with parsing CSV [#6426](https://github.com/ClickHouse/ClickHouse/issues/6426) [#6559](https://github.com/ClickHouse/ClickHouse/pull/6559) ([tavplubix](https://github.com/tavplubix)) +* Fixed data race in system.parts table and ALTER query. This fixes [#6245](https://github.com/ClickHouse/ClickHouse/issues/6245). [#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed possible data loss after `ALTER DELETE` query on table with skipping index. [#6224](https://github.com/ClickHouse/ClickHouse/issues/6224) [#6282](https://github.com/ClickHouse/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) + +#### Security Fix +* If the attacker has write access to ZooKeeper and is able to run custom server available from the network where ClickHouse run, it can create custom-built malicious server that will act as ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. Found by Eldar Zaitov, information security team at Yandex. [#6247](https://github.com/ClickHouse/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### ClickHouse release 19.13.2.19, 2019-08-14 + +#### New Feature +* Sampling profiler on query level. [Example](https://gist.github.com/alexey-milovidov/92758583dd41c24c360fdb8d6a4da194). [#4247](https://github.com/ClickHouse/ClickHouse/issues/4247) ([laplab](https://github.com/laplab)) [#6124](https://github.com/ClickHouse/ClickHouse/pull/6124) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#6250](https://github.com/ClickHouse/ClickHouse/pull/6250) [#6283](https://github.com/ClickHouse/ClickHouse/pull/6283) [#6386](https://github.com/ClickHouse/ClickHouse/pull/6386) +* Allow to specify a list of columns with `COLUMNS('regexp')` expression that works like a more sophisticated variant of `*` asterisk. [#5951](https://github.com/ClickHouse/ClickHouse/pull/5951) ([mfridental](https://github.com/mfridental)), ([alexey-milovidov](https://github.com/alexey-milovidov)) +* `CREATE TABLE AS table_function()` is now possible [#6057](https://github.com/ClickHouse/ClickHouse/pull/6057) ([dimarub2000](https://github.com/dimarub2000)) +* Adam optimizer for stochastic gradient descent is used by default in `stochasticLinearRegression()` and `stochasticLogisticRegression()` aggregate functions, because it shows good quality without almost any tuning. [#6000](https://github.com/ClickHouse/ClickHouse/pull/6000) ([Quid37](https://github.com/Quid37)) +* Added functions for working with the сustom week number [#5212](https://github.com/ClickHouse/ClickHouse/pull/5212) ([Andy Yang](https://github.com/andyyzh)) +* `RENAME` queries now work with all storages. [#5953](https://github.com/ClickHouse/ClickHouse/pull/5953) ([Ivan](https://github.com/abyss7)) +* Now client receive logs from server with any desired level by setting `send_logs_level` regardless to the log level specified in server settings. [#5964](https://github.com/ClickHouse/ClickHouse/pull/5964) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) + +#### Backward Incompatible Change +* The setting `input_format_defaults_for_omitted_fields` is enabled by default. Inserts in Distributed tables need this setting to be the same on cluster (you need to set it before rolling update). It enables calculation of complex default expressions for omitted fields in `JSONEachRow` and `CSV*` formats. It should be the expected behavior but may lead to negligible performance difference. [#6043](https://github.com/ClickHouse/ClickHouse/pull/6043) ([Artem Zuikov](https://github.com/4ertus2)), [#5625](https://github.com/ClickHouse/ClickHouse/pull/5625) ([akuzm](https://github.com/akuzm)) + +#### Experimental features +* New query processing pipeline. Use `experimental_use_processors=1` option to enable it. Use for your own trouble. [#4914](https://github.com/ClickHouse/ClickHouse/pull/4914) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + +#### Bug Fix +* Kafka integration has been fixed in this version. +* Fixed `DoubleDelta` encoding of `Int64` for large `DoubleDelta` values, improved `DoubleDelta` encoding for random data for `Int32`. [#5998](https://github.com/ClickHouse/ClickHouse/pull/5998) ([Vasily Nemkov](https://github.com/Enmk)) +* Fixed overestimation of `max_rows_to_read` if the setting `merge_tree_uniform_read_distribution` is set to 0. [#6019](https://github.com/ClickHouse/ClickHouse/pull/6019) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Improvement +* Throws an exception if `config.d` file doesn't have the corresponding root element as the config file [#6123](https://github.com/ClickHouse/ClickHouse/pull/6123) ([dimarub2000](https://github.com/dimarub2000)) + +#### Performance Improvement +* Optimize `count()`. Now it uses the smallest column (if possible). [#6028](https://github.com/ClickHouse/ClickHouse/pull/6028) ([Amos Bird](https://github.com/amosbird)) + +#### Build/Testing/Packaging Improvement +* Report memory usage in performance tests. [#5899](https://github.com/ClickHouse/ClickHouse/pull/5899) ([akuzm](https://github.com/akuzm)) +* Fix build with external `libcxx` [#6010](https://github.com/ClickHouse/ClickHouse/pull/6010) ([Ivan](https://github.com/abyss7)) +* Fix shared build with `rdkafka` library [#6101](https://github.com/ClickHouse/ClickHouse/pull/6101) ([Ivan](https://github.com/abyss7)) + +## ClickHouse release 19.11 + +### ClickHouse release 19.11.13.74, 2019-11-01 + +#### Bug Fix * Fixed rare crash in `ALTER MODIFY COLUMN` and vertical merge when one of merged/altered parts is empty (0 rows). [#6780](https://github.com/ClickHouse/ClickHouse/pull/6780) ([alesapin](https://github.com/alesapin)) * Manual update of `SIMDJSON`. This fixes possible flooding of stderr files with bogus json diagnostic messages. [#7548](https://github.com/ClickHouse/ClickHouse/pull/7548) ([Alexander Kazakov](https://github.com/Akazz)) * Fixed bug with `mrk` file extension for mutations ([alesapin](https://github.com/alesapin)) -## ClickHouse release 19.11.12.69, 2019-10-02 +### ClickHouse release 19.11.12.69, 2019-10-02 -### Bug Fix +#### Bug Fix * Fixed performance degradation of index analysis on complex keys on large tables. This fixes [#6924](https://github.com/ClickHouse/ClickHouse/issues/6924). [#7075](https://github.com/ClickHouse/ClickHouse/pull/7075) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Avoid rare SIGSEGV while sending data in tables with Distributed engine (`Failed to send batch: file with index XXXXX is absent`). [#7032](https://github.com/ClickHouse/ClickHouse/pull/7032) ([Azat Khuzhin](https://github.com/azat)) * Fix `Unknown identifier` with multiple joins. This fixes [#5254](https://github.com/ClickHouse/ClickHouse/issues/5254). [#7022](https://github.com/ClickHouse/ClickHouse/pull/7022) ([Artem Zuikov](https://github.com/4ertus2)) -## ClickHouse release 19.11.10.54, 2019-09-10 +### ClickHouse release 19.11.11.57, 2019-09-13 +* Fix logical error causing segfaults when selecting from Kafka empty topic. [#6902](https://github.com/ClickHouse/ClickHouse/issues/6902) [#6909](https://github.com/ClickHouse/ClickHouse/pull/6909) ([Ivan](https://github.com/abyss7)) +* Fix for function `АrrayEnumerateUniqRanked` with empty arrays in params. [#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([proller](https://github.com/proller)) -### Bug Fix +### ClickHouse release 19.11.10.54, 2019-09-10 + +#### Bug Fix * Do store offsets for Kafka messages manually to be able to commit them all at once for all partitions. Fixes potential duplication in "one consumer - many partitions" scenario. [#6872](https://github.com/ClickHouse/ClickHouse/pull/6872) ([Ivan](https://github.com/abyss7)) -## ClickHouse release 19.11.9.52, 2019-09-6 +### ClickHouse release 19.11.9.52, 2019-09-6 * Improve error handling in cache dictionaries. [#6737](https://github.com/ClickHouse/ClickHouse/pull/6737) ([Vitaly Baranov](https://github.com/vitlibar)) * Fixed bug in function `arrayEnumerateUniqRanked`. [#6779](https://github.com/ClickHouse/ClickHouse/pull/6779) ([proller](https://github.com/proller)) * Fix `JSONExtract` function while extracting a `Tuple` from JSON. [#6718](https://github.com/ClickHouse/ClickHouse/pull/6718) ([Vitaly Baranov](https://github.com/vitlibar)) @@ -784,63 +843,12 @@ fix comments to make obvious that it may throw. * Fixed error with processing "timezone" in server configuration file. [#6709](https://github.com/ClickHouse/ClickHouse/pull/6709) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix kafka tests. [#6805](https://github.com/ClickHouse/ClickHouse/pull/6805) ([Ivan](https://github.com/abyss7)) -### Security Fix +#### Security Fix * If the attacker has write access to ZooKeeper and is able to run custom server available from the network where ClickHouse runs, it can create custom-built malicious server that will act as ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. Found by Eldar Zaitov, information security team at Yandex. [#6247](https://github.com/ClickHouse/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) -## ClickHouse release 19.13.3.26, 2019-08-22 +### ClickHouse release 19.11.8.46, 2019-08-22 -### Bug Fix -* Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) -* Fix NPE when using IN clause with a subquery with a tuple. [#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) -* Fixed an issue that if a stale replica becomes alive, it may still have data parts that were removed by DROP PARTITION. [#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) -* Fixed issue with parsing CSV [#6426](https://github.com/ClickHouse/ClickHouse/issues/6426) [#6559](https://github.com/ClickHouse/ClickHouse/pull/6559) ([tavplubix](https://github.com/tavplubix)) -* Fixed data race in system.parts table and ALTER query. This fixes [#6245](https://github.com/ClickHouse/ClickHouse/issues/6245). [#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed possible data loss after `ALTER DELETE` query on table with skipping index. [#6224](https://github.com/ClickHouse/ClickHouse/issues/6224) [#6282](https://github.com/ClickHouse/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) - -### Security Fix -* If the attacker has write access to ZooKeeper and is able to run custom server available from the network where ClickHouse run, it can create custom-built malicious server that will act as ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. Found by Eldar Zaitov, information security team at Yandex. [#6247](https://github.com/ClickHouse/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -## ClickHouse release 19.13.2.19, 2019-08-14 - -### New Feature -* Sampling profiler on query level. [Example](https://gist.github.com/alexey-milovidov/92758583dd41c24c360fdb8d6a4da194). [#4247](https://github.com/ClickHouse/ClickHouse/issues/4247) ([laplab](https://github.com/laplab)) [#6124](https://github.com/ClickHouse/ClickHouse/pull/6124) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#6250](https://github.com/ClickHouse/ClickHouse/pull/6250) [#6283](https://github.com/ClickHouse/ClickHouse/pull/6283) [#6386](https://github.com/ClickHouse/ClickHouse/pull/6386) -* Allow to specify a list of columns with `COLUMNS('regexp')` expression that works like a more sophisticated variant of `*` asterisk. [#5951](https://github.com/ClickHouse/ClickHouse/pull/5951) ([mfridental](https://github.com/mfridental)), ([alexey-milovidov](https://github.com/alexey-milovidov)) -* `CREATE TABLE AS table_function()` is now possible [#6057](https://github.com/ClickHouse/ClickHouse/pull/6057) ([dimarub2000](https://github.com/dimarub2000)) -* Adam optimizer for stochastic gradient descent is used by default in `stochasticLinearRegression()` and `stochasticLogisticRegression()` aggregate functions, because it shows good quality without almost any tuning. [#6000](https://github.com/ClickHouse/ClickHouse/pull/6000) ([Quid37](https://github.com/Quid37)) -* Added functions for working with the сustom week number [#5212](https://github.com/ClickHouse/ClickHouse/pull/5212) ([Andy Yang](https://github.com/andyyzh)) -* `RENAME` queries now work with all storages. [#5953](https://github.com/ClickHouse/ClickHouse/pull/5953) ([Ivan](https://github.com/abyss7)) -* Now client receive logs from server with any desired level by setting `send_logs_level` regardless to the log level specified in server settings. [#5964](https://github.com/ClickHouse/ClickHouse/pull/5964) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) - -### Backward Incompatible Change -* The setting `input_format_defaults_for_omitted_fields` is enabled by default. Inserts in Distributed tables need this setting to be the same on cluster (you need to set it before rolling update). It enables calculation of complex default expressions for omitted fields in `JSONEachRow` and `CSV*` formats. It should be the expected behavior but may lead to negligible performance difference. [#6043](https://github.com/ClickHouse/ClickHouse/pull/6043) ([Artem Zuikov](https://github.com/4ertus2)), [#5625](https://github.com/ClickHouse/ClickHouse/pull/5625) ([akuzm](https://github.com/akuzm)) - -### Experimental features -* New query processing pipeline. Use `experimental_use_processors=1` option to enable it. Use for your own trouble. [#4914](https://github.com/ClickHouse/ClickHouse/pull/4914) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - -### Bug Fix -* Kafka integration has been fixed in this version. -* Fixed `DoubleDelta` encoding of `Int64` for large `DoubleDelta` values, improved `DoubleDelta` encoding for random data for `Int32`. [#5998](https://github.com/ClickHouse/ClickHouse/pull/5998) ([Vasily Nemkov](https://github.com/Enmk)) -* Fixed overestimation of `max_rows_to_read` if the setting `merge_tree_uniform_read_distribution` is set to 0. [#6019](https://github.com/ClickHouse/ClickHouse/pull/6019) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -### Improvement -* Throws an exception if `config.d` file doesn't have the corresponding root element as the config file [#6123](https://github.com/ClickHouse/ClickHouse/pull/6123) ([dimarub2000](https://github.com/dimarub2000)) - -### Performance Improvement -* Optimize `count()`. Now it uses the smallest column (if possible). [#6028](https://github.com/ClickHouse/ClickHouse/pull/6028) ([Amos Bird](https://github.com/amosbird)) - -### Build/Testing/Packaging Improvement -* Report memory usage in performance tests. [#5899](https://github.com/ClickHouse/ClickHouse/pull/5899) ([akuzm](https://github.com/akuzm)) -* Fix build with external `libcxx` [#6010](https://github.com/ClickHouse/ClickHouse/pull/6010) ([Ivan](https://github.com/abyss7)) -* Fix shared build with `rdkafka` library [#6101](https://github.com/ClickHouse/ClickHouse/pull/6101) ([Ivan](https://github.com/abyss7)) - -## ClickHouse release 19.11.11.57, 2019-09-13 -* Fix logical error causing segfaults when selecting from Kafka empty topic. [#6902](https://github.com/ClickHouse/ClickHouse/issues/6902) [#6909](https://github.com/ClickHouse/ClickHouse/pull/6909) ([Ivan](https://github.com/abyss7)) -* Fix for function `АrrayEnumerateUniqRanked` with empty arrays in params. [#6928](https://github.com/ClickHouse/ClickHouse/pull/6928) ([proller](https://github.com/proller)) - -## ClickHouse release 19.11.8.46, 2019-08-22 - -### Bug Fix +#### Bug Fix * Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [#6543](https://github.com/ClickHouse/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) * Fix NPE when using IN clause with a subquery with a tuple. [#6125](https://github.com/ClickHouse/ClickHouse/issues/6125) [#6550](https://github.com/ClickHouse/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) * Fixed an issue that if a stale replica becomes alive, it may still have data parts that were removed by DROP PARTITION. [#6522](https://github.com/ClickHouse/ClickHouse/issues/6522) [#6523](https://github.com/ClickHouse/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) @@ -848,9 +856,9 @@ fix comments to make obvious that it may throw. * Fixed data race in system.parts table and ALTER query. This fixes [#6245](https://github.com/ClickHouse/ClickHouse/issues/6245). [#6513](https://github.com/ClickHouse/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [#6514](https://github.com/ClickHouse/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) -## ClickHouse release 19.11.7.40, 2019-08-14 +### ClickHouse release 19.11.7.40, 2019-08-14 -### Bug fix +#### Bug fix * Kafka integration has been fixed in this version. * Fix segfault when using `arrayReduce` for constant arguments. [#6326](https://github.com/ClickHouse/ClickHouse/pull/6326) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed `toFloat()` monotonicity. [#6374](https://github.com/ClickHouse/ClickHouse/pull/6374) ([dimarub2000](https://github.com/dimarub2000)) @@ -865,12 +873,12 @@ fix comments to make obvious that it may throw. * Fixed the possibility of a fabricated query to cause server crash due to stack overflow in SQL parser and possibility of stack overflow in `Merge` and `Distributed` tables [#6433](https://github.com/ClickHouse/ClickHouse/pull/6433) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed Gorilla encoding error on small sequences. [#6444](https://github.com/ClickHouse/ClickHouse/pull/6444) ([Enmk](https://github.com/Enmk)) -### Improvement +#### Improvement * Allow user to override `poll_interval` and `idle_connection_timeout` settings on connection. [#6230](https://github.com/ClickHouse/ClickHouse/pull/6230) ([alexey-milovidov](https://github.com/alexey-milovidov)) -## ClickHouse release 19.11.5.28, 2019-08-05 +### ClickHouse release 19.11.5.28, 2019-08-05 -### Bug fix +#### Bug fix * Fixed the possibility of hanging queries when server is overloaded. [#6301](https://github.com/ClickHouse/ClickHouse/pull/6301) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix FPE in yandexConsistentHash function. This fixes [#6304](https://github.com/ClickHouse/ClickHouse/issues/6304). [#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed bug in conversion of `LowCardinality` types in `AggregateFunctionFactory`. This fixes [#6257](https://github.com/ClickHouse/ClickHouse/issues/6257). [#6281](https://github.com/ClickHouse/ClickHouse/pull/6281) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) @@ -878,12 +886,12 @@ fix comments to make obvious that it may throw. * Fix rare bug with incompatible stream headers in queries to `Distributed` table over `MergeTree` table when part of `WHERE` moves to `PREWHERE`. [#6236](https://github.com/ClickHouse/ClickHouse/pull/6236) ([alesapin](https://github.com/alesapin)) * Fixed overflow in integer division of signed type to unsigned type. This fixes [#6214](https://github.com/ClickHouse/ClickHouse/issues/6214). [#6233](https://github.com/ClickHouse/ClickHouse/pull/6233) ([alexey-milovidov](https://github.com/alexey-milovidov)) -### Backward Incompatible Change +#### Backward Incompatible Change * `Kafka` still broken. -## ClickHouse release 19.11.4.24, 2019-08-01 +### ClickHouse release 19.11.4.24, 2019-08-01 -### Bug Fix +#### Bug Fix * Fix bug with writing secondary indices marks with adaptive granularity. [#6126](https://github.com/ClickHouse/ClickHouse/pull/6126) ([alesapin](https://github.com/alesapin)) * Fix `WITH ROLLUP` and `WITH CUBE` modifiers of `GROUP BY` with two-level aggregation. [#6225](https://github.com/ClickHouse/ClickHouse/pull/6225) ([Anton Popov](https://github.com/CurtizJ)) * Fixed hang in `JSONExtractRaw` function. Fixed [#6195](https://github.com/ClickHouse/ClickHouse/issues/6195) [#6198](https://github.com/ClickHouse/ClickHouse/pull/6198) ([alexey-milovidov](https://github.com/alexey-milovidov)) @@ -898,18 +906,18 @@ fix comments to make obvious that it may throw. * Clearing the Kafka data buffer from the previous read operation that was completed with an error [#6026](https://github.com/ClickHouse/ClickHouse/pull/6026) ([Nikolay](https://github.com/bopohaa)) Note that Kafka is broken in this version. * Since `StorageMergeTree::background_task_handle` is initialized in `startup()` the `MergeTreeBlockOutputStream::write()` may try to use it before initialization. Just check if it is initialized. [#6080](https://github.com/ClickHouse/ClickHouse/pull/6080) ([Ivan](https://github.com/abyss7)) -### Build/Testing/Packaging Improvement +#### Build/Testing/Packaging Improvement * Added official `rpm` packages. [#5740](https://github.com/ClickHouse/ClickHouse/pull/5740) ([proller](https://github.com/proller)) ([alesapin](https://github.com/alesapin)) * Add an ability to build `.rpm` and `.tgz` packages with `packager` script. [#5769](https://github.com/ClickHouse/ClickHouse/pull/5769) ([alesapin](https://github.com/alesapin)) * Fixes for "Arcadia" build system. [#6223](https://github.com/ClickHouse/ClickHouse/pull/6223) ([proller](https://github.com/proller)) -### Backward Incompatible Change +#### Backward Incompatible Change * `Kafka` is broken in this version. -## ClickHouse release 19.11.3.11, 2019-07-18 +### ClickHouse release 19.11.3.11, 2019-07-18 -### New Feature +#### New Feature * Added support for prepared statements. [#5331](https://github.com/ClickHouse/ClickHouse/pull/5331/) ([Alexander](https://github.com/sanych73)) [#5630](https://github.com/ClickHouse/ClickHouse/pull/5630) ([alexey-milovidov](https://github.com/alexey-milovidov)) * `DoubleDelta` and `Gorilla` column codecs [#5600](https://github.com/ClickHouse/ClickHouse/pull/5600) ([Vasily Nemkov](https://github.com/Enmk)) * Added `os_thread_priority` setting that allows to control the "nice" value of query processing threads that is used by OS to adjust dynamic scheduling priority. It requires `CAP_SYS_NICE` capabilities to work. This implements [#5858](https://github.com/ClickHouse/ClickHouse/issues/5858) [#5909](https://github.com/ClickHouse/ClickHouse/pull/5909) ([alexey-milovidov](https://github.com/alexey-milovidov)) @@ -919,7 +927,7 @@ fix comments to make obvious that it may throw. * Add synonim `arrayFlatten` <-> `flatten` [#5764](https://github.com/ClickHouse/ClickHouse/pull/5764) ([hcz](https://github.com/hczhcz)) * Intergate H3 function `geoToH3` from Uber. [#4724](https://github.com/ClickHouse/ClickHouse/pull/4724) ([Remen Ivan](https://github.com/BHYCHIK)) [#5805](https://github.com/ClickHouse/ClickHouse/pull/5805) ([alexey-milovidov](https://github.com/alexey-milovidov)) -### Bug Fix +#### Bug Fix * Implement DNS cache with asynchronous update. Separate thread resolves all hosts and updates DNS cache with period (setting `dns_cache_update_period`). It should help, when ip of hosts changes frequently. [#5857](https://github.com/ClickHouse/ClickHouse/pull/5857) ([Anton Popov](https://github.com/CurtizJ)) * Fix segfault in `Delta` codec which affects columns with values less than 32 bits size. The bug led to random memory corruption. [#5786](https://github.com/ClickHouse/ClickHouse/pull/5786) ([alesapin](https://github.com/alesapin)) * Fix segfault in TTL merge with non-physical columns in block. [#5819](https://github.com/ClickHouse/ClickHouse/pull/5819) ([Anton Popov](https://github.com/CurtizJ)) @@ -946,7 +954,7 @@ fix comments to make obvious that it may throw. * Fix shutdown of SystemLogs [#5802](https://github.com/ClickHouse/ClickHouse/pull/5802) ([Anton Popov](https://github.com/CurtizJ)) * Fix hanging when condition in invalidate_query depends on a dictionary. [#6011](https://github.com/ClickHouse/ClickHouse/pull/6011) ([Vitaly Baranov](https://github.com/vitlibar)) -### Improvement +#### Improvement * Allow unresolvable addresses in cluster configuration. They will be considered unavailable and tried to resolve at every connection attempt. This is especially useful for Kubernetes. This fixes [#5714](https://github.com/ClickHouse/ClickHouse/issues/5714) [#5924](https://github.com/ClickHouse/ClickHouse/pull/5924) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Close idle TCP connections (with one hour timeout by default). This is especially important for large clusters with multiple distributed tables on every server, because every server can possibly keep a connection pool to every other server, and after peak query concurrency, connections will stall. This fixes [#5879](https://github.com/ClickHouse/ClickHouse/issues/5879) [#5880](https://github.com/ClickHouse/ClickHouse/pull/5880) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Better quality of `topK` function. Changed the SavingSpace set behavior to remove the last element if the new element have a bigger weight. [#5833](https://github.com/ClickHouse/ClickHouse/issues/5833) [#5850](https://github.com/ClickHouse/ClickHouse/pull/5850) ([Guillaume Tassery](https://github.com/YiuRULE)) @@ -967,10 +975,10 @@ fix comments to make obvious that it may throw. * Update default value of `max_ast_elements parameter` [#5933](https://github.com/ClickHouse/ClickHouse/pull/5933) ([Artem Konovalov](https://github.com/izebit)) * Added a notion of obsolete settings. The obsolete setting `allow_experimental_low_cardinality_type` can be used with no effect. [0f15c01c6802f7ce1a1494c12c846be8c98944cd](https://github.com/ClickHouse/ClickHouse/commit/0f15c01c6802f7ce1a1494c12c846be8c98944cd) [Alexey Milovidov](https://github.com/alexey-milovidov) -### Performance Improvement +#### Performance Improvement * Increase number of streams to SELECT from Merge table for more uniform distribution of threads. Added setting `max_streams_multiplier_for_merge_tables`. This fixes [#5797](https://github.com/ClickHouse/ClickHouse/issues/5797) [#5915](https://github.com/ClickHouse/ClickHouse/pull/5915) ([alexey-milovidov](https://github.com/alexey-milovidov)) -### Build/Testing/Packaging Improvement +#### Build/Testing/Packaging Improvement * Add a backward compatibility test for client-server interaction with different versions of clickhouse. [#5868](https://github.com/ClickHouse/ClickHouse/pull/5868) ([alesapin](https://github.com/alesapin)) * Test coverage information in every commit and pull request. [#5896](https://github.com/ClickHouse/ClickHouse/pull/5896) ([alesapin](https://github.com/alesapin)) * Cooperate with address sanitizer to support our custom allocators (`Arena` and `ArenaWithFreeLists`) for better debugging of "use-after-free" errors. [#5728](https://github.com/ClickHouse/ClickHouse/pull/5728) ([akuzm](https://github.com/akuzm)) @@ -1007,22 +1015,23 @@ fix comments to make obvious that it may throw. * Performance test concerning the new JIT feature with bigger dataset, as requested here [#5263](https://github.com/ClickHouse/ClickHouse/issues/5263) [#5887](https://github.com/ClickHouse/ClickHouse/pull/5887) ([Guillaume Tassery](https://github.com/YiuRULE)) * Run stateful tests in stress test [12693e568722f11e19859742f56428455501fd2a](https://github.com/ClickHouse/ClickHouse/commit/12693e568722f11e19859742f56428455501fd2a) ([alesapin](https://github.com/alesapin)) -### Backward Incompatible Change +#### Backward Incompatible Change * `Kafka` is broken in this version. * Enable `adaptive_index_granularity` = 10MB by default for new `MergeTree` tables. If you created new MergeTree tables on version 19.11+, downgrade to versions prior to 19.6 will be impossible. [#5628](https://github.com/ClickHouse/ClickHouse/pull/5628) ([alesapin](https://github.com/alesapin)) * Removed obsolete undocumented embedded dictionaries that were used by Yandex.Metrica. The functions `OSIn`, `SEIn`, `OSToRoot`, `SEToRoot`, `OSHierarchy`, `SEHierarchy` are no longer available. If you are using these functions, write email to clickhouse-feedback@yandex-team.com. Note: at the last moment we decided to keep these functions for a while. [#5780](https://github.com/ClickHouse/ClickHouse/pull/5780) ([alexey-milovidov](https://github.com/alexey-milovidov)) -## ClickHouse release 19.10.1.5, 2019-07-12 +## ClickHouse release 19.10 +### ClickHouse release 19.10.1.5, 2019-07-12 -### New Feature +#### New Feature * Add new column codec: `T64`. Made for (U)IntX/EnumX/Data(Time)/DecimalX columns. It should be good for columns with constant or small range values. Codec itself allows enlarge or shrink data type without re-compression. [#5557](https://github.com/ClickHouse/ClickHouse/pull/5557) ([Artem Zuikov](https://github.com/4ertus2)) * Add database engine `MySQL` that allow to view all the tables in remote MySQL server [#5599](https://github.com/ClickHouse/ClickHouse/pull/5599) ([Winter Zhang](https://github.com/zhang2014)) * `bitmapContains` implementation. It's 2x faster than `bitmapHasAny` if the second bitmap contains one element. [#5535](https://github.com/ClickHouse/ClickHouse/pull/5535) ([Zhichang Yu](https://github.com/yuzhichang)) * Support for `crc32` function (with behaviour exactly as in MySQL or PHP). Do not use it if you need a hash function. [#5661](https://github.com/ClickHouse/ClickHouse/pull/5661) ([Remen Ivan](https://github.com/BHYCHIK)) * Implemented `SYSTEM START/STOP DISTRIBUTED SENDS` queries to control asynchronous inserts into `Distributed` tables. [#4935](https://github.com/ClickHouse/ClickHouse/pull/4935) ([Winter Zhang](https://github.com/zhang2014)) -### Bug Fix +#### Bug Fix * Ignore query execution limits and max parts size for merge limits while executing mutations. [#5659](https://github.com/ClickHouse/ClickHouse/pull/5659) ([Anton Popov](https://github.com/CurtizJ)) * Fix bug which may lead to deduplication of normal blocks (extremely rare) and insertion of duplicate blocks (more often). [#5549](https://github.com/ClickHouse/ClickHouse/pull/5549) ([alesapin](https://github.com/alesapin)) * Fix of function `arrayEnumerateUniqRanked` for arguments with empty arrays [#5559](https://github.com/ClickHouse/ClickHouse/pull/5559) ([proller](https://github.com/proller)) @@ -1032,7 +1041,7 @@ fix comments to make obvious that it may throw. * Fix Float to Decimal convert overflow [#5607](https://github.com/ClickHouse/ClickHouse/pull/5607) ([coraxster](https://github.com/coraxster)) * Flush buffer when `WriteBufferFromHDFS`'s destructor is called. This fixes writing into `HDFS`. [#5684](https://github.com/ClickHouse/ClickHouse/pull/5684) ([Xindong Peng](https://github.com/eejoin)) -### Improvement +#### Improvement * Treat empty cells in `CSV` as default values when the setting `input_format_defaults_for_omitted_fields` is enabled. [#5625](https://github.com/ClickHouse/ClickHouse/pull/5625) ([akuzm](https://github.com/akuzm)) * Non-blocking loading of external dictionaries. [#5567](https://github.com/ClickHouse/ClickHouse/pull/5567) ([Vitaly Baranov](https://github.com/vitlibar)) * Network timeouts can be dynamically changed for already established connections according to the settings. [#4558](https://github.com/ClickHouse/ClickHouse/pull/4558) ([Konstantin Podshumok](https://github.com/podshumok)) @@ -1043,21 +1052,22 @@ fix comments to make obvious that it may throw. * Support `` section in `clickhouse-local` config file. [#5540](https://github.com/ClickHouse/ClickHouse/pull/5540) ([proller](https://github.com/proller)) * Allow run query with `remote` table function in `clickhouse-local` [#5627](https://github.com/ClickHouse/ClickHouse/pull/5627) ([proller](https://github.com/proller)) -### Performance Improvement +#### Performance Improvement * Add the possibility to write the final mark at the end of MergeTree columns. It allows to avoid useless reads for keys that are out of table data range. It is enabled only if adaptive index granularity is in use. [#5624](https://github.com/ClickHouse/ClickHouse/pull/5624) ([alesapin](https://github.com/alesapin)) * Improved performance of MergeTree tables on very slow filesystems by reducing number of `stat` syscalls. [#5648](https://github.com/ClickHouse/ClickHouse/pull/5648) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed performance degradation in reading from MergeTree tables that was introduced in version 19.6. Fixes #5631. [#5633](https://github.com/ClickHouse/ClickHouse/pull/5633) ([alexey-milovidov](https://github.com/alexey-milovidov)) -### Build/Testing/Packaging Improvement +#### Build/Testing/Packaging Improvement * Implemented `TestKeeper` as an implementation of ZooKeeper interface used for testing [#5643](https://github.com/ClickHouse/ClickHouse/pull/5643) ([alexey-milovidov](https://github.com/alexey-milovidov)) ([levushkin aleksej](https://github.com/alexey-milovidov)) * From now on `.sql` tests can be run isolated by server, in parallel, with random database. It allows to run them faster, add new tests with custom server configurations, and be sure that different tests doesn't affect each other. [#5554](https://github.com/ClickHouse/ClickHouse/pull/5554) ([Ivan](https://github.com/abyss7)) * Remove `` and `` from performance tests [#5672](https://github.com/ClickHouse/ClickHouse/pull/5672) ([Olga Khvostikova](https://github.com/stavrolia)) * Fixed "select_format" performance test for `Pretty` formats [#5642](https://github.com/ClickHouse/ClickHouse/pull/5642) ([alexey-milovidov](https://github.com/alexey-milovidov)) -## ClickHouse release 19.9.3.31, 2019-07-05 +## ClickHouse release 19.9 +### ClickHouse release 19.9.3.31, 2019-07-05 -### Bug Fix +#### Bug Fix * Fix segfault in Delta codec which affects columns with values less than 32 bits size. The bug led to random memory corruption. [#5786](https://github.com/ClickHouse/ClickHouse/pull/5786) ([alesapin](https://github.com/alesapin)) * Fix rare bug in checking of part with LowCardinality column. [#5832](https://github.com/ClickHouse/ClickHouse/pull/5832) ([alesapin](https://github.com/alesapin)) * Fix segfault in TTL merge with non-physical columns in block. [#5819](https://github.com/ClickHouse/ClickHouse/pull/5819) ([Anton Popov](https://github.com/CurtizJ)) @@ -1067,26 +1077,21 @@ fix comments to make obvious that it may throw. * Fix race condition, which cause that some queries may not appear in query_log instantly after SYSTEM FLUSH LOGS query. [#5685](https://github.com/ClickHouse/ClickHouse/pull/5685) ([Anton Popov](https://github.com/CurtizJ)) * Added missing support for constant arguments to `evalMLModel` function. [#5820](https://github.com/ClickHouse/ClickHouse/pull/5820) ([alexey-milovidov](https://github.com/alexey-milovidov)) -## ClickHouse release 19.7.5.29, 2019-07-05 +### ClickHouse release 19.9.2.4, 2019-06-24 -### Bug Fix -* Fix performance regression in some queries with JOIN. [#5192](https://github.com/ClickHouse/ClickHouse/pull/5192) ([Winter Zhang](https://github.com/zhang2014)) - -## ClickHouse release 19.9.2.4, 2019-06-24 - -### New Feature +#### New Feature * Print information about frozen parts in `system.parts` table. [#5471](https://github.com/ClickHouse/ClickHouse/pull/5471) ([proller](https://github.com/proller)) * Ask client password on clickhouse-client start on tty if not set in arguments [#5092](https://github.com/ClickHouse/ClickHouse/pull/5092) ([proller](https://github.com/proller)) * Implement `dictGet` and `dictGetOrDefault` functions for Decimal types. [#5394](https://github.com/ClickHouse/ClickHouse/pull/5394) ([Artem Zuikov](https://github.com/4ertus2)) -### Improvement +#### Improvement * Debian init: Add service stop timeout [#5522](https://github.com/ClickHouse/ClickHouse/pull/5522) ([proller](https://github.com/proller)) * Add setting forbidden by default to create table with suspicious types for LowCardinality [#5448](https://github.com/ClickHouse/ClickHouse/pull/5448) ([Olga Khvostikova](https://github.com/stavrolia)) * Regression functions return model weights when not used as State in function `evalMLMethod`. [#5411](https://github.com/ClickHouse/ClickHouse/pull/5411) ([Quid37](https://github.com/Quid37)) * Rename and improve regression methods. [#5492](https://github.com/ClickHouse/ClickHouse/pull/5492) ([Quid37](https://github.com/Quid37)) * Clearer interfaces of string searchers. [#5586](https://github.com/ClickHouse/ClickHouse/pull/5586) ([Danila Kutenin](https://github.com/danlark1)) -### Bug Fix +#### Bug Fix * Fix potential data loss in Kafka [#5445](https://github.com/ClickHouse/ClickHouse/pull/5445) ([Ivan](https://github.com/abyss7)) * Fix potential infinite loop in `PrettySpace` format when called with zero columns [#5560](https://github.com/ClickHouse/ClickHouse/pull/5560) ([Olga Khvostikova](https://github.com/stavrolia)) * Fixed UInt32 overflow bug in linear models. Allow eval ML model for non-const model argument. [#5516](https://github.com/ClickHouse/ClickHouse/pull/5516) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) @@ -1106,7 +1111,7 @@ fix comments to make obvious that it may throw. * Throw an exception on wrong integers in `dictGetT` functions instead of crash. [#5446](https://github.com/ClickHouse/ClickHouse/pull/5446) ([Artem Zuikov](https://github.com/4ertus2)) * Fix wrong element_count and load_factor for hashed dictionary in `system.dictionaries` table. [#5440](https://github.com/ClickHouse/ClickHouse/pull/5440) ([Azat Khuzhin](https://github.com/azat)) -### Build/Testing/Packaging Improvement +#### Build/Testing/Packaging Improvement * Fixed build without `Brotli` HTTP compression support (`ENABLE_BROTLI=OFF` cmake variable). [#5521](https://github.com/ClickHouse/ClickHouse/pull/5521) ([Anton Yuzhaninov](https://github.com/citrin)) * Include roaring.h as roaring/roaring.h [#5523](https://github.com/ClickHouse/ClickHouse/pull/5523) ([Orivej Desh](https://github.com/orivej)) * Fix gcc9 warnings in hyperscan (#line directive is evil!) [#5546](https://github.com/ClickHouse/ClickHouse/pull/5546) ([Danila Kutenin](https://github.com/danlark1)) @@ -1121,9 +1126,10 @@ fix comments to make obvious that it may throw. * Fix build clickhouse as submodule [#5574](https://github.com/ClickHouse/ClickHouse/pull/5574) ([proller](https://github.com/proller)) * Improve JSONExtract performance tests [#5444](https://github.com/ClickHouse/ClickHouse/pull/5444) ([Vitaly Baranov](https://github.com/vitlibar)) -## ClickHouse release 19.8.3.8, 2019-06-11 +## ClickHouse release 19.8 +### ClickHouse release 19.8.3.8, 2019-06-11 -### New Features +#### New Features * Added functions to work with JSON [#4686](https://github.com/ClickHouse/ClickHouse/pull/4686) ([hcz](https://github.com/hczhcz)) [#5124](https://github.com/ClickHouse/ClickHouse/pull/5124). ([Vitaly Baranov](https://github.com/vitlibar)) * Add a function basename, with a similar behaviour to a basename function, which exists in a lot of languages (`os.path.basename` in python, `basename` in PHP, etc...). Work with both an UNIX-like path or a Windows path. [#5136](https://github.com/ClickHouse/ClickHouse/pull/5136) ([Guillaume Tassery](https://github.com/YiuRULE)) * Added `LIMIT n, m BY` or `LIMIT m OFFSET n BY` syntax to set offset of n for LIMIT BY clause. [#5138](https://github.com/ClickHouse/ClickHouse/pull/5138) ([Anton Popov](https://github.com/CurtizJ)) @@ -1144,7 +1150,7 @@ fix comments to make obvious that it may throw. * Added functions `IPv4CIDRtoIPv4Range` and `IPv6CIDRtoIPv6Range` to calculate the lower and higher bounds for an IP in the subnet using a CIDR. [#5095](https://github.com/ClickHouse/ClickHouse/pull/5095) ([Guillaume Tassery](https://github.com/YiuRULE)) * Add a X-ClickHouse-Summary header when we send a query using HTTP with enabled setting `send_progress_in_http_headers`. Return the usual information of X-ClickHouse-Progress, with additional information like how many rows and bytes were inserted in the query. [#5116](https://github.com/ClickHouse/ClickHouse/pull/5116) ([Guillaume Tassery](https://github.com/YiuRULE)) -### Improvements +#### Improvements * Added `max_parts_in_total` setting for MergeTree family of tables (default: 100 000) that prevents unsafe specification of partition key #5166. [#5171](https://github.com/ClickHouse/ClickHouse/pull/5171) ([alexey-milovidov](https://github.com/alexey-milovidov)) * `clickhouse-obfuscator`: derive seed for individual columns by combining initial seed with column name, not column position. This is intended to transform datasets with multiple related tables, so that tables will remain JOINable after transformation. [#5178](https://github.com/ClickHouse/ClickHouse/pull/5178) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Added functions `JSONExtractRaw`, `JSONExtractKeyAndValues`. Renamed functions `jsonExtract` to `JSONExtract`. When something goes wrong these functions return the correspondent values, not `NULL`. Modified function `JSONExtract`, now it gets the return type from its last parameter and doesn't inject nullables. Implemented fallback to RapidJSON in case AVX2 instructions are not available. Simdjson library updated to a new version. [#5235](https://github.com/ClickHouse/ClickHouse/pull/5235) ([Vitaly Baranov](https://github.com/vitlibar)) @@ -1168,7 +1174,7 @@ It allows to set commit mode: after every batch of messages is handled, or after * Respect query settings in asynchronous INSERTs into Distributed tables. [#4936](https://github.com/ClickHouse/ClickHouse/pull/4936) ([TCeason](https://github.com/TCeason)) * Renamed functions `leastSqr` to `simpleLinearRegression`, `LinearRegression` to `linearRegression`, `LogisticRegression` to `logisticRegression`. [#5391](https://github.com/ClickHouse/ClickHouse/pull/5391) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -### Performance Improvements +#### Performance Improvements * Parallelize processing of parts of non-replicated MergeTree tables in ALTER MODIFY query. [#4639](https://github.com/ClickHouse/ClickHouse/pull/4639) ([Ivan Kush](https://github.com/IvanKush)) * Optimizations in regular expressions extraction. [#5193](https://github.com/ClickHouse/ClickHouse/pull/5193) [#5191](https://github.com/ClickHouse/ClickHouse/pull/5191) ([Danila Kutenin](https://github.com/danlark1)) * Do not add right join key column to join result if it's used only in join on section. [#5260](https://github.com/ClickHouse/ClickHouse/pull/5260) ([Artem Zuikov](https://github.com/4ertus2)) @@ -1178,7 +1184,7 @@ It allows to set commit mode: after every batch of messages is handled, or after * Upgrade our LZ4 implementation with reference one to have faster decompression. [#5070](https://github.com/ClickHouse/ClickHouse/pull/5070) ([Danila Kutenin](https://github.com/danlark1)) * Implemented MSD radix sort (based on kxsort), and partial sorting. [#5129](https://github.com/ClickHouse/ClickHouse/pull/5129) ([Evgenii Pravda](https://github.com/kvinty)) -### Bug Fixes +#### Bug Fixes * Fix push require columns with join [#5192](https://github.com/ClickHouse/ClickHouse/pull/5192) ([Winter Zhang](https://github.com/zhang2014)) * Fixed bug, when ClickHouse is run by systemd, the command `sudo service clickhouse-server forcerestart` was not working as expected. [#5204](https://github.com/ClickHouse/ClickHouse/pull/5204) ([proller](https://github.com/proller)) * Fix http error codes in DataPartsExchange (interserver http server on 9009 port always returned code 200, even on errors). [#5216](https://github.com/ClickHouse/ClickHouse/pull/5216) ([proller](https://github.com/proller)) @@ -1189,7 +1195,7 @@ It allows to set commit mode: after every batch of messages is handled, or after * Fix `retention` function. Now all conditions that satisfy in a row of data are added to the data state. [#5119](https://github.com/ClickHouse/ClickHouse/pull/5119) ([小路](https://github.com/nicelulu)) * Fix result type for `quantileExact` with Decimals. [#5304](https://github.com/ClickHouse/ClickHouse/pull/5304) ([Artem Zuikov](https://github.com/4ertus2)) -### Documentation +#### Documentation * Translate documentation for `CollapsingMergeTree` to chinese. [#5168](https://github.com/ClickHouse/ClickHouse/pull/5168) ([张风啸](https://github.com/AlexZFX)) * Translate some documentation about table engines to chinese. [#5134](https://github.com/ClickHouse/ClickHouse/pull/5134) @@ -1197,7 +1203,7 @@ It allows to set commit mode: after every batch of messages is handled, or after ([never lee](https://github.com/neverlee)) -### Build/Testing/Packaging Improvements +#### Build/Testing/Packaging Improvements * Fix some sanitizer reports that show probable use-after-free.[#5139](https://github.com/ClickHouse/ClickHouse/pull/5139) [#5143](https://github.com/ClickHouse/ClickHouse/pull/5143) [#5393](https://github.com/ClickHouse/ClickHouse/pull/5393) ([Ivan](https://github.com/abyss7)) * Move performance tests out of separate directories for convenience. [#5158](https://github.com/ClickHouse/ClickHouse/pull/5158) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix incorrect performance tests. [#5255](https://github.com/ClickHouse/ClickHouse/pull/5255) ([alesapin](https://github.com/alesapin)) @@ -1206,12 +1212,19 @@ It allows to set commit mode: after every batch of messages is handled, or after * Add small instruction how to write performance tests. [#5408](https://github.com/ClickHouse/ClickHouse/pull/5408) ([alesapin](https://github.com/alesapin)) * Add ability to make substitutions in create, fill and drop query in performance tests [#5367](https://github.com/ClickHouse/ClickHouse/pull/5367) ([Olga Khvostikova](https://github.com/stavrolia)) -## ClickHouse release 19.7.5.27, 2019-06-09 +## ClickHouse release 19.7 -### New features +### ClickHouse release 19.7.5.29, 2019-07-05 + +#### Bug Fix +* Fix performance regression in some queries with JOIN. [#5192](https://github.com/ClickHouse/ClickHouse/pull/5192) ([Winter Zhang](https://github.com/zhang2014)) + +### ClickHouse release 19.7.5.27, 2019-06-09 + +#### New features * Added bitmap related functions `bitmapHasAny` and `bitmapHasAll` analogous to `hasAny` and `hasAll` functions for arrays. [#5279](https://github.com/ClickHouse/ClickHouse/pull/5279) ([Sergi Vladykin](https://github.com/svladykin)) -### Bug Fixes +#### Bug Fixes * Fix segfault on `minmax` INDEX with Null value. [#5246](https://github.com/ClickHouse/ClickHouse/pull/5246) ([Nikita Vasilev](https://github.com/nikvas0)) * Mark all input columns in LIMIT BY as required output. It fixes 'Not found column' error in some distributed queries. [#5407](https://github.com/ClickHouse/ClickHouse/pull/5407) ([Constantin S. Pan](https://github.com/kvap)) * Fix "Column '0' already exists" error in `SELECT .. PREWHERE` on column with DEFAULT [#5397](https://github.com/ClickHouse/ClickHouse/pull/5397) ([proller](https://github.com/proller)) @@ -1232,9 +1245,9 @@ It allows to set commit mode: after every batch of messages is handled, or after did not process it, but already get list of children, will terminate the DDLWorker thread. [#5489](https://github.com/ClickHouse/ClickHouse/pull/5489) ([Azat Khuzhin](https://github.com/azat)) * Fix INSERT into Distributed() table with MATERIALIZED column. [#5429](https://github.com/ClickHouse/ClickHouse/pull/5429) ([Azat Khuzhin](https://github.com/azat)) -## ClickHouse release 19.7.3.9, 2019-05-30 +### ClickHouse release 19.7.3.9, 2019-05-30 -### New Features +#### New Features * Allow to limit the range of a setting that can be specified by user. These constraints can be set up in user settings profile. [#4931](https://github.com/ClickHouse/ClickHouse/pull/4931) ([Vitaly @@ -1250,7 +1263,7 @@ Tassery](https://github.com/YiuRULE)) [#5081](https://github.com/ClickHouse/ClickHouse/pull/5081) ([Alexander](https://github.com/Akazz)) -### Bug Fixes +#### Bug Fixes * Crash with uncompressed_cache + JOIN during merge (#5197) [#5133](https://github.com/ClickHouse/ClickHouse/pull/5133) ([Danila Kutenin](https://github.com/danlark1)) @@ -1262,14 +1275,14 @@ Kutenin](https://github.com/danlark1)) ([Ivan](https://github.com/abyss7)) * Fixed very rare data race condition that could happen when executing a query with UNION ALL involving at least two SELECTs from system.columns, system.tables, system.parts, system.parts_tables or tables of Merge family and performing ALTER of columns of the related tables concurrently. [#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) -### Performance Improvements +#### Performance Improvements * Use radix sort for sorting by single numeric column in `ORDER BY` without `LIMIT`. [#5106](https://github.com/ClickHouse/ClickHouse/pull/5106), [#4439](https://github.com/ClickHouse/ClickHouse/pull/4439) ([Evgenii Pravda](https://github.com/kvinty), [alexey-milovidov](https://github.com/alexey-milovidov)) -### Documentation +#### Documentation * Translate documentation for some table engines to Chinese. [#5107](https://github.com/ClickHouse/ClickHouse/pull/5107), [#5094](https://github.com/ClickHouse/ClickHouse/pull/5094), @@ -1278,7 +1291,7 @@ Kutenin](https://github.com/danlark1)) [#5068](https://github.com/ClickHouse/ClickHouse/pull/5068) ([never lee](https://github.com/neverlee)) -### Build/Testing/Packaging Improvements +#### Build/Testing/Packaging Improvements * Print UTF-8 characters properly in `clickhouse-test`. [#5084](https://github.com/ClickHouse/ClickHouse/pull/5084) ([alexey-milovidov](https://github.com/alexey-milovidov)) @@ -1294,9 +1307,10 @@ lee](https://github.com/neverlee)) [#5110](https://github.com/ClickHouse/ClickHouse/pull/5110) ([proller](https://github.com/proller)) -## ClickHouse release 19.6.3.18, 2019-06-13 +## ClickHouse release 19.6 +### ClickHouse release 19.6.3.18, 2019-06-13 -### Bug Fixes +#### Bug Fixes * Fixed IN condition pushdown for queries from table functions `mysql` and `odbc` and corresponding table engines. This fixes #3540 and #2384. [#5313](https://github.com/ClickHouse/ClickHouse/pull/5313) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix deadlock in Zookeeper. [#5297](https://github.com/ClickHouse/ClickHouse/pull/5297) ([github1youlc](https://github.com/github1youlc)) * Allow quoted decimals in CSV. [#5284](https://github.com/ClickHouse/ClickHouse/pull/5284) ([Artem Zuikov](https://github.com/4ertus2) @@ -1304,18 +1318,18 @@ lee](https://github.com/neverlee)) * Fix data race in rename query. [#5247](https://github.com/ClickHouse/ClickHouse/pull/5247) ([Winter Zhang](https://github.com/zhang2014)) * Temporarily disable LFAlloc. Usage of LFAlloc might lead to a lot of MAP_FAILED in allocating UncompressedCache and in a result to crashes of queries at high loaded servers. [cfdba93](https://github.com/ClickHouse/ClickHouse/commit/cfdba938ce22f16efeec504f7f90206a515b1280)([Danila Kutenin](https://github.com/danlark1)) -## ClickHouse release 19.6.2.11, 2019-05-13 +### ClickHouse release 19.6.2.11, 2019-05-13 -### New Features +#### New Features * TTL expressions for columns and tables. [#4212](https://github.com/ClickHouse/ClickHouse/pull/4212) ([Anton Popov](https://github.com/CurtizJ)) * Added support for `brotli` compression for HTTP responses (Accept-Encoding: br) [#4388](https://github.com/ClickHouse/ClickHouse/pull/4388) ([Mikhail](https://github.com/fandyushin)) * Added new function `isValidUTF8` for checking whether a set of bytes is correctly utf-8 encoded. [#4934](https://github.com/ClickHouse/ClickHouse/pull/4934) ([Danila Kutenin](https://github.com/danlark1)) * Add new load balancing policy `first_or_random` which sends queries to the first specified host and if it's inaccessible send queries to random hosts of shard. Useful for cross-replication topology setups. [#5012](https://github.com/ClickHouse/ClickHouse/pull/5012) ([nvartolomei](https://github.com/nvartolomei)) -### Experimental Features +#### Experimental Features * Add setting `index_granularity_bytes` (adaptive index granularity) for MergeTree* tables family. [#4826](https://github.com/ClickHouse/ClickHouse/pull/4826) ([alesapin](https://github.com/alesapin)) -### Improvements +#### Improvements * Added support for non-constant and negative size and length arguments for function `substringUTF8`. [#4989](https://github.com/ClickHouse/ClickHouse/pull/4989) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Disable push-down to right table in left join, left table in right join, and both tables in full join. This fixes wrong JOIN results in some cases. [#4846](https://github.com/ClickHouse/ClickHouse/pull/4846) ([Ivan](https://github.com/abyss7)) * `clickhouse-copier`: auto upload task configuration from `--task-file` option [#4876](https://github.com/ClickHouse/ClickHouse/pull/4876) ([proller](https://github.com/proller)) @@ -1323,13 +1337,13 @@ lee](https://github.com/neverlee)) * Support asterisks and qualified asterisks for multiple joins without subqueries [#4898](https://github.com/ClickHouse/ClickHouse/pull/4898) ([Artem Zuikov](https://github.com/4ertus2)) * Make missing column error message more user friendly. [#4915](https://github.com/ClickHouse/ClickHouse/pull/4915) ([Artem Zuikov](https://github.com/4ertus2)) -### Performance Improvements +#### Performance Improvements * Significant speedup of ASOF JOIN [#4924](https://github.com/ClickHouse/ClickHouse/pull/4924) ([Martijn Bakker](https://github.com/Gladdy)) -### Backward Incompatible Changes +#### Backward Incompatible Changes * HTTP header `Query-Id` was renamed to `X-ClickHouse-Query-Id` for consistency. [#4972](https://github.com/ClickHouse/ClickHouse/pull/4972) ([Mikhail](https://github.com/fandyushin)) -### Bug Fixes +#### Bug Fixes * Fixed potential null pointer dereference in `clickhouse-copier`. [#4900](https://github.com/ClickHouse/ClickHouse/pull/4900) ([proller](https://github.com/proller)) * Fixed error on query with JOIN + ARRAY JOIN [#4938](https://github.com/ClickHouse/ClickHouse/pull/4938) ([Artem Zuikov](https://github.com/4ertus2)) * Fixed hanging on start of the server when a dictionary depends on another dictionary via a database with engine=Dictionary. [#4962](https://github.com/ClickHouse/ClickHouse/pull/4962) ([Vitaly Baranov](https://github.com/vitlibar)) @@ -1337,7 +1351,7 @@ lee](https://github.com/neverlee)) * Fix potentially wrong result for `SELECT DISTINCT` with `JOIN` [#5001](https://github.com/ClickHouse/ClickHouse/pull/5001) ([Artem Zuikov](https://github.com/4ertus2)) * Fixed very rare data race condition that could happen when executing a query with UNION ALL involving at least two SELECTs from system.columns, system.tables, system.parts, system.parts_tables or tables of Merge family and performing ALTER of columns of the related tables concurrently. [#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) -### Build/Testing/Packaging Improvements +#### Build/Testing/Packaging Improvements * Fixed test failures when running clickhouse-server on different host [#4713](https://github.com/ClickHouse/ClickHouse/pull/4713) ([Vasily Nemkov](https://github.com/Enmk)) * clickhouse-test: Disable color control sequences in non tty environment. [#4937](https://github.com/ClickHouse/ClickHouse/pull/4937) ([alesapin](https://github.com/alesapin)) * clickhouse-test: Allow use any test database (remove `test.` qualification where it possible) [#5008](https://github.com/ClickHouse/ClickHouse/pull/5008) ([proller](https://github.com/proller)) @@ -1346,24 +1360,25 @@ lee](https://github.com/neverlee)) * Python util to help with backports and changelogs. [#4949](https://github.com/ClickHouse/ClickHouse/pull/4949) ([Ivan](https://github.com/abyss7)) -## ClickHouse release 19.5.4.22, 2019-05-13 +## ClickHouse release 19.5 +### ClickHouse release 19.5.4.22, 2019-05-13 -### Bug fixes +#### Bug fixes * Fixed possible crash in bitmap* functions [#5220](https://github.com/ClickHouse/ClickHouse/pull/5220) [#5228](https://github.com/ClickHouse/ClickHouse/pull/5228) ([Andy Yang](https://github.com/andyyzh)) * Fixed very rare data race condition that could happen when executing a query with UNION ALL involving at least two SELECTs from system.columns, system.tables, system.parts, system.parts_tables or tables of Merge family and performing ALTER of columns of the related tables concurrently. [#5189](https://github.com/ClickHouse/ClickHouse/pull/5189) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed error `Set for IN is not created yet in case of using single LowCardinality column in the left part of IN`. This error happened if LowCardinality column was the part of primary key. #5031 [#5154](https://github.com/ClickHouse/ClickHouse/pull/5154) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) * Modification of retention function: If a row satisfies both the first and NTH condition, only the first satisfied condition is added to the data state. Now all conditions that satisfy in a row of data are added to the data state. [#5119](https://github.com/ClickHouse/ClickHouse/pull/5119) ([小路](https://github.com/nicelulu)) -## ClickHouse release 19.5.3.8, 2019-04-18 +### ClickHouse release 19.5.3.8, 2019-04-18 -### Bug fixes +#### Bug fixes * Fixed type of setting `max_partitions_per_insert_block` from boolean to UInt64. [#5028](https://github.com/ClickHouse/ClickHouse/pull/5028) ([Mohammad Hossein Sekhavat](https://github.com/mhsekhavat)) -## ClickHouse release 19.5.2.6, 2019-04-15 +### ClickHouse release 19.5.2.6, 2019-04-15 -### New Features +#### New Features * [Hyperscan](https://github.com/intel/hyperscan) multiple regular expression matching was added (functions `multiMatchAny`, `multiMatchAnyIndex`, `multiFuzzyMatchAny`, `multiFuzzyMatchAnyIndex`). [#4780](https://github.com/ClickHouse/ClickHouse/pull/4780), [#4841](https://github.com/ClickHouse/ClickHouse/pull/4841) ([Danila Kutenin](https://github.com/danlark1)) * `multiSearchFirstPosition` function was added. [#4780](https://github.com/ClickHouse/ClickHouse/pull/4780) ([Danila Kutenin](https://github.com/danlark1)) @@ -1372,7 +1387,7 @@ lee](https://github.com/neverlee)) * Added `ASOF JOIN` which allows to run queries that join to the most recent value known. [#4774](https://github.com/ClickHouse/ClickHouse/pull/4774) [#4867](https://github.com/ClickHouse/ClickHouse/pull/4867) [#4863](https://github.com/ClickHouse/ClickHouse/pull/4863) [#4875](https://github.com/ClickHouse/ClickHouse/pull/4875) ([Martijn Bakker](https://github.com/Gladdy), [Artem Zuikov](https://github.com/4ertus2)) * Rewrite multiple `COMMA JOIN` to `CROSS JOIN`. Then rewrite them to `INNER JOIN` if possible. [#4661](https://github.com/ClickHouse/ClickHouse/pull/4661) ([Artem Zuikov](https://github.com/4ertus2)) -### Improvement +#### Improvement * `topK` and `topKWeighted` now supports custom `loadFactor` (fixes issue [#4252](https://github.com/ClickHouse/ClickHouse/issues/4252)). [#4634](https://github.com/ClickHouse/ClickHouse/pull/4634) ([Kirill Danshin](https://github.com/kirillDanshin)) * Allow to use `parallel_replicas_count > 1` even for tables without sampling (the setting is simply ignored for them). In previous versions it was lead to exception. [#4637](https://github.com/ClickHouse/ClickHouse/pull/4637) ([Alexey Elymanov](https://github.com/digitalist)) @@ -1389,7 +1404,7 @@ lee](https://github.com/neverlee)) * Improved data skipping indices calculation. [#4640](https://github.com/ClickHouse/ClickHouse/pull/4640) ([Nikita Vasilev](https://github.com/nikvas0)) * Keep ordinary, `DEFAULT`, `MATERIALIZED` and `ALIAS` columns in a single list (fixes issue [#2867](https://github.com/ClickHouse/ClickHouse/issues/2867)). [#4707](https://github.com/ClickHouse/ClickHouse/pull/4707) ([Alex Zatelepin](https://github.com/ztlpn)) -### Bug Fix +#### Bug Fix * Avoid `std::terminate` in case of memory allocation failure. Now `std::bad_alloc` exception is thrown as expected. [#4665](https://github.com/ClickHouse/ClickHouse/pull/4665) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixes capnproto reading from buffer. Sometimes files wasn't loaded successfully by HTTP. [#4674](https://github.com/ClickHouse/ClickHouse/pull/4674) ([Vladislav](https://github.com/smirnov-vs)) @@ -1429,19 +1444,19 @@ lee](https://github.com/neverlee)) * Fix function `toISOWeek` result for year 1970. [#4988](https://github.com/ClickHouse/ClickHouse/pull/4988) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix `DROP`, `TRUNCATE` and `OPTIMIZE` queries duplication, when executed on `ON CLUSTER` for `ReplicatedMergeTree*` tables family. [#4991](https://github.com/ClickHouse/ClickHouse/pull/4991) ([alesapin](https://github.com/alesapin)) -### Backward Incompatible Change +#### Backward Incompatible Change * Rename setting `insert_sample_with_metadata` to setting `input_format_defaults_for_omitted_fields`. [#4771](https://github.com/ClickHouse/ClickHouse/pull/4771) ([Artem Zuikov](https://github.com/4ertus2)) * Added setting `max_partitions_per_insert_block` (with value 100 by default). If inserted block contains larger number of partitions, an exception is thrown. Set it to 0 if you want to remove the limit (not recommended). [#4845](https://github.com/ClickHouse/ClickHouse/pull/4845) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Multi-search functions were renamed (`multiPosition` to `multiSearchAllPositions`, `multiSearch` to `multiSearchAny`, `firstMatch` to `multiSearchFirstIndex`). [#4780](https://github.com/ClickHouse/ClickHouse/pull/4780) ([Danila Kutenin](https://github.com/danlark1)) -### Performance Improvement +#### Performance Improvement * Optimize Volnitsky searcher by inlining, giving about 5-10% search improvement for queries with many needles or many similar bigrams. [#4862](https://github.com/ClickHouse/ClickHouse/pull/4862) ([Danila Kutenin](https://github.com/danlark1)) * Fix performance issue when setting `use_uncompressed_cache` is greater than zero, which appeared when all read data contained in cache. [#4913](https://github.com/ClickHouse/ClickHouse/pull/4913) ([alesapin](https://github.com/alesapin)) -### Build/Testing/Packaging Improvement +#### Build/Testing/Packaging Improvement * Hardening debug build: more granular memory mappings and ASLR; add memory protection for mark cache and index. This allows to find more memory stomping bugs in case when ASan and MSan cannot do it. [#4632](https://github.com/ClickHouse/ClickHouse/pull/4632) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Add support for cmake variables `ENABLE_PROTOBUF`, `ENABLE_PARQUET` and `ENABLE_BROTLI` which allows to enable/disable the above features (same as we can do for librdkafka, mysql, etc). [#4669](https://github.com/ClickHouse/ClickHouse/pull/4669) ([Silviu Caragea](https://github.com/silviucpp)) @@ -1456,9 +1471,10 @@ lee](https://github.com/neverlee)) * Disable usage of `mremap` when compiled with Thread Sanitizer. Surprisingly enough, TSan does not intercept `mremap` (though it does intercept `mmap`, `munmap`) that leads to false positives. Fixed TSan report in stateful tests. [#4859](https://github.com/ClickHouse/ClickHouse/pull/4859) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Add test checking using format schema via HTTP interface. [#4864](https://github.com/ClickHouse/ClickHouse/pull/4864) ([Vitaly Baranov](https://github.com/vitlibar)) -## ClickHouse release 19.4.4.33, 2019-04-17 +## ClickHouse release 19.4 +### ClickHouse release 19.4.4.33, 2019-04-17 -### Bug Fixes +#### Bug Fixes * Avoid `std::terminate` in case of memory allocation failure. Now `std::bad_alloc` exception is thrown as expected. [#4665](https://github.com/ClickHouse/ClickHouse/pull/4665) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixes capnproto reading from buffer. Sometimes files wasn't loaded successfully by HTTP. [#4674](https://github.com/ClickHouse/ClickHouse/pull/4674) ([Vladislav](https://github.com/smirnov-vs)) @@ -1493,34 +1509,34 @@ lee](https://github.com/neverlee)) * Fix function `toISOWeek` result for year 1970. [#4988](https://github.com/ClickHouse/ClickHouse/pull/4988) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix `DROP`, `TRUNCATE` and `OPTIMIZE` queries duplication, when executed on `ON CLUSTER` for `ReplicatedMergeTree*` tables family. [#4991](https://github.com/ClickHouse/ClickHouse/pull/4991) ([alesapin](https://github.com/alesapin)) -### Improvements +#### Improvements * Keep ordinary, `DEFAULT`, `MATERIALIZED` and `ALIAS` columns in a single list (fixes issue [#2867](https://github.com/ClickHouse/ClickHouse/issues/2867)). [#4707](https://github.com/ClickHouse/ClickHouse/pull/4707) ([Alex Zatelepin](https://github.com/ztlpn)) -## ClickHouse release 19.4.3.11, 2019-04-02 +### ClickHouse release 19.4.3.11, 2019-04-02 -### Bug Fixes +#### Bug Fixes * Fix crash in `FULL/RIGHT JOIN` when we joining on nullable vs not nullable. [#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) * Fix segmentation fault in `clickhouse-copier`. [#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) -### Build/Testing/Packaging Improvement +#### Build/Testing/Packaging Improvement * Add a way to launch clickhouse-server image from a custom user. [#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) -## ClickHouse release 19.4.2.7, 2019-03-30 +### ClickHouse release 19.4.2.7, 2019-03-30 -### Bug Fixes +#### Bug Fixes * Fixed reading from `Array(LowCardinality)` column in rare case when column contained a long sequence of empty arrays. [#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -## ClickHouse release 19.4.1.3, 2019-03-19 +### ClickHouse release 19.4.1.3, 2019-03-19 -### Bug Fixes +#### Bug Fixes * Fixed remote queries which contain both `LIMIT BY` and `LIMIT`. Previously, if `LIMIT BY` and `LIMIT` were used for remote query, `LIMIT` could happen before `LIMIT BY`, which led to too filtered result. [#4708](https://github.com/ClickHouse/ClickHouse/pull/4708) ([Constantin S. Pan](https://github.com/kvap)) -## ClickHouse release 19.4.0.49, 2019-03-09 +### ClickHouse release 19.4.0.49, 2019-03-09 -### New Features +#### New Features * Added full support for `Protobuf` format (input and output, nested data structures). [#4174](https://github.com/ClickHouse/ClickHouse/pull/4174) [#4493](https://github.com/ClickHouse/ClickHouse/pull/4493) ([Vitaly Baranov](https://github.com/vitlibar)) * Added bitmap functions with Roaring Bitmaps. [#4207](https://github.com/ClickHouse/ClickHouse/pull/4207) ([Andy Yang](https://github.com/andyyzh)) [#4568](https://github.com/ClickHouse/ClickHouse/pull/4568) ([Vitaly Baranov](https://github.com/vitlibar)) * Parquet format support. [#4448](https://github.com/ClickHouse/ClickHouse/pull/4448) ([proller](https://github.com/proller)) @@ -1531,7 +1547,7 @@ lee](https://github.com/neverlee)) * Added functions `arrayEnumerateDenseRanked` and `arrayEnumerateUniqRanked` (it's like `arrayEnumerateUniq` but allows to fine tune array depth to look inside multidimensional arrays). [#4475](https://github.com/ClickHouse/ClickHouse/pull/4475) ([proller](https://github.com/proller)) [#4601](https://github.com/ClickHouse/ClickHouse/pull/4601) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Multiple JOINS with some restrictions: no asterisks, no complex aliases in ON/WHERE/GROUP BY/... [#4462](https://github.com/ClickHouse/ClickHouse/pull/4462) ([Artem Zuikov](https://github.com/4ertus2)) -### Bug Fixes +#### Bug Fixes * This release also contains all bug fixes from 19.3 and 19.1. * Fixed bug in data skipping indices: order of granules after INSERT was incorrect. [#4407](https://github.com/ClickHouse/ClickHouse/pull/4407) ([Nikita Vasilev](https://github.com/nikvas0)) * Fixed `set` index for `Nullable` and `LowCardinality` columns. Before it, `set` index with `Nullable` or `LowCardinality` column led to error `Data type must be deserialized with multiple streams` while selecting. [#4594](https://github.com/ClickHouse/ClickHouse/pull/4594) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) @@ -1553,19 +1569,19 @@ lee](https://github.com/neverlee)) * Fix lambda function with predicate optimizer. [#4408](https://github.com/ClickHouse/ClickHouse/pull/4408) ([Winter Zhang](https://github.com/zhang2014)) * Multiple JOINs multiple fixes. [#4595](https://github.com/ClickHouse/ClickHouse/pull/4595) ([Artem Zuikov](https://github.com/4ertus2)) -### Improvements +#### Improvements * Support aliases in JOIN ON section for right table columns. [#4412](https://github.com/ClickHouse/ClickHouse/pull/4412) ([Artem Zuikov](https://github.com/4ertus2)) * Result of multiple JOINs need correct result names to be used in subselects. Replace flat aliases with source names in result. [#4474](https://github.com/ClickHouse/ClickHouse/pull/4474) ([Artem Zuikov](https://github.com/4ertus2)) * Improve push-down logic for joined statements. [#4387](https://github.com/ClickHouse/ClickHouse/pull/4387) ([Ivan](https://github.com/abyss7)) -### Performance Improvements +#### Performance Improvements * Improved heuristics of "move to PREWHERE" optimization. [#4405](https://github.com/ClickHouse/ClickHouse/pull/4405) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Use proper lookup tables that uses HashTable's API for 8-bit and 16-bit keys. [#4536](https://github.com/ClickHouse/ClickHouse/pull/4536) ([Amos Bird](https://github.com/amosbird)) * Improved performance of string comparison. [#4564](https://github.com/ClickHouse/ClickHouse/pull/4564) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Cleanup distributed DDL queue in a separate thread so that it doesn't slow down the main loop that processes distributed DDL tasks. [#4502](https://github.com/ClickHouse/ClickHouse/pull/4502) ([Alex Zatelepin](https://github.com/ztlpn)) * When `min_bytes_to_use_direct_io` is set to 1, not every file was opened with O_DIRECT mode because the data size to read was sometimes underestimated by the size of one compressed block. [#4526](https://github.com/ClickHouse/ClickHouse/pull/4526) ([alexey-milovidov](https://github.com/alexey-milovidov)) -### Build/Testing/Packaging Improvement +#### Build/Testing/Packaging Improvement * Added support for clang-9 [#4604](https://github.com/ClickHouse/ClickHouse/pull/4604) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix wrong `__asm__` instructions (again) [#4621](https://github.com/ClickHouse/ClickHouse/pull/4621) ([Konstantin Podshumok](https://github.com/podshumok)) * Add ability to specify settings for `clickhouse-performance-test` from command line. [#4437](https://github.com/ClickHouse/ClickHouse/pull/4437) ([alesapin](https://github.com/alesapin)) @@ -1577,29 +1593,30 @@ lee](https://github.com/neverlee)) * Fix compilation on Mac. [#4371](https://github.com/ClickHouse/ClickHouse/pull/4371) ([Vitaly Baranov](https://github.com/vitlibar)) * Build fixes for FreeBSD and various unusual build configurations. [#4444](https://github.com/ClickHouse/ClickHouse/pull/4444) ([proller](https://github.com/proller)) -## ClickHouse release 19.3.9.1, 2019-04-02 +## ClickHouse release 19.3 +### ClickHouse release 19.3.9.1, 2019-04-02 -### Bug Fixes +#### Bug Fixes * Fix crash in `FULL/RIGHT JOIN` when we joining on nullable vs not nullable. [#4855](https://github.com/ClickHouse/ClickHouse/pull/4855) ([Artem Zuikov](https://github.com/4ertus2)) * Fix segmentation fault in `clickhouse-copier`. [#4835](https://github.com/ClickHouse/ClickHouse/pull/4835) ([proller](https://github.com/proller)) * Fixed reading from `Array(LowCardinality)` column in rare case when column contained a long sequence of empty arrays. [#4850](https://github.com/ClickHouse/ClickHouse/pull/4850) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -### Build/Testing/Packaging Improvement +#### Build/Testing/Packaging Improvement * Add a way to launch clickhouse-server image from a custom user [#4753](https://github.com/ClickHouse/ClickHouse/pull/4753) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) -## ClickHouse release 19.3.7, 2019-03-12 +### ClickHouse release 19.3.7, 2019-03-12 -### Bug fixes +#### Bug fixes * Fixed error in #3920. This error manifests itself as random cache corruption (messages `Unknown codec family code`, `Cannot seek through file`) and segfaults. This bug first appeared in version 19.1 and is present in versions up to 19.1.10 and 19.3.6. [#4623](https://github.com/ClickHouse/ClickHouse/pull/4623) ([alexey-milovidov](https://github.com/alexey-milovidov)) -## ClickHouse release 19.3.6, 2019-03-02 +### ClickHouse release 19.3.6, 2019-03-02 -### Bug fixes +#### Bug fixes * When there are more than 1000 threads in a thread pool, `std::terminate` may happen on thread exit. [Azat Khuzhin](https://github.com/azat) [#4485](https://github.com/ClickHouse/ClickHouse/pull/4485) [#4505](https://github.com/ClickHouse/ClickHouse/pull/4505) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Now it's possible to create `ReplicatedMergeTree*` tables with comments on columns without defaults and tables with columns codecs without comments and defaults. Also fix comparison of codecs. [#4523](https://github.com/ClickHouse/ClickHouse/pull/4523) ([alesapin](https://github.com/alesapin)) @@ -1608,7 +1625,7 @@ lee](https://github.com/neverlee)) * Fixed hangup on server shutdown if distributed DDLs were used. [#4472](https://github.com/ClickHouse/ClickHouse/pull/4472) ([Alex Zatelepin](https://github.com/ztlpn)) * Incorrect column numbers were printed in error message about text format parsing for columns with number greater than 10. [#4484](https://github.com/ClickHouse/ClickHouse/pull/4484) ([alexey-milovidov](https://github.com/alexey-milovidov)) -### Build/Testing/Packaging Improvements +#### Build/Testing/Packaging Improvements * Fixed build with AVX enabled. [#4527](https://github.com/ClickHouse/ClickHouse/pull/4527) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Enable extended accounting and IO accounting based on good known version instead of kernel under which it is compiled. [#4541](https://github.com/ClickHouse/ClickHouse/pull/4541) ([nvartolomei](https://github.com/nvartolomei)) @@ -1616,33 +1633,33 @@ lee](https://github.com/neverlee)) * Removed the `inline` tags of `void readBinary(...)` in `Field.cpp`. Also merged redundant `namespace DB` blocks. [#4530](https://github.com/ClickHouse/ClickHouse/pull/4530) ([hcz](https://github.com/hczhcz)) -## ClickHouse release 19.3.5, 2019-02-21 +### ClickHouse release 19.3.5, 2019-02-21 -### Bug fixes +#### Bug fixes * Fixed bug with large http insert queries processing. [#4454](https://github.com/ClickHouse/ClickHouse/pull/4454) ([alesapin](https://github.com/alesapin)) * Fixed backward incompatibility with old versions due to wrong implementation of `send_logs_level` setting. [#4445](https://github.com/ClickHouse/ClickHouse/pull/4445) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed backward incompatibility of table function `remote` introduced with column comments. [#4446](https://github.com/ClickHouse/ClickHouse/pull/4446) ([alexey-milovidov](https://github.com/alexey-milovidov)) -## ClickHouse release 19.3.4, 2019-02-16 +### ClickHouse release 19.3.4, 2019-02-16 -### Improvements +#### Improvements * Table index size is not accounted for memory limits when doing `ATTACH TABLE` query. Avoided the possibility that a table cannot be attached after being detached. [#4396](https://github.com/ClickHouse/ClickHouse/pull/4396) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Slightly raised up the limit on max string and array size received from ZooKeeper. It allows to continue to work with increased size of `CLIENT_JVMFLAGS=-Djute.maxbuffer=...` on ZooKeeper. [#4398](https://github.com/ClickHouse/ClickHouse/pull/4398) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Allow to repair abandoned replica even if it already has huge number of nodes in its queue. [#4399](https://github.com/ClickHouse/ClickHouse/pull/4399) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Add one required argument to `SET` index (max stored rows number). [#4386](https://github.com/ClickHouse/ClickHouse/pull/4386) ([Nikita Vasilev](https://github.com/nikvas0)) -### Bug Fixes +#### Bug Fixes * Fixed `WITH ROLLUP` result for group by single `LowCardinality` key. [#4384](https://github.com/ClickHouse/ClickHouse/pull/4384) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) * Fixed bug in the set index (dropping a granule if it contains more than `max_rows` rows). [#4386](https://github.com/ClickHouse/ClickHouse/pull/4386) ([Nikita Vasilev](https://github.com/nikvas0)) * A lot of FreeBSD build fixes. [#4397](https://github.com/ClickHouse/ClickHouse/pull/4397) ([proller](https://github.com/proller)) * Fixed aliases substitution in queries with subquery containing same alias (issue [#4110](https://github.com/ClickHouse/ClickHouse/issues/4110)). [#4351](https://github.com/ClickHouse/ClickHouse/pull/4351) ([Artem Zuikov](https://github.com/4ertus2)) -### Build/Testing/Packaging Improvements +#### Build/Testing/Packaging Improvements * Add ability to run `clickhouse-server` for stateless tests in docker image. [#4347](https://github.com/ClickHouse/ClickHouse/pull/4347) ([Vasily Nemkov](https://github.com/Enmk)) -## ClickHouse release 19.3.3, 2019-02-13 +### ClickHouse release 19.3.3, 2019-02-13 -### New Features +#### New Features * Added the `KILL MUTATION` statement that allows removing mutations that are for some reasons stuck. Added `latest_failed_part`, `latest_fail_time`, `latest_fail_reason` fields to the `system.mutations` table for easier troubleshooting. [#4287](https://github.com/ClickHouse/ClickHouse/pull/4287) ([Alex Zatelepin](https://github.com/ztlpn)) * Added aggregate function `entropy` which computes Shannon entropy. [#4238](https://github.com/ClickHouse/ClickHouse/pull/4238) ([Quid37](https://github.com/Quid37)) * Added ability to send queries `INSERT INTO tbl VALUES (....` to server without splitting on `query` and `data` parts. [#4301](https://github.com/ClickHouse/ClickHouse/pull/4301) ([alesapin](https://github.com/alesapin)) @@ -1662,11 +1679,11 @@ lee](https://github.com/neverlee)) * Added hints while user make typo in function name or type in command line client. [#4239](https://github.com/ClickHouse/ClickHouse/pull/4239) ([Danila Kutenin](https://github.com/danlark1)) * Added `Query-Id` to Server's HTTP Response header. [#4231](https://github.com/ClickHouse/ClickHouse/pull/4231) ([Mikhail ](https://github.com/fandyushin)) -### Experimental features +#### Experimental features * Added `minmax` and `set` data skipping indices for MergeTree table engines family. [#4143](https://github.com/ClickHouse/ClickHouse/pull/4143) ([Nikita Vasilev](https://github.com/nikvas0)) * Added conversion of `CROSS JOIN` to `INNER JOIN` if possible. [#4221](https://github.com/ClickHouse/ClickHouse/pull/4221) [#4266](https://github.com/ClickHouse/ClickHouse/pull/4266) ([Artem Zuikov](https://github.com/4ertus2)) -### Bug Fixes +#### Bug Fixes * Fixed `Not found column` for duplicate columns in `JOIN ON` section. [#4279](https://github.com/ClickHouse/ClickHouse/pull/4279) ([Artem Zuikov](https://github.com/4ertus2)) * Make `START REPLICATED SENDS` command start replicated sends. [#4229](https://github.com/ClickHouse/ClickHouse/pull/4229) ([nvartolomei](https://github.com/nvartolomei)) * Fixed aggregate functions execution with `Array(LowCardinality)` arguments. [#4055](https://github.com/ClickHouse/ClickHouse/pull/4055) ([KochetovNicolai](https://github.com/KochetovNicolai)) @@ -1696,7 +1713,7 @@ lee](https://github.com/neverlee)) * Fix install package with missing /etc/clickhouse-server/config.xml. [#4343](https://github.com/ClickHouse/ClickHouse/pull/4343) ([proller](https://github.com/proller)) -### Build/Testing/Packaging Improvements +#### Build/Testing/Packaging Improvements * Debian package: correct /etc/clickhouse-server/preprocessed link according to config. [#4205](https://github.com/ClickHouse/ClickHouse/pull/4205) ([proller](https://github.com/proller)) * Various build fixes for FreeBSD. [#4225](https://github.com/ClickHouse/ClickHouse/pull/4225) ([proller](https://github.com/proller)) * Added ability to create, fill and drop tables in perftest. [#4220](https://github.com/ClickHouse/ClickHouse/pull/4220) ([alesapin](https://github.com/alesapin)) @@ -1716,17 +1733,17 @@ lee](https://github.com/neverlee)) * Added checking SSE and AVX instruction at start. [#4234](https://github.com/ClickHouse/ClickHouse/pull/4234) ([Igr](https://github.com/igron99)) * Init script will wait server until start. [#4281](https://github.com/ClickHouse/ClickHouse/pull/4281) ([proller](https://github.com/proller)) -### Backward Incompatible Changes +#### Backward Incompatible Changes * Removed `allow_experimental_low_cardinality_type` setting. `LowCardinality` data types are production ready. [#4323](https://github.com/ClickHouse/ClickHouse/pull/4323) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Reduce mark cache size and uncompressed cache size accordingly to available memory amount. [#4240](https://github.com/ClickHouse/ClickHouse/pull/4240) ([Lopatin Konstantin](https://github.com/k-lopatin) * Added keyword `INDEX` in `CREATE TABLE` query. A column with name `index` must be quoted with backticks or double quotes: `` `index` ``. [#4143](https://github.com/ClickHouse/ClickHouse/pull/4143) ([Nikita Vasilev](https://github.com/nikvas0)) * `sumMap` now promote result type instead of overflow. The old `sumMap` behavior can be obtained by using `sumMapWithOverflow` function. [#4151](https://github.com/ClickHouse/ClickHouse/pull/4151) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) -### Performance Improvements +#### Performance Improvements * `std::sort` replaced by `pdqsort` for queries without `LIMIT`. [#4236](https://github.com/ClickHouse/ClickHouse/pull/4236) ([Evgenii Pravda](https://github.com/kvinty)) * Now server reuse threads from global thread pool. This affects performance in some corner cases. [#4150](https://github.com/ClickHouse/ClickHouse/pull/4150) ([alexey-milovidov](https://github.com/alexey-milovidov)) -### Improvements +#### Improvements * Implemented AIO support for FreeBSD. [#4305](https://github.com/ClickHouse/ClickHouse/pull/4305) ([urgordeadbeef](https://github.com/urgordeadbeef)) * `SELECT * FROM a JOIN b USING a, b` now return `a` and `b` columns only from the left table. [#4141](https://github.com/ClickHouse/ClickHouse/pull/4141) ([Artem Zuikov](https://github.com/4ertus2)) * Allow `-C` option of client to work as `-c` option. [#4232](https://github.com/ClickHouse/ClickHouse/pull/4232) ([syominsergey](https://github.com/syominsergey)) @@ -1742,34 +1759,37 @@ lee](https://github.com/neverlee)) * Added info about the replicated_can_become_leader setting to system.replicas and add logging if the replica won't try to become leader. [#4379](https://github.com/ClickHouse/ClickHouse/pull/4379) ([Alex Zatelepin](https://github.com/ztlpn)) -## ClickHouse release 19.1.14, 2019-03-14 +## ClickHouse release 19.1 +### ClickHouse release 19.1.14, 2019-03-14 * Fixed error `Column ... queried more than once` that may happen if the setting `asterisk_left_columns_only` is set to 1 in case of using `GLOBAL JOIN` with `SELECT *` (rare case). The issue does not exist in 19.3 and newer. [6bac7d8d](https://github.com/ClickHouse/ClickHouse/pull/4692/commits/6bac7d8d11a9b0d6de0b32b53c47eb2f6f8e7062) ([Artem Zuikov](https://github.com/4ertus2)) -## ClickHouse release 19.1.13, 2019-03-12 +### ClickHouse release 19.1.13, 2019-03-12 This release contains exactly the same set of patches as 19.3.7. -## ClickHouse release 19.1.10, 2019-03-03 +### ClickHouse release 19.1.10, 2019-03-03 This release contains exactly the same set of patches as 19.3.6. -## ClickHouse release 19.1.9, 2019-02-21 +## ClickHouse release 19.1 +### ClickHouse release 19.1.9, 2019-02-21 -### Bug fixes +#### Bug fixes * Fixed backward incompatibility with old versions due to wrong implementation of `send_logs_level` setting. [#4445](https://github.com/ClickHouse/ClickHouse/pull/4445) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed backward incompatibility of table function `remote` introduced with column comments. [#4446](https://github.com/ClickHouse/ClickHouse/pull/4446) ([alexey-milovidov](https://github.com/alexey-milovidov)) -## ClickHouse release 19.1.8, 2019-02-16 +### ClickHouse release 19.1.8, 2019-02-16 -### Bug Fixes +#### Bug Fixes * Fix install package with missing /etc/clickhouse-server/config.xml. [#4343](https://github.com/ClickHouse/ClickHouse/pull/4343) ([proller](https://github.com/proller)) -## ClickHouse release 19.1.7, 2019-02-15 +## ClickHouse release 19.1 +### ClickHouse release 19.1.7, 2019-02-15 -### Bug Fixes +#### Bug Fixes * Correctly return the right type and properly handle locks in `joinGet` function. [#4153](https://github.com/ClickHouse/ClickHouse/pull/4153) ([Amos Bird](https://github.com/amosbird)) * Fixed error when system logs are tried to create again at server shutdown. [#4254](https://github.com/ClickHouse/ClickHouse/pull/4254) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed error: if there is a database with `Dictionary` engine, all dictionaries forced to load at server startup, and if there is a dictionary with ClickHouse source from localhost, the dictionary cannot load. [#4255](https://github.com/ClickHouse/ClickHouse/pull/4255) ([alexey-milovidov](https://github.com/alexey-milovidov)) @@ -1795,9 +1815,9 @@ This release contains exactly the same set of patches as 19.3.6. * Fixed bug with incorrect `Date` and `DateTime` comparison. [#4237](https://github.com/ClickHouse/ClickHouse/pull/4237) ([valexey](https://github.com/valexey)) * Fixed incorrect result when `Date` and `DateTime` arguments are used in branches of conditional operator (function `if`). Added generic case for function `if`. [#4243](https://github.com/ClickHouse/ClickHouse/pull/4243) ([alexey-milovidov](https://github.com/alexey-milovidov)) -## ClickHouse release 19.1.6, 2019-01-24 +### ClickHouse release 19.1.6, 2019-01-24 -### New Features +#### New Features * Custom per column compression codecs for tables. [#3899](https://github.com/ClickHouse/ClickHouse/pull/3899) [#4111](https://github.com/ClickHouse/ClickHouse/pull/4111) ([alesapin](https://github.com/alesapin), [Winter Zhang](https://github.com/zhang2014), [Anatoly](https://github.com/Sindbag)) * Added compression codec `Delta`. [#4052](https://github.com/ClickHouse/ClickHouse/pull/4052) ([alesapin](https://github.com/alesapin)) @@ -1815,12 +1835,12 @@ This release contains exactly the same set of patches as 19.3.6. * Added table function `remoteSecure`. Function works as `remote`, but uses secure connection. [#4088](https://github.com/ClickHouse/ClickHouse/pull/4088) ([proller](https://github.com/proller)) -### Experimental features +#### Experimental features * Added multiple JOINs emulation (`allow_experimental_multiple_joins_emulation` setting). [#3946](https://github.com/ClickHouse/ClickHouse/pull/3946) ([Artem Zuikov](https://github.com/4ertus2)) -### Bug Fixes +#### Bug Fixes * Make `compiled_expression_cache_size` setting limited by default to lower memory consumption. [#4041](https://github.com/ClickHouse/ClickHouse/pull/4041) ([alesapin](https://github.com/alesapin)) * Fix a bug that led to hangups in threads that perform ALTERs of Replicated tables and in the thread that updates configuration from ZooKeeper. [#2947](https://github.com/ClickHouse/ClickHouse/issues/2947) [#3891](https://github.com/ClickHouse/ClickHouse/issues/3891) [#3934](https://github.com/ClickHouse/ClickHouse/pull/3934) ([Alex Zatelepin](https://github.com/ztlpn)) @@ -1849,7 +1869,7 @@ This release contains exactly the same set of patches as 19.3.6. * Fixed column aliases for query with `JOIN ON` syntax and distributed tables. [#3980](https://github.com/ClickHouse/ClickHouse/pull/3980) ([Winter Zhang](https://github.com/zhang2014)) * Fixed error in internal implementation of `quantileTDigest` (found by Artem Vakhrushev). This error never happens in ClickHouse and was relevant only for those who use ClickHouse codebase as a library directly. [#3935](https://github.com/ClickHouse/ClickHouse/pull/3935) ([alexey-milovidov](https://github.com/alexey-milovidov)) -### Improvements +#### Improvements * Support for `IF NOT EXISTS` in `ALTER TABLE ADD COLUMN` statements along with `IF EXISTS` in `DROP/MODIFY/CLEAR/COMMENT COLUMN`. [#3900](https://github.com/ClickHouse/ClickHouse/pull/3900) ([Boris Granveaud](https://github.com/bgranvea)) * Function `parseDateTimeBestEffort`: support for formats `DD.MM.YYYY`, `DD.MM.YY`, `DD-MM-YYYY`, `DD-Mon-YYYY`, `DD/Month/YYYY` and similar. [#3922](https://github.com/ClickHouse/ClickHouse/pull/3922) ([alexey-milovidov](https://github.com/alexey-milovidov)) @@ -1864,7 +1884,7 @@ This release contains exactly the same set of patches as 19.3.6. * Add check that `SET send_logs_level = 'value'` query accept appropriate value. [#3873](https://github.com/ClickHouse/ClickHouse/pull/3873) ([Sabyanin Maxim](https://github.com/s-mx)) * Fixed data type check in type conversion functions. [#3896](https://github.com/ClickHouse/ClickHouse/pull/3896) ([Winter Zhang](https://github.com/zhang2014)) -### Performance Improvements +#### Performance Improvements * Add a MergeTree setting `use_minimalistic_part_header_in_zookeeper`. If enabled, Replicated tables will store compact part metadata in a single part znode. This can dramatically reduce ZooKeeper snapshot size (especially if the tables have a lot of columns). Note that after enabling this setting you will not be able to downgrade to a version that doesn't support it. [#3960](https://github.com/ClickHouse/ClickHouse/pull/3960) ([Alex Zatelepin](https://github.com/ztlpn)) * Add an DFA-based implementation for functions `sequenceMatch` and `sequenceCount` in case pattern doesn't contain time. [#4004](https://github.com/ClickHouse/ClickHouse/pull/4004) ([Léo Ercolanelli](https://github.com/ercolanelli-leo)) @@ -1872,13 +1892,13 @@ This release contains exactly the same set of patches as 19.3.6. * Zero left padding PODArray so that -1 element is always valid and zeroed. It's used for branchless calculation of offsets. [#3920](https://github.com/ClickHouse/ClickHouse/pull/3920) ([Amos Bird](https://github.com/amosbird)) * Reverted `jemalloc` version which lead to performance degradation. [#4018](https://github.com/ClickHouse/ClickHouse/pull/4018) ([alexey-milovidov](https://github.com/alexey-milovidov)) -### Backward Incompatible Changes +#### Backward Incompatible Changes * Removed undocumented feature `ALTER MODIFY PRIMARY KEY` because it was superseded by the `ALTER MODIFY ORDER BY` command. [#3887](https://github.com/ClickHouse/ClickHouse/pull/3887) ([Alex Zatelepin](https://github.com/ztlpn)) * Removed function `shardByHash`. [#3833](https://github.com/ClickHouse/ClickHouse/pull/3833) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Forbid using scalar subqueries with result of type `AggregateFunction`. [#3865](https://github.com/ClickHouse/ClickHouse/pull/3865) ([Ivan](https://github.com/abyss7)) -### Build/Testing/Packaging Improvements +#### Build/Testing/Packaging Improvements * Added support for PowerPC (`ppc64le`) build. [#4132](https://github.com/ClickHouse/ClickHouse/pull/4132) ([Danila Kutenin](https://github.com/danlark1)) * Stateful functional tests are run on public available dataset. [#3969](https://github.com/ClickHouse/ClickHouse/pull/3969) ([alexey-milovidov](https://github.com/alexey-milovidov)) @@ -1906,24 +1926,25 @@ This release contains exactly the same set of patches as 19.3.6. * Fixed typos in comments. [#4089](https://github.com/ClickHouse/ClickHouse/pull/4089) ([Evgenii Pravda](https://github.com/kvinty)) -## ClickHouse release 18.16.1, 2018-12-21 +## ClickHouse release 18.16 +### ClickHouse release 18.16.1, 2018-12-21 -### Bug fixes: +#### Bug fixes: * Fixed an error that led to problems with updating dictionaries with the ODBC source. [#3825](https://github.com/ClickHouse/ClickHouse/issues/3825), [#3829](https://github.com/ClickHouse/ClickHouse/issues/3829) * JIT compilation of aggregate functions now works with LowCardinality columns. [#3838](https://github.com/ClickHouse/ClickHouse/issues/3838) -### Improvements: +#### Improvements: * Added the `low_cardinality_allow_in_native_format` setting (enabled by default). When disabled, LowCardinality columns will be converted to ordinary columns for SELECT queries and ordinary columns will be expected for INSERT queries. [#3879](https://github.com/ClickHouse/ClickHouse/pull/3879) -### Build improvements: +#### Build improvements: * Fixes for builds on macOS and ARM. -## ClickHouse release 18.16.0, 2018-12-14 +### ClickHouse release 18.16.0, 2018-12-14 -### New features: +#### New features: * `DEFAULT` expressions are evaluated for missing fields when loading data in semi-structured input formats (`JSONEachRow`, `TSKV`). The feature is enabled with the `insert_sample_with_metadata` setting. [#3555](https://github.com/ClickHouse/ClickHouse/pull/3555) * The `ALTER TABLE` query now has the `MODIFY ORDER BY` action for changing the sorting key when adding or removing a table column. This is useful for tables in the `MergeTree` family that perform additional tasks when merging based on this sorting key, such as `SummingMergeTree`, `AggregatingMergeTree`, and so on. [#3581](https://github.com/ClickHouse/ClickHouse/pull/3581) [#3755](https://github.com/ClickHouse/ClickHouse/pull/3755) @@ -1942,7 +1963,7 @@ This release contains exactly the same set of patches as 19.3.6. * Added the `is_in_partition_key`, `is_in_sorting_key`, `is_in_primary_key`, and `is_in_sampling_key` columns to the `system.columns` table. [#3609](https://github.com/ClickHouse/ClickHouse/pull/3609) * Added the `min_time` and `max_time` columns to the `system.parts` table. These columns are populated when the partitioning key is an expression consisting of `DateTime` columns. [Emmanuel Donin de Rosière](https://github.com/ClickHouse/ClickHouse/pull/3800) -### Bug fixes: +#### Bug fixes: * Fixes and performance improvements for the `LowCardinality` data type. `GROUP BY` using `LowCardinality(Nullable(...))`. Getting the values of `extremes`. Processing high-order functions. `LEFT ARRAY JOIN`. Distributed `GROUP BY`. Functions that return `Array`. Execution of `ORDER BY`. Writing to `Distributed` tables (nicelulu). Backward compatibility for `INSERT` queries from old clients that implement the `Native` protocol. Support for `LowCardinality` for `JOIN`. Improved performance when working in a single stream. [#3823](https://github.com/ClickHouse/ClickHouse/pull/3823) [#3803](https://github.com/ClickHouse/ClickHouse/pull/3803) [#3799](https://github.com/ClickHouse/ClickHouse/pull/3799) [#3769](https://github.com/ClickHouse/ClickHouse/pull/3769) [#3744](https://github.com/ClickHouse/ClickHouse/pull/3744) [#3681](https://github.com/ClickHouse/ClickHouse/pull/3681) [#3651](https://github.com/ClickHouse/ClickHouse/pull/3651) [#3649](https://github.com/ClickHouse/ClickHouse/pull/3649) [#3641](https://github.com/ClickHouse/ClickHouse/pull/3641) [#3632](https://github.com/ClickHouse/ClickHouse/pull/3632) [#3568](https://github.com/ClickHouse/ClickHouse/pull/3568) [#3523](https://github.com/ClickHouse/ClickHouse/pull/3523) [#3518](https://github.com/ClickHouse/ClickHouse/pull/3518) * Fixed how the `select_sequential_consistency` option works. Previously, when this setting was enabled, an incomplete result was sometimes returned after beginning to write to a new partition. [#2863](https://github.com/ClickHouse/ClickHouse/pull/2863) @@ -1969,7 +1990,7 @@ This release contains exactly the same set of patches as 19.3.6. * Fixed a race condition when reading from `Buffer` tables and simultaneously performing `ALTER` or `DROP` on the target tables. [#3719](https://github.com/ClickHouse/ClickHouse/pull/3719) * Fixed a segfault if the `max_temporary_non_const_columns` limit was exceeded. [#3788](https://github.com/ClickHouse/ClickHouse/pull/3788) -### Improvements: +#### Improvements: * The server does not write the processed configuration files to the `/etc/clickhouse-server/` directory. Instead, it saves them in the `preprocessed_configs` directory inside `path`. This means that the `/etc/clickhouse-server/` directory doesn't have write access for the `clickhouse` user, which improves security. [#2443](https://github.com/ClickHouse/ClickHouse/pull/2443) * The `min_merge_bytes_to_use_direct_io` option is set to 10 GiB by default. A merge that forms large parts of tables from the MergeTree family will be performed in `O_DIRECT` mode, which prevents excessive page cache eviction. [#3504](https://github.com/ClickHouse/ClickHouse/pull/3504) @@ -2001,7 +2022,7 @@ This release contains exactly the same set of patches as 19.3.6. * Fixed the behavior of stateful functions like `rowNumberInAllBlocks`. They previously output a result that was one number larger due to starting during query analysis. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3729) * If the `force_restore_data` file can't be deleted, an error message is displayed. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3794) -### Build improvements: +#### Build improvements: * Updated the `jemalloc` library, which fixes a potential memory leak. [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/3557) * Profiling with `jemalloc` is enabled by default in order to debug builds. [2cc82f5c](https://github.com/ClickHouse/ClickHouse/commit/2cc82f5cbe266421cd4c1165286c2c47e5ffcb15) @@ -2012,95 +2033,96 @@ This release contains exactly the same set of patches as 19.3.6. * For a Docker image, added support for initializing databases using files in the `/docker-entrypoint-initdb.d` directory. [Konstantin Lebedev](https://github.com/ClickHouse/ClickHouse/pull/3695) * Fixes for builds on ARM. [#3709](https://github.com/ClickHouse/ClickHouse/pull/3709) -### Backward incompatible changes: +#### Backward incompatible changes: * Removed the ability to compare the `Date` type with a number. Instead of `toDate('2018-12-18') = 17883`, you must use explicit type conversion `= toDate(17883)` [#3687](https://github.com/ClickHouse/ClickHouse/pull/3687) -## ClickHouse release 18.14.19, 2018-12-19 +## ClickHouse release 18.14 +### ClickHouse release 18.14.19, 2018-12-19 -### Bug fixes: +#### Bug fixes: * Fixed an error that led to problems with updating dictionaries with the ODBC source. [#3825](https://github.com/ClickHouse/ClickHouse/issues/3825), [#3829](https://github.com/ClickHouse/ClickHouse/issues/3829) * Databases are correctly specified when executing DDL `ON CLUSTER` queries. [#3460](https://github.com/ClickHouse/ClickHouse/pull/3460) * Fixed a segfault if the `max_temporary_non_const_columns` limit was exceeded. [#3788](https://github.com/ClickHouse/ClickHouse/pull/3788) -### Build improvements: +#### Build improvements: * Fixes for builds on ARM. -## ClickHouse release 18.14.18, 2018-12-04 +### ClickHouse release 18.14.18, 2018-12-04 -### Bug fixes: +#### Bug fixes: * Fixed error in `dictGet...` function for dictionaries of type `range`, if one of the arguments is constant and other is not. [#3751](https://github.com/ClickHouse/ClickHouse/pull/3751) * Fixed error that caused messages `netlink: '...': attribute type 1 has an invalid length` to be printed in Linux kernel log, that was happening only on fresh enough versions of Linux kernel. [#3749](https://github.com/ClickHouse/ClickHouse/pull/3749) * Fixed segfault in function `empty` for argument of `FixedString` type. [Daniel, Dao Quang Minh](https://github.com/ClickHouse/ClickHouse/pull/3703) * Fixed excessive memory allocation when using large value of `max_query_size` setting (a memory chunk of `max_query_size` bytes was preallocated at once). [#3720](https://github.com/ClickHouse/ClickHouse/pull/3720) -### Build changes: +#### Build changes: * Fixed build with LLVM/Clang libraries of version 7 from the OS packages (these libraries are used for runtime query compilation). [#3582](https://github.com/ClickHouse/ClickHouse/pull/3582) -## ClickHouse release 18.14.17, 2018-11-30 +### ClickHouse release 18.14.17, 2018-11-30 -### Bug fixes: +#### Bug fixes: * Fixed cases when the ODBC bridge process did not terminate with the main server process. [#3642](https://github.com/ClickHouse/ClickHouse/pull/3642) * Fixed synchronous insertion into the `Distributed` table with a columns list that differs from the column list of the remote table. [#3673](https://github.com/ClickHouse/ClickHouse/pull/3673) * Fixed a rare race condition that can lead to a crash when dropping a MergeTree table. [#3643](https://github.com/ClickHouse/ClickHouse/pull/3643) * Fixed a query deadlock in case when query thread creation fails with the `Resource temporarily unavailable` error. [#3643](https://github.com/ClickHouse/ClickHouse/pull/3643) * Fixed parsing of the `ENGINE` clause when the `CREATE AS table` syntax was used and the `ENGINE` clause was specified before the `AS table` (the error resulted in ignoring the specified engine). [#3692](https://github.com/ClickHouse/ClickHouse/pull/3692) -## ClickHouse release 18.14.15, 2018-11-21 +### ClickHouse release 18.14.15, 2018-11-21 -### Bug fixes: +#### Bug fixes: * The size of memory chunk was overestimated while deserializing the column of type `Array(String)` that leads to "Memory limit exceeded" errors. The issue appeared in version 18.12.13. [#3589](https://github.com/ClickHouse/ClickHouse/issues/3589) -## ClickHouse release 18.14.14, 2018-11-20 +### ClickHouse release 18.14.14, 2018-11-20 -### Bug fixes: +#### Bug fixes: * Fixed `ON CLUSTER` queries when cluster configured as secure (flag ``). [#3599](https://github.com/ClickHouse/ClickHouse/pull/3599) -### Build changes: +#### Build changes: * Fixed problems (llvm-7 from system, macos) [#3582](https://github.com/ClickHouse/ClickHouse/pull/3582) -## ClickHouse release 18.14.13, 2018-11-08 +### ClickHouse release 18.14.13, 2018-11-08 -### Bug fixes: +#### Bug fixes: * Fixed the `Block structure mismatch in MergingSorted stream` error. [#3162](https://github.com/ClickHouse/ClickHouse/issues/3162) * Fixed `ON CLUSTER` queries in case when secure connections were turned on in the cluster config (the `` flag). [#3465](https://github.com/ClickHouse/ClickHouse/pull/3465) * Fixed an error in queries that used `SAMPLE`, `PREWHERE` and alias columns. [#3543](https://github.com/ClickHouse/ClickHouse/pull/3543) * Fixed a rare `unknown compression method` error when the `min_bytes_to_use_direct_io` setting was enabled. [3544](https://github.com/ClickHouse/ClickHouse/pull/3544) -### Performance improvements: +#### Performance improvements: * Fixed performance regression of queries with `GROUP BY` of columns of UInt16 or Date type when executing on AMD EPYC processors. [Igor Lapko](https://github.com/ClickHouse/ClickHouse/pull/3512) * Fixed performance regression of queries that process long strings. [#3530](https://github.com/ClickHouse/ClickHouse/pull/3530) -### Build improvements: +#### Build improvements: * Improvements for simplifying the Arcadia build. [#3475](https://github.com/ClickHouse/ClickHouse/pull/3475), [#3535](https://github.com/ClickHouse/ClickHouse/pull/3535) -## ClickHouse release 18.14.12, 2018-11-02 +### ClickHouse release 18.14.12, 2018-11-02 -### Bug fixes: +#### Bug fixes: * Fixed a crash on joining two unnamed subqueries. [#3505](https://github.com/ClickHouse/ClickHouse/pull/3505) * Fixed generating incorrect queries (with an empty `WHERE` clause) when querying external databases. [hotid](https://github.com/ClickHouse/ClickHouse/pull/3477) * Fixed using an incorrect timeout value in ODBC dictionaries. [Marek Vavruša](https://github.com/ClickHouse/ClickHouse/pull/3511) -## ClickHouse release 18.14.11, 2018-10-29 +### ClickHouse release 18.14.11, 2018-10-29 -### Bug fixes: +#### Bug fixes: * Fixed the error `Block structure mismatch in UNION stream: different number of columns` in LIMIT queries. [#2156](https://github.com/ClickHouse/ClickHouse/issues/2156) * Fixed errors when merging data in tables containing arrays inside Nested structures. [#3397](https://github.com/ClickHouse/ClickHouse/pull/3397) * Fixed incorrect query results if the `merge_tree_uniform_read_distribution` setting is disabled (it is enabled by default). [#3429](https://github.com/ClickHouse/ClickHouse/pull/3429) * Fixed an error on inserts to a Distributed table in Native format. [#3411](https://github.com/ClickHouse/ClickHouse/issues/3411) -## ClickHouse release 18.14.10, 2018-10-23 +### ClickHouse release 18.14.10, 2018-10-23 * The `compile_expressions` setting (JIT compilation of expressions) is disabled by default. [#3410](https://github.com/ClickHouse/ClickHouse/pull/3410) * The `enable_optimize_predicate_expression` setting is disabled by default. -## ClickHouse release 18.14.9, 2018-10-16 +### ClickHouse release 18.14.9, 2018-10-16 -### New features: +#### New features: * The `WITH CUBE` modifier for `GROUP BY` (the alternative syntax `GROUP BY CUBE(...)` is also available). [#3172](https://github.com/ClickHouse/ClickHouse/pull/3172) * Added the `formatDateTime` function. [Alexandr Krasheninnikov](https://github.com/ClickHouse/ClickHouse/pull/2770) @@ -2113,12 +2135,12 @@ This release contains exactly the same set of patches as 19.3.6. * Now you can use pre-defined `database` and `table` macros when declaring `Replicated` tables. [#3251](https://github.com/ClickHouse/ClickHouse/pull/3251) * Added the ability to read `Decimal` type values in engineering notation (indicating powers of ten). [#3153](https://github.com/ClickHouse/ClickHouse/pull/3153) -### Experimental features: +#### Experimental features: * Optimization of the GROUP BY clause for `LowCardinality data types.` [#3138](https://github.com/ClickHouse/ClickHouse/pull/3138) * Optimized calculation of expressions for `LowCardinality data types.` [#3200](https://github.com/ClickHouse/ClickHouse/pull/3200) -### Improvements: +#### Improvements: * Significantly reduced memory consumption for queries with `ORDER BY` and `LIMIT`. See the `max_bytes_before_remerge_sort` setting. [#3205](https://github.com/ClickHouse/ClickHouse/pull/3205) * In the absence of `JOIN` (`LEFT`, `INNER`, ...), `INNER JOIN` is assumed. [#3147](https://github.com/ClickHouse/ClickHouse/pull/3147) @@ -2149,7 +2171,7 @@ This release contains exactly the same set of patches as 19.3.6. * Reduced the number of `open` and `close` system calls when reading from a `MergeTree table`. [#3283](https://github.com/ClickHouse/ClickHouse/pull/3283) * A `TRUNCATE TABLE` query can be executed on any replica (the query is passed to the leader replica). [Kirill Shvakov](https://github.com/ClickHouse/ClickHouse/pull/3375) -### Bug fixes: +#### Bug fixes: * Fixed an issue with `Dictionary` tables for `range_hashed` dictionaries. This error occurred in version 18.12.17. [#1702](https://github.com/ClickHouse/ClickHouse/pull/1702) * Fixed an error when loading `range_hashed` dictionaries (the message `Unsupported type Nullable (...)`). This error occurred in version 18.12.17. [#3362](https://github.com/ClickHouse/ClickHouse/pull/3362) @@ -2185,13 +2207,15 @@ This release contains exactly the same set of patches as 19.3.6. * Fixed segfault that could occur in rare cases after optimization that replaced AND chains from equality evaluations with the corresponding IN expression. [liuyimin-bytedance](https://github.com/ClickHouse/ClickHouse/pull/3339) * Minor corrections to `clickhouse-benchmark`: previously, client information was not sent to the server; now the number of queries executed is calculated more accurately when shutting down and for limiting the number of iterations. [#3351](https://github.com/ClickHouse/ClickHouse/pull/3351) [#3352](https://github.com/ClickHouse/ClickHouse/pull/3352) -### Backward incompatible changes: +#### Backward incompatible changes: * Removed the `allow_experimental_decimal_type` option. The `Decimal` data type is available for default use. [#3329](https://github.com/ClickHouse/ClickHouse/pull/3329) -## ClickHouse release 18.12.17, 2018-09-16 +## ClickHouse release 18.12 -### New features: +### ClickHouse release 18.12.17, 2018-09-16 + +#### New features: * `invalidate_query` (the ability to specify a query to check whether an external dictionary needs to be updated) is implemented for the `clickhouse` source. [#3126](https://github.com/ClickHouse/ClickHouse/pull/3126) * Added the ability to use `UInt*`, `Int*`, and `DateTime` data types (along with the `Date` type) as a `range_hashed` external dictionary key that defines the boundaries of ranges. Now `NULL` can be used to designate an open range. [Vasily Nemkov](https://github.com/ClickHouse/ClickHouse/pull/3123) @@ -2199,32 +2223,32 @@ This release contains exactly the same set of patches as 19.3.6. * The `Decimal` type now supports mathematical functions (`exp`, `sin` and so on.) [#3129](https://github.com/ClickHouse/ClickHouse/pull/3129) * The `system.part_log` table now has the `partition_id` column. [#3089](https://github.com/ClickHouse/ClickHouse/pull/3089) -### Bug fixes: +#### Bug fixes: * `Merge` now works correctly on `Distributed` tables. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3159) * Fixed incompatibility (unnecessary dependency on the `glibc` version) that made it impossible to run ClickHouse on `Ubuntu Precise` and older versions. The incompatibility arose in version 18.12.13. [#3130](https://github.com/ClickHouse/ClickHouse/pull/3130) * Fixed errors in the `enable_optimize_predicate_expression` setting. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3107) * Fixed a minor issue with backwards compatibility that appeared when working with a cluster of replicas on versions earlier than 18.12.13 and simultaneously creating a new replica of a table on a server with a newer version (shown in the message `Can not clone replica, because the ... updated to new ClickHouse version`, which is logical, but shouldn't happen). [#3122](https://github.com/ClickHouse/ClickHouse/pull/3122) -### Backward incompatible changes: +#### Backward incompatible changes: * The `enable_optimize_predicate_expression` option is enabled by default (which is rather optimistic). If query analysis errors occur that are related to searching for the column names, set `enable_optimize_predicate_expression` to 0. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3107) -## ClickHouse release 18.12.14, 2018-09-13 +### ClickHouse release 18.12.14, 2018-09-13 -### New features: +#### New features: * Added support for `ALTER UPDATE` queries. [#3035](https://github.com/ClickHouse/ClickHouse/pull/3035) * Added the `allow_ddl` option, which restricts the user's access to DDL queries. [#3104](https://github.com/ClickHouse/ClickHouse/pull/3104) * Added the `min_merge_bytes_to_use_direct_io` option for `MergeTree` engines, which allows you to set a threshold for the total size of the merge (when above the threshold, data part files will be handled using O_DIRECT). [#3117](https://github.com/ClickHouse/ClickHouse/pull/3117) * The `system.merges` system table now contains the `partition_id` column. [#3099](https://github.com/ClickHouse/ClickHouse/pull/3099) -### Improvements +#### Improvements * If a data part remains unchanged during mutation, it isn't downloaded by replicas. [#3103](https://github.com/ClickHouse/ClickHouse/pull/3103) * Autocomplete is available for names of settings when working with `clickhouse-client`. [#3106](https://github.com/ClickHouse/ClickHouse/pull/3106) -### Bug fixes: +#### Bug fixes: * Added a check for the sizes of arrays that are elements of `Nested` type fields when inserting. [#3118](https://github.com/ClickHouse/ClickHouse/pull/3118) * Fixed an error updating external dictionaries with the `ODBC` source and `hashed` storage. This error occurred in version 18.12.13. @@ -2232,9 +2256,9 @@ This release contains exactly the same set of patches as 19.3.6. * Fixed an error in aggregate functions for arrays that can have `NULL` elements. [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/3097) -## ClickHouse release 18.12.13, 2018-09-10 +### ClickHouse release 18.12.13, 2018-09-10 -### New features: +#### New features: * Added the `DECIMAL(digits, scale)` data type (`Decimal32(scale)`, `Decimal64(scale)`, `Decimal128(scale)`). To enable it, use the setting `allow_experimental_decimal_type`. [#2846](https://github.com/ClickHouse/ClickHouse/pull/2846) [#2970](https://github.com/ClickHouse/ClickHouse/pull/2970) [#3008](https://github.com/ClickHouse/ClickHouse/pull/3008) [#3047](https://github.com/ClickHouse/ClickHouse/pull/3047) * New `WITH ROLLUP` modifier for `GROUP BY` (alternative syntax: `GROUP BY ROLLUP(...)`). [#2948](https://github.com/ClickHouse/ClickHouse/pull/2948) @@ -2258,12 +2282,12 @@ This release contains exactly the same set of patches as 19.3.6. * Now you can add (merge) states of aggregate functions by using the plus operator, and multiply the states of aggregate functions by a nonnegative constant. [#3062](https://github.com/ClickHouse/ClickHouse/pull/3062) [#3034](https://github.com/ClickHouse/ClickHouse/pull/3034) * Tables in the MergeTree family now have the virtual column `_partition_id`. [#3089](https://github.com/ClickHouse/ClickHouse/pull/3089) -### Experimental features: +#### Experimental features: * Added the `LowCardinality(T)` data type. This data type automatically creates a local dictionary of values and allows data processing without unpacking the dictionary. [#2830](https://github.com/ClickHouse/ClickHouse/pull/2830) * Added a cache of JIT-compiled functions and a counter for the number of uses before compiling. To JIT compile expressions, enable the `compile_expressions` setting. [#2990](https://github.com/ClickHouse/ClickHouse/pull/2990) [#3077](https://github.com/ClickHouse/ClickHouse/pull/3077) -### Improvements: +#### Improvements: * Fixed the problem with unlimited accumulation of the replication log when there are abandoned replicas. Added an effective recovery mode for replicas with a long lag. * Improved performance of `GROUP BY` with multiple aggregation fields when one of them is string and the others are fixed length. @@ -2292,7 +2316,7 @@ This release contains exactly the same set of patches as 19.3.6. * Added randomization when running the cleanup thread periodically for `ReplicatedMergeTree` tables in order to avoid periodic load spikes when there are a very large number of `ReplicatedMergeTree` tables. * Support for `ATTACH TABLE ... ON CLUSTER` queries. [#3025](https://github.com/ClickHouse/ClickHouse/pull/3025) -### Bug fixes: +#### Bug fixes: * Fixed an issue with `Dictionary` tables (throws the `Size of offsets doesn't match size of column` or `Unknown compression method` exception). This bug appeared in version 18.10.3. [#2913](https://github.com/ClickHouse/ClickHouse/issues/2913) * Fixed a bug when merging `CollapsingMergeTree` tables if one of the data parts is empty (these parts are formed during merge or `ALTER DELETE` if all data was deleted), and the `vertical` algorithm was used for the merge. [#3049](https://github.com/ClickHouse/ClickHouse/pull/3049) @@ -2316,17 +2340,17 @@ This release contains exactly the same set of patches as 19.3.6. * Fixed incorrect code for adding nested data structures in a `SummingMergeTree`. * When allocating memory for states of aggregate functions, alignment is correctly taken into account, which makes it possible to use operations that require alignment when implementing states of aggregate functions. [chenxing-xc](https://github.com/ClickHouse/ClickHouse/pull/2808) -### Security fix: +#### Security fix: * Safe use of ODBC data sources. Interaction with ODBC drivers uses a separate `clickhouse-odbc-bridge` process. Errors in third-party ODBC drivers no longer cause problems with server stability or vulnerabilities. [#2828](https://github.com/ClickHouse/ClickHouse/pull/2828) [#2879](https://github.com/ClickHouse/ClickHouse/pull/2879) [#2886](https://github.com/ClickHouse/ClickHouse/pull/2886) [#2893](https://github.com/ClickHouse/ClickHouse/pull/2893) [#2921](https://github.com/ClickHouse/ClickHouse/pull/2921) * Fixed incorrect validation of the file path in the `catBoostPool` table function. [#2894](https://github.com/ClickHouse/ClickHouse/pull/2894) * The contents of system tables (`tables`, `databases`, `parts`, `columns`, `parts_columns`, `merges`, `mutations`, `replicas`, and `replication_queue`) are filtered according to the user's configured access to databases (`allow_databases`). [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2856) -### Backward incompatible changes: +#### Backward incompatible changes: * In queries with JOIN, the star character expands to a list of columns in all tables, in compliance with the SQL standard. You can restore the old behavior by setting `asterisk_left_columns_only` to 1 on the user configuration level. -### Build changes: +#### Build changes: * Most integration tests can now be run by commit. * Code style checks can also be run by commit. @@ -2335,16 +2359,18 @@ This release contains exactly the same set of patches as 19.3.6. * Debugging the build uses the `jemalloc` debug option. * The interface of the library for interacting with ZooKeeper is declared abstract. [#2950](https://github.com/ClickHouse/ClickHouse/pull/2950) -## ClickHouse release 18.10.3, 2018-08-13 +## ClickHouse release 18.10 -### New features: +### ClickHouse release 18.10.3, 2018-08-13 + +#### New features: * HTTPS can be used for replication. [#2760](https://github.com/ClickHouse/ClickHouse/pull/2760) * Added the functions `murmurHash2_64`, `murmurHash3_32`, `murmurHash3_64`, and `murmurHash3_128` in addition to the existing `murmurHash2_32`. [#2791](https://github.com/ClickHouse/ClickHouse/pull/2791) * Support for Nullable types in the ClickHouse ODBC driver (`ODBCDriver2` output format). [#2834](https://github.com/ClickHouse/ClickHouse/pull/2834) * Support for `UUID` in the key columns. -### Improvements: +#### Improvements: * Clusters can be removed without restarting the server when they are deleted from the config files. [#2777](https://github.com/ClickHouse/ClickHouse/pull/2777) * External dictionaries can be removed without restarting the server when they are removed from config files. [#2779](https://github.com/ClickHouse/ClickHouse/pull/2779) @@ -2360,7 +2386,7 @@ This release contains exactly the same set of patches as 19.3.6. * Added the `prefer_localhost_replica` setting for disabling the preference for a local replica and going to a local replica without inter-process interaction. [#2832](https://github.com/ClickHouse/ClickHouse/pull/2832) * The `quantileExact` aggregate function returns `nan` in the case of aggregation on an empty `Float32` or `Float64` set. [Sundy Li](https://github.com/ClickHouse/ClickHouse/pull/2855) -### Bug fixes: +#### Bug fixes: * Removed unnecessary escaping of the connection string parameters for ODBC, which made it impossible to establish a connection. This error occurred in version 18.6.0. * Fixed the logic for processing `REPLACE PARTITION` commands in the replication queue. If there are two `REPLACE` commands for the same partition, the incorrect logic could cause one of them to remain in the replication queue and not be executed. [#2814](https://github.com/ClickHouse/ClickHouse/pull/2814) @@ -2371,11 +2397,11 @@ This release contains exactly the same set of patches as 19.3.6. * Fixed incorrect clickhouse-client response code in case of a query error. * Fixed incorrect behavior of materialized views containing DISTINCT. [#2795](https://github.com/ClickHouse/ClickHouse/issues/2795) -### Backward incompatible changes +#### Backward incompatible changes * Removed support for CHECK TABLE queries for Distributed tables. -### Build changes: +#### Build changes: * The allocator has been replaced: `jemalloc` is now used instead of `tcmalloc`. In some scenarios, this increases speed up to 20%. However, there are queries that have slowed by up to 20%. Memory consumption has been reduced by approximately 10% in some scenarios, with improved stability. With highly competitive loads, CPU usage in userspace and in system shows just a slight increase. [#2773](https://github.com/ClickHouse/ClickHouse/pull/2773) * Use of libressl from a submodule. [#1983](https://github.com/ClickHouse/ClickHouse/pull/1983) [#2807](https://github.com/ClickHouse/ClickHouse/pull/2807) @@ -2383,37 +2409,43 @@ This release contains exactly the same set of patches as 19.3.6. * Use of mariadb-connector-c from a submodule. [#2785](https://github.com/ClickHouse/ClickHouse/pull/2785) * Added functional test files to the repository that depend on the availability of test data (for the time being, without the test data itself). -## ClickHouse release 18.6.0, 2018-08-02 +## ClickHouse release 18.6 -### New features: +### ClickHouse release 18.6.0, 2018-08-02 + +#### New features: * Added support for ON expressions for the JOIN ON syntax: `JOIN ON Expr([table.]column ...) = Expr([table.]column, ...) [AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]` The expression must be a chain of equalities joined by the AND operator. Each side of the equality can be an arbitrary expression over the columns of one of the tables. The use of fully qualified column names is supported (`table.name`, `database.table.name`, `table_alias.name`, `subquery_alias.name`) for the right table. [#2742](https://github.com/ClickHouse/ClickHouse/pull/2742) * HTTPS can be enabled for replication. [#2760](https://github.com/ClickHouse/ClickHouse/pull/2760) -### Improvements: +#### Improvements: * The server passes the patch component of its version to the client. Data about the patch version component is in `system.processes` and `query_log`. [#2646](https://github.com/ClickHouse/ClickHouse/pull/2646) -## ClickHouse release 18.5.1, 2018-07-31 +## ClickHouse release 18.5 -### New features: +### ClickHouse release 18.5.1, 2018-07-31 + +#### New features: * Added the hash function `murmurHash2_32` [#2756](https://github.com/ClickHouse/ClickHouse/pull/2756). -### Improvements: +#### Improvements: * Now you can use the `from_env` [#2741](https://github.com/ClickHouse/ClickHouse/pull/2741) attribute to set values in config files from environment variables. * Added case-insensitive versions of the `coalesce`, `ifNull`, and `nullIf functions` [#2752](https://github.com/ClickHouse/ClickHouse/pull/2752). -### Bug fixes: +#### Bug fixes: * Fixed a possible bug when starting a replica [#2759](https://github.com/ClickHouse/ClickHouse/pull/2759). -## ClickHouse release 18.4.0, 2018-07-28 +## ClickHouse release 18.4 -### New features: +### ClickHouse release 18.4.0, 2018-07-28 + +#### New features: * Added system tables: `formats`, `data_type_families`, `aggregate_function_combinators`, `table_functions`, `table_engines`, `collations` [#2721](https://github.com/ClickHouse/ClickHouse/pull/2721). * Added the ability to use a table function instead of a table as an argument of a `remote` or `cluster table function` [#2708](https://github.com/ClickHouse/ClickHouse/pull/2708). @@ -2421,26 +2453,28 @@ The expression must be a chain of equalities joined by the AND operator. Each si * The `has` function now allows searching for a numeric value in an array of `Enum` values [Maxim Khrisanfov](https://github.com/ClickHouse/ClickHouse/pull/2699). * Support for adding arbitrary message separators when reading from `Kafka` [Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2701). -### Improvements: +#### Improvements: * The `ALTER TABLE t DELETE WHERE` query does not rewrite data parts that were not affected by the WHERE condition [#2694](https://github.com/ClickHouse/ClickHouse/pull/2694). * The `use_minimalistic_checksums_in_zookeeper` option for `ReplicatedMergeTree` tables is enabled by default. This setting was added in version 1.1.54378, 2018-04-16. Versions that are older than 1.1.54378 can no longer be installed. * Support for running `KILL` and `OPTIMIZE` queries that specify `ON CLUSTER` [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2689). -### Bug fixes: +#### Bug fixes: * Fixed the error `Column ... is not under an aggregate function and not in GROUP BY` for aggregation with an IN expression. This bug appeared in version 18.1.0. ([bbdd780b](https://github.com/ClickHouse/ClickHouse/commit/bbdd780be0be06a0f336775941cdd536878dd2c2)) * Fixed a bug in the `windowFunnel aggregate function` [Winter Zhang](https://github.com/ClickHouse/ClickHouse/pull/2735). * Fixed a bug in the `anyHeavy` aggregate function ([a2101df2](https://github.com/ClickHouse/ClickHouse/commit/a2101df25a6a0fba99aa71f8793d762af2b801ee)) * Fixed server crash when using the `countArray()` aggregate function. -### Backward incompatible changes: +#### Backward incompatible changes: * Parameters for `Kafka` engine was changed from `Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format[, kafka_schema, kafka_num_consumers])` to `Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format[, kafka_row_delimiter, kafka_schema, kafka_num_consumers])`. If your tables use `kafka_schema` or `kafka_num_consumers` parameters, you have to manually edit the metadata files `path/metadata/database/table.sql` and add `kafka_row_delimiter` parameter with `''` value. -## ClickHouse release 18.1.0, 2018-07-23 +## ClickHouse release 18.1 -### New features: +### ClickHouse release 18.1.0, 2018-07-23 + +#### New features: * Support for the `ALTER TABLE t DELETE WHERE` query for non-replicated MergeTree tables ([#2634](https://github.com/ClickHouse/ClickHouse/pull/2634)). * Support for arbitrary types for the `uniq*` family of aggregate functions ([#2010](https://github.com/ClickHouse/ClickHouse/issues/2010)). @@ -2449,13 +2483,13 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Added the `arrayDistinct` function ([#2670](https://github.com/ClickHouse/ClickHouse/pull/2670)). * The SummingMergeTree engine can now work with AggregateFunction type columns ([Constantin S. Pan](https://github.com/ClickHouse/ClickHouse/pull/2566)). -### Improvements: +#### Improvements: * Changed the numbering scheme for release versions. Now the first part contains the year of release (A.D., Moscow timezone, minus 2000), the second part contains the number for major changes (increases for most releases), and the third part is the patch version. Releases are still backward compatible, unless otherwise stated in the changelog. * Faster conversions of floating-point numbers to a string ([Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2664)). * If some rows were skipped during an insert due to parsing errors (this is possible with the `input_allow_errors_num` and `input_allow_errors_ratio` settings enabled), the number of skipped rows is now written to the server log ([Leonardo Cecchi](https://github.com/ClickHouse/ClickHouse/pull/2669)). -### Bug fixes: +#### Bug fixes: * Fixed the TRUNCATE command for temporary tables ([Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2624)). * Fixed a rare deadlock in the ZooKeeper client library that occurred when there was a network error while reading the response ([c315200](https://github.com/ClickHouse/ClickHouse/commit/c315200e64b87e44bdf740707fc857d1fdf7e947)). @@ -2466,18 +2500,20 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Fixed incompatibility between servers with different versions in distributed queries that use a `CAST` function that isn't in uppercase letters ([fe8c4d6](https://github.com/ClickHouse/ClickHouse/commit/fe8c4d64e434cacd4ceef34faa9005129f2190a5)). * Added missing quoting of identifiers for queries to an external DBMS ([#2635](https://github.com/ClickHouse/ClickHouse/issues/2635)). -### Backward incompatible changes: +#### Backward incompatible changes: * Converting a string containing the number zero to DateTime does not work. Example: `SELECT toDateTime('0')`. This is also the reason that `DateTime DEFAULT '0'` does not work in tables, as well as `0` in dictionaries. Solution: replace `0` with `0000-00-00 00:00:00`. -## ClickHouse release 1.1.54394, 2018-07-12 +## ClickHouse release 1.1 -### New features: +### ClickHouse release 1.1.54394, 2018-07-12 + +#### New features: * Added the `histogram` aggregate function ([Mikhail Surin](https://github.com/ClickHouse/ClickHouse/pull/2521)). * Now `OPTIMIZE TABLE ... FINAL` can be used without specifying partitions for `ReplicatedMergeTree` ([Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2600)). -### Bug fixes: +#### Bug fixes: * Fixed a problem with a very small timeout for sockets (one second) for reading and writing when sending and downloading replicated data, which made it impossible to download larger parts if there is a load on the network or disk (it resulted in cyclical attempts to download parts). This error occurred in version 1.1.54388. * Fixed issues when using chroot in ZooKeeper if you inserted duplicate data blocks in the table. @@ -2486,15 +2522,15 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Fixed how an empty `TinyLog` table works after inserting an empty data block ([#2563](https://github.com/ClickHouse/ClickHouse/issues/2563)). * The `system.zookeeper` table works if the value of the node in ZooKeeper is NULL. -## ClickHouse release 1.1.54390, 2018-07-06 +### ClickHouse release 1.1.54390, 2018-07-06 -### New features: +#### New features: * Queries can be sent in `multipart/form-data` format (in the `query` field), which is useful if external data is also sent for query processing ([Olga Hvostikova](https://github.com/ClickHouse/ClickHouse/pull/2490)). * Added the ability to enable or disable processing single or double quotes when reading data in CSV format. You can configure this in the `format_csv_allow_single_quotes` and `format_csv_allow_double_quotes` settings ([Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2574)). * Now `OPTIMIZE TABLE ... FINAL` can be used without specifying the partition for non-replicated variants of `MergeTree` ([Amos Bird](https://github.com/ClickHouse/ClickHouse/pull/2599)). -### Improvements: +#### Improvements: * Improved performance, reduced memory consumption, and correct memory consumption tracking with use of the IN operator when a table index could be used ([#2584](https://github.com/ClickHouse/ClickHouse/pull/2584)). * Removed redundant checking of checksums when adding a data part. This is important when there are a large number of replicas, because in these cases the total number of checks was equal to N^2. @@ -2504,7 +2540,7 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Faster selection of data parts for merging in `ReplicatedMergeTree` tables. Faster recovery of the ZooKeeper session ([#2597](https://github.com/ClickHouse/ClickHouse/pull/2597)). * The `format_version.txt` file for `MergeTree` tables is re-created if it is missing, which makes sense if ClickHouse is launched after copying the directory structure without files ([Ciprian Hacman](https://github.com/ClickHouse/ClickHouse/pull/2593)). -### Bug fixes: +#### Bug fixes: * Fixed a bug when working with ZooKeeper that could make it impossible to recover the session and readonly states of tables before restarting the server. * Fixed a bug when working with ZooKeeper that could result in old nodes not being deleted if the session is interrupted. @@ -2514,13 +2550,13 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Fixed switching to the default database when reconnecting the client ([#2583](https://github.com/ClickHouse/ClickHouse/pull/2583)). * Fixed a bug that occurred when the `use_index_for_in_with_subqueries` setting was disabled. -### Security fix: +#### Security fix: * Sending files is no longer possible when connected to MySQL (`LOAD DATA LOCAL INFILE`). -## ClickHouse release 1.1.54388, 2018-06-28 +### ClickHouse release 1.1.54388, 2018-06-28 -### New features: +#### New features: * Support for the `ALTER TABLE t DELETE WHERE` query for replicated tables. Added the `system.mutations` table to track progress of this type of queries. * Support for the `ALTER TABLE t [REPLACE|ATTACH] PARTITION` query for \*MergeTree tables. @@ -2538,12 +2574,12 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Added the `date_time_input_format` setting. If you switch this setting to `'best_effort'`, DateTime values will be read in a wide range of formats. * Added the `clickhouse-obfuscator` utility for data obfuscation. Usage example: publishing data used in performance tests. -### Experimental features: +#### Experimental features: * Added the ability to calculate `and` arguments only where they are needed ([Anastasia Tsarkova](https://github.com/ClickHouse/ClickHouse/pull/2272)) * JIT compilation to native code is now available for some expressions ([pyos](https://github.com/ClickHouse/ClickHouse/pull/2277)). -### Bug fixes: +#### Bug fixes: * Duplicates no longer appear for a query with `DISTINCT` and `ORDER BY`. * Queries with `ARRAY JOIN` and `arrayFilter` no longer return an incorrect result. @@ -2565,7 +2601,7 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Fixed SSRF in the remote() table function. * Fixed exit behavior of `clickhouse-client` in multiline mode ([#2510](https://github.com/ClickHouse/ClickHouse/issues/2510)). -### Improvements: +#### Improvements: * Background tasks in replicated tables are now performed in a thread pool instead of in separate threads ([Silviu Caragea](https://github.com/ClickHouse/ClickHouse/pull/1722)). * Improved LZ4 compression performance. @@ -2578,7 +2614,7 @@ The expression must be a chain of equalities joined by the AND operator. Each si * When calculating the number of available CPU cores, limits on cgroups are now taken into account ([Atri Sharma](https://github.com/ClickHouse/ClickHouse/pull/2325)). * Added chown for config directories in the systemd config file ([Mikhail Shiryaev](https://github.com/ClickHouse/ClickHouse/pull/2421)). -### Build changes: +#### Build changes: * The gcc8 compiler can be used for builds. * Added the ability to build llvm from submodule. @@ -2589,41 +2625,41 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Added the ability to use the libtinfo library instead of libtermcap ([Georgy Kondratiev](https://github.com/ClickHouse/ClickHouse/pull/2519)). * Fixed a header file conflict in Fedora Rawhide ([#2520](https://github.com/ClickHouse/ClickHouse/issues/2520)). -### Backward incompatible changes: +#### Backward incompatible changes: * Removed escaping in `Vertical` and `Pretty*` formats and deleted the `VerticalRaw` format. * If servers with version 1.1.54388 (or newer) and servers with an older version are used simultaneously in a distributed query and the query has the `cast(x, 'Type')` expression without the `AS` keyword and doesn't have the word `cast` in uppercase, an exception will be thrown with a message like `Not found column cast(0, 'UInt8') in block`. Solution: Update the server on the entire cluster. -## ClickHouse release 1.1.54385, 2018-06-01 +### ClickHouse release 1.1.54385, 2018-06-01 -### Bug fixes: +#### Bug fixes: * Fixed an error that in some cases caused ZooKeeper operations to block. -## ClickHouse release 1.1.54383, 2018-05-22 +### ClickHouse release 1.1.54383, 2018-05-22 -### Bug fixes: +#### Bug fixes: * Fixed a slowdown of replication queue if a table has many replicas. -## ClickHouse release 1.1.54381, 2018-05-14 +### ClickHouse release 1.1.54381, 2018-05-14 -### Bug fixes: +#### Bug fixes: * Fixed a nodes leak in ZooKeeper when ClickHouse loses connection to ZooKeeper server. -## ClickHouse release 1.1.54380, 2018-04-21 +### ClickHouse release 1.1.54380, 2018-04-21 -### New features: +#### New features: * Added the table function `file(path, format, structure)`. An example reading bytes from `/dev/urandom`: `ln -s /dev/urandom /var/lib/clickhouse/user_files/random``clickhouse-client -q "SELECT * FROM file('random', 'RowBinary', 'd UInt8') LIMIT 10"`. -### Improvements: +#### Improvements: * Subqueries can be wrapped in `()` brackets to enhance query readability. For example: `(SELECT 1) UNION ALL (SELECT 1)`. * Simple `SELECT` queries from the `system.processes` table are not included in the `max_concurrent_queries` limit. -### Bug fixes: +#### Bug fixes: * Fixed incorrect behavior of the `IN` operator when select from `MATERIALIZED VIEW`. * Fixed incorrect filtering by partition index in expressions like `partition_key_column IN (...)`. @@ -2632,13 +2668,13 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Fixed freezing of `KILL QUERY`. * Fixed an error in ZooKeeper client library which led to loss of watches, freezing of distributed DDL queue, and slowdowns in the replication queue if a non-empty `chroot` prefix is used in the ZooKeeper configuration. -### Backward incompatible changes: +#### Backward incompatible changes: * Removed support for expressions like `(a, b) IN (SELECT (a, b))` (you can use the equivalent expression `(a, b) IN (SELECT a, b)`). In previous releases, these expressions led to undetermined `WHERE` filtering or caused errors. -## ClickHouse release 1.1.54378, 2018-04-16 +### ClickHouse release 1.1.54378, 2018-04-16 -### New features: +#### New features: * Logging level can be changed without restarting the server. * Added the `SHOW CREATE DATABASE` query. @@ -2652,7 +2688,7 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Multiple comma-separated `topics` can be specified for the `Kafka` engine (Tobias Adamson) * When a query is stopped by `KILL QUERY` or `replace_running_query`, the client receives the `Query was canceled` exception instead of an incomplete result. -### Improvements: +#### Improvements: * `ALTER TABLE ... DROP/DETACH PARTITION` queries are run at the front of the replication queue. * `SELECT ... FINAL` and `OPTIMIZE ... FINAL` can be used even when the table has a single data part. @@ -2663,7 +2699,7 @@ The expression must be a chain of equalities joined by the AND operator. Each si * More robust crash recovery for asynchronous insertion into `Distributed` tables. * The return type of the `countEqual` function changed from `UInt32` to `UInt64` (谢磊). -### Bug fixes: +#### Bug fixes: * Fixed an error with `IN` when the left side of the expression is `Nullable`. * Correct results are now returned when using tuples with `IN` when some of the tuple components are in the table index. @@ -2679,30 +2715,30 @@ The expression must be a chain of equalities joined by the AND operator. Each si * `SummingMergeTree` now works correctly for summation of nested data structures with a composite key. * Fixed the possibility of a race condition when choosing the leader for `ReplicatedMergeTree` tables. -### Build changes: +#### Build changes: * The build supports `ninja` instead of `make` and uses `ninja` by default for building releases. * Renamed packages: `clickhouse-server-base` in `clickhouse-common-static`; `clickhouse-server-common` in `clickhouse-server`; `clickhouse-common-dbg` in `clickhouse-common-static-dbg`. To install, use `clickhouse-server clickhouse-client`. Packages with the old names will still load in the repositories for backward compatibility. -### Backward incompatible changes: +#### Backward incompatible changes: * Removed the special interpretation of an IN expression if an array is specified on the left side. Previously, the expression `arr IN (set)` was interpreted as "at least one `arr` element belongs to the `set`". To get the same behavior in the new version, write `arrayExists(x -> x IN (set), arr)`. * Disabled the incorrect use of the socket option `SO_REUSEPORT`, which was incorrectly enabled by default in the Poco library. Note that on Linux there is no longer any reason to simultaneously specify the addresses `::` and `0.0.0.0` for listen – use just `::`, which allows listening to the connection both over IPv4 and IPv6 (with the default kernel config settings). You can also revert to the behavior from previous versions by specifying `1` in the config. -## ClickHouse release 1.1.54370, 2018-03-16 +### ClickHouse release 1.1.54370, 2018-03-16 -### New features: +#### New features: * Added the `system.macros` table and auto updating of macros when the config file is changed. * Added the `SYSTEM RELOAD CONFIG` query. * Added the `maxIntersections(left_col, right_col)` aggregate function, which returns the maximum number of simultaneously intersecting intervals `[left; right]`. The `maxIntersectionsPosition(left, right)` function returns the beginning of the "maximum" interval. ([Michael Furmur](https://github.com/ClickHouse/ClickHouse/pull/2012)). -### Improvements: +#### Improvements: * When inserting data in a `Replicated` table, fewer requests are made to `ZooKeeper` (and most of the user-level errors have disappeared from the `ZooKeeper` log). * Added the ability to create aliases for data sets. Example: `WITH (1, 2, 3) AS set SELECT number IN set FROM system.numbers LIMIT 10`. -### Bug fixes: +#### Bug fixes: * Fixed the `Illegal PREWHERE` error when reading from Merge tables for `Distributed`tables. * Added fixes that allow you to start clickhouse-server in IPv4-only Docker containers. @@ -2716,9 +2752,9 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Restored the behavior for queries like `SELECT * FROM remote('server2', default.table) WHERE col IN (SELECT col2 FROM default.table)` when the right side of the `IN` should use a remote `default.table` instead of a local one. This behavior was broken in version 1.1.54358. * Removed extraneous error-level logging of `Not found column ... in block`. -## Clickhouse Release 1.1.54362, 2018-03-11 +### Clickhouse Release 1.1.54362, 2018-03-11 -### New features: +#### New features: * Aggregation without `GROUP BY` for an empty set (such as `SELECT count(*) FROM table WHERE 0`) now returns a result with one row with null values for aggregate functions, in compliance with the SQL standard. To restore the old behavior (return an empty result), set `empty_result_for_aggregation_by_empty_set` to 1. * Added type conversion for `UNION ALL`. Different alias names are allowed in `SELECT` positions in `UNION ALL`, in compliance with the SQL standard. @@ -2756,7 +2792,7 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Added the `odbc_default_field_size` option, which allows you to extend the maximum size of the value loaded from an ODBC source (by default, it is 1024). * The `system.processes` table and `SHOW PROCESSLIST` now have the `is_cancelled` and `peak_memory_usage` columns. -### Improvements: +#### Improvements: * Limits and quotas on the result are no longer applied to intermediate data for `INSERT SELECT` queries or for `SELECT` subqueries. * Fewer false triggers of `force_restore_data` when checking the status of `Replicated` tables when the server starts. @@ -2772,7 +2808,7 @@ The expression must be a chain of equalities joined by the AND operator. Each si * `Enum` values can be used in `min`, `max`, `sum` and some other functions. In these cases, it uses the corresponding numeric values. This feature was previously available but was lost in the release 1.1.54337. * Added `max_expanded_ast_elements` to restrict the size of the AST after recursively expanding aliases. -### Bug fixes: +#### Bug fixes: * Fixed cases when unnecessary columns were removed from subqueries in error, or not removed from subqueries containing `UNION ALL`. * Fixed a bug in merges for `ReplacingMergeTree` tables. @@ -2800,19 +2836,19 @@ The expression must be a chain of equalities joined by the AND operator. Each si * Prohibited the use of queries with `UNION ALL` in a `MATERIALIZED VIEW`. * Fixed an error during initialization of the `part_log` system table when the server starts (by default, `part_log` is disabled). -### Backward incompatible changes: +#### Backward incompatible changes: * Removed the `distributed_ddl_allow_replicated_alter` option. This behavior is enabled by default. * Removed the `strict_insert_defaults` setting. If you were using this functionality, write to `clickhouse-feedback@yandex-team.com`. * Removed the `UnsortedMergeTree` engine. -## Clickhouse Release 1.1.54343, 2018-02-05 +### Clickhouse Release 1.1.54343, 2018-02-05 * Added macros support for defining cluster names in distributed DDL queries and constructors of Distributed tables: `CREATE TABLE distr ON CLUSTER '{cluster}' (...) ENGINE = Distributed('{cluster}', 'db', 'table')`. * Now queries like `SELECT ... FROM table WHERE expr IN (subquery)` are processed using the `table` index. * Improved processing of duplicates when inserting to Replicated tables, so they no longer slow down execution of the replication queue. -## Clickhouse Release 1.1.54342, 2018-01-22 +### Clickhouse Release 1.1.54342, 2018-01-22 This release contains bug fixes for the previous release 1.1.54337: @@ -2824,9 +2860,9 @@ This release contains bug fixes for the previous release 1.1.54337: * Buffer tables now work correctly when MATERIALIZED columns are present in the destination table (by zhang2014). * Fixed a bug in implementation of NULL. -## Clickhouse Release 1.1.54337, 2018-01-18 +### Clickhouse Release 1.1.54337, 2018-01-18 -### New features: +#### New features: * Added support for storage of multi-dimensional arrays and tuples (`Tuple` data type) in tables. * Support for table functions for `DESCRIBE` and `INSERT` queries. Added support for subqueries in `DESCRIBE`. Examples: `DESC TABLE remote('host', default.hits)`; `DESC TABLE (SELECT 1)`; `INSERT INTO TABLE FUNCTION remote('host', default.hits)`. Support for `INSERT INTO TABLE` in addition to `INSERT INTO`. @@ -2857,7 +2893,7 @@ This release contains bug fixes for the previous release 1.1.54337: * Added the `--silent` option for the `clickhouse-local` tool. It suppresses printing query execution info in stderr. * Added support for reading values of type `Date` from text in a format where the month and/or day of the month is specified using a single digit instead of two digits (Amos Bird). -### Performance optimizations: +#### Performance optimizations: * Improved performance of aggregate functions `min`, `max`, `any`, `anyLast`, `anyHeavy`, `argMin`, `argMax` from string arguments. * Improved performance of the functions `isInfinite`, `isFinite`, `isNaN`, `roundToExp2`. @@ -2866,7 +2902,7 @@ This release contains bug fixes for the previous release 1.1.54337: * Lowered memory usage for `JOIN` in the case when the left and right parts have columns with identical names that are not contained in `USING` . * Improved performance of aggregate functions `varSamp`, `varPop`, `stddevSamp`, `stddevPop`, `covarSamp`, `covarPop`, `corr` by reducing computational stability. The old functions are available under the names `varSampStable`, `varPopStable`, `stddevSampStable`, `stddevPopStable`, `covarSampStable`, `covarPopStable`, `corrStable`. -### Bug fixes: +#### Bug fixes: * Fixed data deduplication after running a `DROP` or `DETACH PARTITION` query. In the previous version, dropping a partition and inserting the same data again was not working because inserted blocks were considered duplicates. * Fixed a bug that could lead to incorrect interpretation of the `WHERE` clause for ` CREATE MATERIALIZED VIEW` queries with `POPULATE` . @@ -2905,7 +2941,7 @@ This release contains bug fixes for the previous release 1.1.54337: * Fixed the ` SYSTEM DROP DNS CACHE` query: the cache was flushed but addresses of cluster nodes were not updated. * Fixed the behavior of ` MATERIALIZED VIEW` after executing ` DETACH TABLE` for the table under the view (Marek Vavruša). -### Build improvements: +#### Build improvements: * The `pbuilder` tool is used for builds. The build process is almost completely independent of the build host environment. * A single build is used for different OS versions. Packages and binaries have been made compatible with a wide range of Linux systems. @@ -2919,7 +2955,7 @@ This release contains bug fixes for the previous release 1.1.54337: * Removed usage of GNU extensions from the code. Enabled the `-Wextra` option. When building with `clang` the default is `libc++` instead of `libstdc++`. * Extracted `clickhouse_parsers` and `clickhouse_common_io` libraries to speed up builds of various tools. -### Backward incompatible changes: +#### Backward incompatible changes: * The format for marks in `Log` type tables that contain `Nullable` columns was changed in a backward incompatible way. If you have these tables, you should convert them to the `TinyLog` type before starting up the new server version. To do this, replace `ENGINE = Log` with `ENGINE = TinyLog` in the corresponding `.sql` file in the `metadata` directory. If your table doesn't have `Nullable` columns or if the type of your table is not `Log`, then you don't need to do anything. * Removed the `experimental_allow_extended_storage_definition_syntax` setting. Now this feature is enabled by default. @@ -2930,18 +2966,18 @@ This release contains bug fixes for the previous release 1.1.54337: * In previous server versions there was an undocumented feature: if an aggregate function depends on parameters, you can still specify it without parameters in the AggregateFunction data type. Example: `AggregateFunction(quantiles, UInt64)` instead of `AggregateFunction(quantiles(0.5, 0.9), UInt64)`. This feature was lost. Although it was undocumented, we plan to support it again in future releases. * Enum data types cannot be used in min/max aggregate functions. This ability will be returned in the next release. -### Please note when upgrading: +#### Please note when upgrading: * When doing a rolling update on a cluster, at the point when some of the replicas are running the old version of ClickHouse and some are running the new version, replication is temporarily stopped and the message ` unknown parameter 'shard'` appears in the log. Replication will continue after all replicas of the cluster are updated. * If different versions of ClickHouse are running on the cluster servers, it is possible that distributed queries using the following functions will have incorrect results: `varSamp`, `varPop`, `stddevSamp`, `stddevPop`, `covarSamp`, `covarPop`, `corr`. You should update all cluster nodes. -## ClickHouse release 1.1.54327, 2017-12-21 +### ClickHouse release 1.1.54327, 2017-12-21 This release contains bug fixes for the previous release 1.1.54318: * Fixed bug with possible race condition in replication that could lead to data loss. This issue affects versions 1.1.54310 and 1.1.54318. If you use one of these versions with Replicated tables, the update is strongly recommended. This issue shows in logs in Warning messages like ` Part ... from own log doesn't exist.` The issue is relevant even if you don't see these messages in logs. -## ClickHouse release 1.1.54318, 2017-11-30 +### ClickHouse release 1.1.54318, 2017-11-30 This release contains bug fixes for the previous release 1.1.54310: @@ -2951,9 +2987,9 @@ This release contains bug fixes for the previous release 1.1.54310: * Fixed an issue that was causing the replication queue to stop running * Fixed rotation and archiving of server logs -## ClickHouse release 1.1.54310, 2017-11-01 +### ClickHouse release 1.1.54310, 2017-11-01 -### New features: +#### New features: * Custom partitioning key for the MergeTree family of table engines. * [Kafka](https://clickhouse.yandex/docs/en/operations/table_engines/kafka/) table engine. @@ -2970,13 +3006,13 @@ This release contains bug fixes for the previous release 1.1.54310: * Added support for the Cap'n Proto input format. * You can now customize compression level when using the zstd algorithm. -### Backward incompatible changes: +#### Backward incompatible changes: * Creation of temporary tables with an engine other than Memory is not allowed. * Explicit creation of tables with the View or MaterializedView engine is not allowed. * During table creation, a new check verifies that the sampling key expression is included in the primary key. -### Bug fixes: +#### Bug fixes: * Fixed hangups when synchronously inserting into a Distributed table. * Fixed nonatomic adding and removing of parts in Replicated tables. @@ -2987,17 +3023,17 @@ This release contains bug fixes for the previous release 1.1.54310: * Fixed hangups when the disk volume containing server logs is full. * Fixed an overflow in the toRelativeWeekNum function for the first week of the Unix epoch. -### Build improvements: +#### Build improvements: * Several third-party libraries (notably Poco) were updated and converted to git submodules. -## ClickHouse release 1.1.54304, 2017-10-19 +### ClickHouse release 1.1.54304, 2017-10-19 -### New features: +#### New features: * TLS support in the native protocol (to enable, set `tcp_ssl_port` in `config.xml` ). -### Bug fixes: +#### Bug fixes: * `ALTER` for replicated tables now tries to start running as soon as possible. * Fixed crashing when reading data with the setting `preferred_block_size_bytes=0.` @@ -3011,9 +3047,9 @@ This release contains bug fixes for the previous release 1.1.54310: * Users are updated correctly with invalid `users.xml` * Correct handling when an executable dictionary returns a non-zero response code. -## ClickHouse release 1.1.54292, 2017-09-20 +### ClickHouse release 1.1.54292, 2017-09-20 -### New features: +#### New features: * Added the `pointInPolygon` function for working with coordinates on a coordinate plane. * Added the `sumMap` aggregate function for calculating the sum of arrays, similar to `SummingMergeTree`. @@ -3021,7 +3057,7 @@ This release contains bug fixes for the previous release 1.1.54310: * The ClickHouse executable file is now less dependent on the libc version. The same ClickHouse executable file can run on a wide variety of Linux systems. There is still a dependency when using compiled queries (with the setting ` compile = 1` , which is not used by default). * Reduced the time needed for dynamic compilation of queries. -### Bug fixes: +#### Bug fixes: * Fixed an error that sometimes produced ` part ... intersects previous part` messages and weakened replica consistency. * Fixed an error that caused the server to lock up if ZooKeeper was unavailable during shutdown. @@ -3030,9 +3066,9 @@ This release contains bug fixes for the previous release 1.1.54310: * Fixed an error in the concat function that occurred if the first column in a block has the Array type. * Progress is now displayed correctly in the system.merges table. -## ClickHouse release 1.1.54289, 2017-09-13 +### ClickHouse release 1.1.54289, 2017-09-13 -### New features: +#### New features: * `SYSTEM` queries for server administration: `SYSTEM RELOAD DICTIONARY`, `SYSTEM RELOAD DICTIONARIES`, `SYSTEM DROP DNS CACHE`, `SYSTEM SHUTDOWN`, `SYSTEM KILL`. * Added functions for working with arrays: `concat`, `arraySlice`, `arrayPushBack`, `arrayPushFront`, `arrayPopBack`, `arrayPopFront`. @@ -3048,7 +3084,7 @@ This release contains bug fixes for the previous release 1.1.54310: * Option to set `umask` in the config file. * Improved performance for queries with `DISTINCT` . -### Bug fixes: +#### Bug fixes: * Improved the process for deleting old nodes in ZooKeeper. Previously, old nodes sometimes didn't get deleted if there were very frequent inserts, which caused the server to be slow to shut down, among other things. * Fixed randomization when choosing hosts for the connection to ZooKeeper. @@ -3062,21 +3098,21 @@ This release contains bug fixes for the previous release 1.1.54310: * Resolved the appearance of zombie processes when using a dictionary with an `executable` source. * Fixed segfault for the HEAD query. -### Improved workflow for developing and assembling ClickHouse: +#### Improved workflow for developing and assembling ClickHouse: * You can use `pbuilder` to build ClickHouse. * You can use `libc++` instead of `libstdc++` for builds on Linux. * Added instructions for using static code analysis tools: `Coverage`, `clang-tidy`, `cppcheck`. -### Please note when upgrading: +#### Please note when upgrading: * There is now a higher default value for the MergeTree setting `max_bytes_to_merge_at_max_space_in_pool` (the maximum total size of data parts to merge, in bytes): it has increased from 100 GiB to 150 GiB. This might result in large merges running after the server upgrade, which could cause an increased load on the disk subsystem. If the free space available on the server is less than twice the total amount of the merges that are running, this will cause all other merges to stop running, including merges of small data parts. As a result, INSERT queries will fail with the message "Merges are processing significantly slower than inserts." Use the ` SELECT * FROM system.merges` query to monitor the situation. You can also check the `DiskSpaceReservedForMerge` metric in the `system.metrics` table, or in Graphite. You don't need to do anything to fix this, since the issue will resolve itself once the large merges finish. If you find this unacceptable, you can restore the previous value for the `max_bytes_to_merge_at_max_space_in_pool` setting. To do this, go to the section in config.xml, set ```107374182400` and restart the server. -## ClickHouse release 1.1.54284, 2017-08-29 +### ClickHouse release 1.1.54284, 2017-08-29 * This is a bugfix release for the previous 1.1.54282 release. It fixes leaks in the parts directory in ZooKeeper. -## ClickHouse release 1.1.54282, 2017-08-23 +### ClickHouse release 1.1.54282, 2017-08-23 This release contains bug fixes for the previous release 1.1.54276: @@ -3084,9 +3120,9 @@ This release contains bug fixes for the previous release 1.1.54276: * Fixed parsing when inserting in RowBinary format if input data starts with';'. * Errors during runtime compilation of certain aggregate functions (e.g. `groupArray()`). -## Clickhouse Release 1.1.54276, 2017-08-16 +### Clickhouse Release 1.1.54276, 2017-08-16 -### New features: +#### New features: * Added an optional WITH section for a SELECT query. Example query: `WITH 1+1 AS a SELECT a, a*a` * INSERT can be performed synchronously in a Distributed table: OK is returned only after all the data is saved on all the shards. This is activated by the setting insert_distributed_sync=1. @@ -3097,7 +3133,7 @@ This release contains bug fixes for the previous release 1.1.54276: * Added support for non-constant arguments and negative offsets in the function `substring(str, pos, len).` * Added the max_size parameter for the `groupArray(max_size)(column)` aggregate function, and optimized its performance. -### Main changes: +#### Main changes: * Security improvements: all server files are created with 0640 permissions (can be changed via config parameter). * Improved error messages for queries with invalid syntax. @@ -3105,11 +3141,11 @@ This release contains bug fixes for the previous release 1.1.54276: * Significantly increased the performance of data merges for the ReplacingMergeTree engine. * Improved performance for asynchronous inserts from a Distributed table by combining multiple source inserts. To enable this functionality, use the setting distributed_directory_monitor_batch_inserts=1. -### Backward incompatible changes: +#### Backward incompatible changes: * Changed the binary format of aggregate states of `groupArray(array_column)` functions for arrays. -### Complete list of changes: +#### Complete list of changes: * Added the `output_format_json_quote_denormals` setting, which enables outputting nan and inf values in JSON format. * Optimized stream allocation when reading from a Distributed table. @@ -3128,7 +3164,7 @@ This release contains bug fixes for the previous release 1.1.54276: * It is possible to connect to MySQL through a socket in the file system. * The system.parts table has a new column with information about the size of marks, in bytes. -### Bug fixes: +#### Bug fixes: * Distributed tables using a Merge table now work correctly for a SELECT query with a condition on the `_table` field. * Fixed a rare race condition in ReplicatedMergeTree when checking data parts. @@ -3152,15 +3188,15 @@ This release contains bug fixes for the previous release 1.1.54276: * Fixed the "Cannot mremap" error when using arrays in IN and JOIN clauses with more than 2 billion elements. * Fixed the failover for dictionaries with MySQL as the source. -### Improved workflow for developing and assembling ClickHouse: +#### Improved workflow for developing and assembling ClickHouse: * Builds can be assembled in Arcadia. * You can use gcc 7 to compile ClickHouse. * Parallel builds using ccache+distcc are faster now. -## ClickHouse release 1.1.54245, 2017-07-04 +### ClickHouse release 1.1.54245, 2017-07-04 -### New features: +#### New features: * Distributed DDL (for example, `CREATE TABLE ON CLUSTER`) * The replicated query `ALTER TABLE CLEAR COLUMN IN PARTITION.` @@ -3172,16 +3208,16 @@ This release contains bug fixes for the previous release 1.1.54276: * Sessions in the HTTP interface. * The OPTIMIZE query for a Replicated table can can run not only on the leader. -### Backward incompatible changes: +#### Backward incompatible changes: * Removed SET GLOBAL. -### Minor changes: +#### Minor changes: * Now after an alert is triggered, the log prints the full stack trace. * Relaxed the verification of the number of damaged/extra data parts at startup (there were too many false positives). -### Bug fixes: +#### Bug fixes: * Fixed a bad connection "sticking" when inserting into a Distributed table. * GLOBAL IN now works for a query from a Merge table that looks at a Distributed table. From 57517b4edefbebe278c27cfbee57b80cfc351ad5 Mon Sep 17 00:00:00 2001 From: Alexander Burmak Date: Fri, 10 Jan 2020 11:48:24 +0300 Subject: [PATCH 84/85] Replaced deprecated INSTANTIATE_TEST_CASE_P with INSTANTIATE_TEST_SUITE_P --- .../tests/gtest_compressionCodec.cpp | 48 +++++++++---------- .../src/Core/tests/gtest_DecimalFunctions.cpp | 2 +- .../tests/gtest_data_type_get_common_type.cpp | 4 +- .../gtest_DateTime64_parsing_and_writing.cpp | 6 +-- dbms/src/IO/tests/gtest_bit_io.cpp | 4 +- 5 files changed, 32 insertions(+), 32 deletions(-) diff --git a/dbms/src/Compression/tests/gtest_compressionCodec.cpp b/dbms/src/Compression/tests/gtest_compressionCodec.cpp index f15cb881798..8693ce86c2a 100644 --- a/dbms/src/Compression/tests/gtest_compressionCodec.cpp +++ b/dbms/src/Compression/tests/gtest_compressionCodec.cpp @@ -820,7 +820,7 @@ const auto DefaultCodecsToTest = ::testing::Values( // test cases /////////////////////////////////////////////////////////////////////////////////////////////////// -INSTANTIATE_TEST_CASE_P(Simple, +INSTANTIATE_TEST_SUITE_P(Simple, CodecTest, ::testing::Combine( DefaultCodecsToTest, @@ -830,7 +830,7 @@ INSTANTIATE_TEST_CASE_P(Simple, ) ); -INSTANTIATE_TEST_CASE_P(SmallSequences, +INSTANTIATE_TEST_SUITE_P(SmallSequences, CodecTest, ::testing::Combine( DefaultCodecsToTest, @@ -847,7 +847,7 @@ INSTANTIATE_TEST_CASE_P(SmallSequences, ) ); -INSTANTIATE_TEST_CASE_P(Mixed, +INSTANTIATE_TEST_SUITE_P(Mixed, CodecTest, ::testing::Combine( DefaultCodecsToTest, @@ -864,7 +864,7 @@ INSTANTIATE_TEST_CASE_P(Mixed, ) ); -INSTANTIATE_TEST_CASE_P(SameValueInt, +INSTANTIATE_TEST_SUITE_P(SameValueInt, CodecTest, ::testing::Combine( DefaultCodecsToTest, @@ -881,7 +881,7 @@ INSTANTIATE_TEST_CASE_P(SameValueInt, ) ); -INSTANTIATE_TEST_CASE_P(SameNegativeValueInt, +INSTANTIATE_TEST_SUITE_P(SameNegativeValueInt, CodecTest, ::testing::Combine( DefaultCodecsToTest, @@ -898,7 +898,7 @@ INSTANTIATE_TEST_CASE_P(SameNegativeValueInt, ) ); -INSTANTIATE_TEST_CASE_P(SameValueFloat, +INSTANTIATE_TEST_SUITE_P(SameValueFloat, CodecTest, ::testing::Combine( ::testing::Values( @@ -912,7 +912,7 @@ INSTANTIATE_TEST_CASE_P(SameValueFloat, ) ); -INSTANTIATE_TEST_CASE_P(SameNegativeValueFloat, +INSTANTIATE_TEST_SUITE_P(SameNegativeValueFloat, CodecTest, ::testing::Combine( ::testing::Values( @@ -926,7 +926,7 @@ INSTANTIATE_TEST_CASE_P(SameNegativeValueFloat, ) ); -INSTANTIATE_TEST_CASE_P(SequentialInt, +INSTANTIATE_TEST_SUITE_P(SequentialInt, CodecTest, ::testing::Combine( DefaultCodecsToTest, @@ -945,7 +945,7 @@ INSTANTIATE_TEST_CASE_P(SequentialInt, // -1, -2, -3, ... etc for signed // 0xFF, 0xFE, 0xFD, ... for unsigned -INSTANTIATE_TEST_CASE_P(SequentialReverseInt, +INSTANTIATE_TEST_SUITE_P(SequentialReverseInt, CodecTest, ::testing::Combine( DefaultCodecsToTest, @@ -962,7 +962,7 @@ INSTANTIATE_TEST_CASE_P(SequentialReverseInt, ) ); -INSTANTIATE_TEST_CASE_P(SequentialFloat, +INSTANTIATE_TEST_SUITE_P(SequentialFloat, CodecTest, ::testing::Combine( ::testing::Values( @@ -976,7 +976,7 @@ INSTANTIATE_TEST_CASE_P(SequentialFloat, ) ); -INSTANTIATE_TEST_CASE_P(SequentialReverseFloat, +INSTANTIATE_TEST_SUITE_P(SequentialReverseFloat, CodecTest, ::testing::Combine( ::testing::Values( @@ -990,7 +990,7 @@ INSTANTIATE_TEST_CASE_P(SequentialReverseFloat, ) ); -INSTANTIATE_TEST_CASE_P(MonotonicInt, +INSTANTIATE_TEST_SUITE_P(MonotonicInt, CodecTest, ::testing::Combine( DefaultCodecsToTest, @@ -1007,7 +1007,7 @@ INSTANTIATE_TEST_CASE_P(MonotonicInt, ) ); -INSTANTIATE_TEST_CASE_P(MonotonicReverseInt, +INSTANTIATE_TEST_SUITE_P(MonotonicReverseInt, CodecTest, ::testing::Combine( DefaultCodecsToTest, @@ -1024,7 +1024,7 @@ INSTANTIATE_TEST_CASE_P(MonotonicReverseInt, ) ); -INSTANTIATE_TEST_CASE_P(MonotonicFloat, +INSTANTIATE_TEST_SUITE_P(MonotonicFloat, CodecTest, ::testing::Combine( ::testing::Values( @@ -1037,7 +1037,7 @@ INSTANTIATE_TEST_CASE_P(MonotonicFloat, ) ); -INSTANTIATE_TEST_CASE_P(MonotonicReverseFloat, +INSTANTIATE_TEST_SUITE_P(MonotonicReverseFloat, CodecTest, ::testing::Combine( ::testing::Values( @@ -1050,7 +1050,7 @@ INSTANTIATE_TEST_CASE_P(MonotonicReverseFloat, ) ); -INSTANTIATE_TEST_CASE_P(RandomInt, +INSTANTIATE_TEST_SUITE_P(RandomInt, CodecTest, ::testing::Combine( DefaultCodecsToTest, @@ -1063,7 +1063,7 @@ INSTANTIATE_TEST_CASE_P(RandomInt, ) ); -INSTANTIATE_TEST_CASE_P(RandomishInt, +INSTANTIATE_TEST_SUITE_P(RandomishInt, CodecTest, ::testing::Combine( DefaultCodecsToTest, @@ -1078,7 +1078,7 @@ INSTANTIATE_TEST_CASE_P(RandomishInt, ) ); -INSTANTIATE_TEST_CASE_P(RandomishFloat, +INSTANTIATE_TEST_SUITE_P(RandomishFloat, CodecTest, ::testing::Combine( DefaultCodecsToTest, @@ -1090,7 +1090,7 @@ INSTANTIATE_TEST_CASE_P(RandomishFloat, ); // Double delta overflow case, deltas are out of bounds for target type -INSTANTIATE_TEST_CASE_P(OverflowInt, +INSTANTIATE_TEST_SUITE_P(OverflowInt, CodecTest, ::testing::Combine( ::testing::Values( @@ -1106,7 +1106,7 @@ INSTANTIATE_TEST_CASE_P(OverflowInt, ) ); -INSTANTIATE_TEST_CASE_P(OverflowFloat, +INSTANTIATE_TEST_SUITE_P(OverflowFloat, CodecTest, ::testing::Combine( ::testing::Values( @@ -1156,7 +1156,7 @@ auto DDCompatibilityTestSequence() #define BIN_STR(x) std::string{x, sizeof(x) - 1} -INSTANTIATE_TEST_CASE_P(DoubleDelta, +INSTANTIATE_TEST_SUITE_P(DoubleDelta, CodecTest_Compatibility, ::testing::Combine( ::testing::Values(Codec("DoubleDelta")), @@ -1230,7 +1230,7 @@ auto GCompatibilityTestSequence() return generateSeq(G(PrimesWithMultiplierGenerator(intExp10(sizeof(ValueType)))), 0, 42); } -INSTANTIATE_TEST_CASE_P(Gorilla, +INSTANTIATE_TEST_SUITE_P(Gorilla, CodecTest_Compatibility, ::testing::Combine( ::testing::Values(Codec("Gorilla")), @@ -1292,7 +1292,7 @@ INSTANTIATE_TEST_CASE_P(Gorilla, // return sequence; //}; -//INSTANTIATE_TEST_CASE_P(DoubleDelta, +//INSTANTIATE_TEST_SUITE_P(DoubleDelta, // CodecTest_Performance, // ::testing::Combine( // ::testing::Values(Codec("DoubleDelta")), @@ -1309,7 +1309,7 @@ INSTANTIATE_TEST_CASE_P(Gorilla, // ), //); -//INSTANTIATE_TEST_CASE_P(Gorilla, +//INSTANTIATE_TEST_SUITE_P(Gorilla, // CodecTest_Performance, // ::testing::Combine( // ::testing::Values(Codec("Gorilla")), diff --git a/dbms/src/Core/tests/gtest_DecimalFunctions.cpp b/dbms/src/Core/tests/gtest_DecimalFunctions.cpp index 84bbb987819..fc304446057 100644 --- a/dbms/src/Core/tests/gtest_DecimalFunctions.cpp +++ b/dbms/src/Core/tests/gtest_DecimalFunctions.cpp @@ -120,7 +120,7 @@ TEST_P(DecimalUtilsSplitAndCombineTest, getFractionalPart_Decimal128) } // Intentionally small values that fit into 32-bit in order to cover Decimal32, Decimal64 and Decimal128 with single set of data. -INSTANTIATE_TEST_CASE_P(Basic, +INSTANTIATE_TEST_SUITE_P(Basic, DecimalUtilsSplitAndCombineTest, ::testing::ValuesIn(std::initializer_list{ { diff --git a/dbms/src/DataTypes/tests/gtest_data_type_get_common_type.cpp b/dbms/src/DataTypes/tests/gtest_data_type_get_common_type.cpp index f71dca14e43..602320f5fca 100644 --- a/dbms/src/DataTypes/tests/gtest_data_type_get_common_type.cpp +++ b/dbms/src/DataTypes/tests/gtest_data_type_get_common_type.cpp @@ -104,7 +104,7 @@ TEST_P(MostSubtypeTest, getLeastSupertype) } } -INSTANTIATE_TEST_CASE_P(data_type, +INSTANTIATE_TEST_SUITE_P(data_type, LeastSuperTypeTest, ::testing::ValuesIn( std::initializer_list{ @@ -162,7 +162,7 @@ INSTANTIATE_TEST_CASE_P(data_type, ) ); -INSTANTIATE_TEST_CASE_P(data_type, +INSTANTIATE_TEST_SUITE_P(data_type, MostSubtypeTest, ::testing::ValuesIn( std::initializer_list{ diff --git a/dbms/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp b/dbms/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp index c1a6068a17f..04fdb6f4a34 100644 --- a/dbms/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp +++ b/dbms/src/IO/tests/gtest_DateTime64_parsing_and_writing.cpp @@ -79,7 +79,7 @@ TEST_P(DateTime64StringParseBestEffortTest, parse) // YYYY-MM-DD HH:MM:SS.NNNNNNNNN -INSTANTIATE_TEST_CASE_P(Basic, +INSTANTIATE_TEST_SUITE_P(Basic, DateTime64StringParseTest, ::testing::ValuesIn(std::initializer_list{ { @@ -133,7 +133,7 @@ INSTANTIATE_TEST_CASE_P(Basic, }) ); -INSTANTIATE_TEST_CASE_P(BestEffort, +INSTANTIATE_TEST_SUITE_P(BestEffort, DateTime64StringParseBestEffortTest, ::testing::ValuesIn(std::initializer_list{ { @@ -148,7 +148,7 @@ INSTANTIATE_TEST_CASE_P(BestEffort, // TODO: add negative test cases for invalid strings, verifying that error is reported properly -INSTANTIATE_TEST_CASE_P(Basic, +INSTANTIATE_TEST_SUITE_P(Basic, DateTime64StringWriteTest, ::testing::ValuesIn(std::initializer_list{ { diff --git a/dbms/src/IO/tests/gtest_bit_io.cpp b/dbms/src/IO/tests/gtest_bit_io.cpp index b6b4cf998bd..57fe45ca1a6 100644 --- a/dbms/src/IO/tests/gtest_bit_io.cpp +++ b/dbms/src/IO/tests/gtest_bit_io.cpp @@ -177,7 +177,7 @@ TEST_P(BitIO, WriteAndRead) } } -INSTANTIATE_TEST_CASE_P(Simple, +INSTANTIATE_TEST_SUITE_P(Simple, BitIO, ::testing::ValuesIn(std::initializer_list{ { @@ -241,7 +241,7 @@ TestCaseParameter primes_case(UInt8 repeat_times, UInt64 pattern) return TestCaseParameter(test_data); } -INSTANTIATE_TEST_CASE_P(Primes, +INSTANTIATE_TEST_SUITE_P(Primes, BitIO, ::testing::Values( primes_case(11, 0xFFFFFFFFFFFFFFFFULL), From fd5599939874cf512f2463110ff1b1ca87955ea6 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Fri, 10 Jan 2020 12:24:05 +0300 Subject: [PATCH 85/85] Initialize set for index when table filter is presented (#8357) --- dbms/src/IO/S3Common.cpp | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 33 ++++++++++--------- .../tests/integration/test_row_policy/test.py | 3 +- 3 files changed, 21 insertions(+), 17 deletions(-) diff --git a/dbms/src/IO/S3Common.cpp b/dbms/src/IO/S3Common.cpp index a9015ca5982..b981c34c2d2 100644 --- a/dbms/src/IO/S3Common.cpp +++ b/dbms/src/IO/S3Common.cpp @@ -29,7 +29,7 @@ const std::pair & convertLogLevel(Aws::Utils::Logg return mapping.at(log_level); } -class AWSLogger : public Aws::Utils::Logging::LogSystemInterface +class AWSLogger final : public Aws::Utils::Logging::LogSystemInterface { public: ~AWSLogger() final = default; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 1d54900ae93..743e79f7373 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -503,28 +503,31 @@ Block InterpreterSelectQuery::getSampleBlockImpl() /// Do all AST changes here, because actions from analysis_result will be used later in readImpl. - /// PREWHERE optimization. - /// Turn off, if the table filter (row-level security) is applied. - if (storage && !context->getRowPolicy()->getCondition(storage->getDatabaseName(), storage->getTableName(), RowPolicy::SELECT_FILTER)) + if (storage) { query_analyzer->makeSetsForIndex(query.where()); query_analyzer->makeSetsForIndex(query.prewhere()); - auto optimize_prewhere = [&](auto & merge_tree) + /// PREWHERE optimization. + /// Turn off, if the table filter (row-level security) is applied. + if (!context->getRowPolicy()->getCondition(storage->getDatabaseName(), storage->getTableName(), RowPolicy::SELECT_FILTER)) { - SelectQueryInfo current_info; - current_info.query = query_ptr; - current_info.syntax_analyzer_result = syntax_analyzer_result; - current_info.sets = query_analyzer->getPreparedSets(); + auto optimize_prewhere = [&](auto & merge_tree) + { + SelectQueryInfo current_info; + current_info.query = query_ptr; + current_info.syntax_analyzer_result = syntax_analyzer_result; + current_info.sets = query_analyzer->getPreparedSets(); - /// Try transferring some condition from WHERE to PREWHERE if enabled and viable - if (settings.optimize_move_to_prewhere && query.where() && !query.prewhere() && !query.final()) - MergeTreeWhereOptimizer{current_info, *context, merge_tree, - syntax_analyzer_result->requiredSourceColumns(), log}; - }; + /// Try transferring some condition from WHERE to PREWHERE if enabled and viable + if (settings.optimize_move_to_prewhere && query.where() && !query.prewhere() && !query.final()) + MergeTreeWhereOptimizer{current_info, *context, merge_tree, + syntax_analyzer_result->requiredSourceColumns(), log}; + }; - if (const auto * merge_tree_data = dynamic_cast(storage.get())) - optimize_prewhere(*merge_tree_data); + if (const auto * merge_tree_data = dynamic_cast(storage.get())) + optimize_prewhere(*merge_tree_data); + } } if (storage && !options.only_analyze) diff --git a/dbms/tests/integration/test_row_policy/test.py b/dbms/tests/integration/test_row_policy/test.py index 421a4b0510c..f5136faff54 100644 --- a/dbms/tests/integration/test_row_policy/test.py +++ b/dbms/tests/integration/test_row_policy/test.py @@ -34,7 +34,7 @@ def started_cluster(): CREATE TABLE mydb.filtered_table3 (a UInt8, b UInt8, c UInt16 ALIAS a + b) ENGINE MergeTree ORDER BY a; INSERT INTO mydb.filtered_table3 values (0, 0), (0, 1), (1, 0), (1, 1); ''') - + yield cluster finally: @@ -58,6 +58,7 @@ def test_smoke(): assert instance.query("SELECT a FROM mydb.filtered_table1") == "1\n1\n" assert instance.query("SELECT b FROM mydb.filtered_table1") == "0\n1\n" assert instance.query("SELECT a FROM mydb.filtered_table1 WHERE a = 1") == "1\n1\n" + assert instance.query("SELECT a FROM mydb.filtered_table1 WHERE a IN (1)") == "1\n1\n" assert instance.query("SELECT a = 1 FROM mydb.filtered_table1") == "1\n1\n" assert instance.query("SELECT a FROM mydb.filtered_table3") == "0\n1\n"