From 6b1030c9b879154e65b3fc456466ea1e36ce6aeb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 13 Aug 2021 19:32:29 +0300 Subject: [PATCH 001/142] Rewrite PushingToViewsBIS part 1. --- src/Common/ThreadStatus.h | 6 +- .../PushingToViewsBlockOutputStream.cpp | 50 ++++++------- .../PushingToViewsBlockOutputStream.h | 3 +- src/Processors/Drain.cpp | 75 +++++++++++++++++++ src/Processors/Drain.h | 23 ++++++ 5 files changed, 129 insertions(+), 28 deletions(-) create mode 100644 src/Processors/Drain.cpp create mode 100644 src/Processors/Drain.h diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index dbfb33a320c..e4edad45bb5 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -145,7 +145,6 @@ protected: Poco::Logger * log = nullptr; friend class CurrentThread; - friend class PushingToViewsBlockOutputStream; /// Use ptr not to add extra dependencies in the header std::unique_ptr last_rusage; @@ -188,6 +187,11 @@ public: return query_context.lock(); } + void disableProfiling() + { + query_profiled_enabled = false; + } + /// Starts new query and create new thread group for it, current thread becomes master thread of the query void initializeQuery(); diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index aec1209a454..5e8aca96675 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -30,17 +30,13 @@ namespace DB { -PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( - const StoragePtr & storage_, - const StorageMetadataPtr & metadata_snapshot_, - ContextPtr context_, - const ASTPtr & query_ptr_, - bool no_destination) - : WithContext(context_) - , storage(storage_) - , metadata_snapshot(metadata_snapshot_) - , log(&Poco::Logger::get("PushingToViewsBlockOutputStream")) - , query_ptr(query_ptr_) +Drain buildPushingToViewsDrainImpl( + const StoragePtr & storage, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + const ASTPtr & query_ptr, + bool no_destination, + std::vector & locks) { checkStackSize(); @@ -48,19 +44,20 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( * Although now any insertion into the table is done via PushingToViewsBlockOutputStream, * but it's clear that here is not the best place for this functionality. */ - addTableLock( - storage->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout)); + locks.emplace_back(storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout)); /// If the "root" table deduplicates blocks, there are no need to make deduplication for children /// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks bool disable_deduplication_for_children = false; - if (!getContext()->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) + if (!context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) disable_deduplication_for_children = !no_destination && storage->supportsDeduplication(); auto table_id = storage->getStorageID(); Dependencies dependencies = DatabaseCatalog::instance().getDependencies(table_id); /// We need special context for materialized views insertions + ContextMutablePtr select_context; + ContextMutablePtr insert_context; if (!dependencies.empty()) { select_context = Context::createCopy(context); @@ -79,21 +76,22 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( insert_context->setSetting("min_insert_block_size_bytes", insert_settings.min_insert_block_size_bytes_for_materialized_views.value); } + std::vector views; + for (const auto & database_table : dependencies) { - auto dependent_table = DatabaseCatalog::instance().getTable(database_table, getContext()); + auto dependent_table = DatabaseCatalog::instance().getTable(database_table, context); auto dependent_metadata_snapshot = dependent_table->getInMemoryMetadataPtr(); ASTPtr query; - BlockOutputStreamPtr out; + Drain out; QueryViewsLogElement::ViewType type = QueryViewsLogElement::ViewType::DEFAULT; String target_name = database_table.getFullTableName(); if (auto * materialized_view = dynamic_cast(dependent_table.get())) { type = QueryViewsLogElement::ViewType::MATERIALIZED; - addTableLock( - materialized_view->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout)); + locks.emplace_back(materialized_view->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout)); StoragePtr inner_table = materialized_view->getTargetTable(); auto inner_table_id = inner_table->getStorageID(); @@ -129,12 +127,12 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( { type = QueryViewsLogElement::ViewType::LIVE; query = live_view->getInnerQuery(); // Used only to log in system.query_views_log - out = std::make_shared( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true); + out = buildPushingToViewsDrainImpl( + dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true, locks); } else - out = std::make_shared( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr()); + out = buildPushingToViewsDrainImpl( + dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), false, locks); /// If the materialized view is executed outside of a query, for example as a result of SYSTEM FLUSH LOGS or /// SYSTEM FLUSH DISTRIBUTED ..., we can't attach to any thread group and we won't log, so there is no point on collecting metrics @@ -142,7 +140,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( ThreadGroupStatusPtr running_group = current_thread && current_thread->getThreadGroup() ? current_thread->getThreadGroup() - : MainThreadStatus::getInstance().thread_group; + : MainThreadStatus::getInstance().getThreadGroup(); if (running_group) { /// We are creating a ThreadStatus per view to store its metrics individually @@ -156,8 +154,8 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( /// Disable query profiler for this ThreadStatus since the running (main query) thread should already have one /// If we didn't disable it, then we could end up with N + 1 (N = number of dependencies) profilers which means /// N times more interruptions - thread_status->query_profiled_enabled = false; - thread_status->setupState(running_group); + thread_status->disableProfiling(); + thread_status->attachQuery(running_group); } QueryViewsLogElement::ViewRuntimeStats runtime_stats{ @@ -173,7 +171,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( /// Add the view to the query access info so it can appear in system.query_log if (!no_destination) { - getContext()->getQueryContext()->addQueryAccessInfo( + context->getQueryContext()->addQueryAccessInfo( backQuoteIfNeed(database_table.getDatabaseName()), target_name, {}, "", database_table.getFullTableName()); } } diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index ba125e28829..776398cc107 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace Poco { @@ -20,7 +21,7 @@ struct ViewRuntimeData { const ASTPtr query; StorageID table_id; - BlockOutputStreamPtr out; + Drain out; std::exception_ptr exception; QueryViewsLogElement::ViewRuntimeStats runtime_stats; diff --git a/src/Processors/Drain.cpp b/src/Processors/Drain.cpp new file mode 100644 index 00000000000..a6a010f0a42 --- /dev/null +++ b/src/Processors/Drain.cpp @@ -0,0 +1,75 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +static void checkSink(const IProcessor & sink) +{ + if (!sink.getOutputs().empty()) + throw Exception("Sink for drain shouldn't have any output, but " + sink.getName() + " has " + + toString(sink.getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); + + if (sink.getInputs().empty()) + throw Exception("Sink for drain should have single input, but it doesn't have any", + ErrorCodes::LOGICAL_ERROR); + + if (sink.getInputs().size() > 1) + throw Exception("Sink for drain should have single input, but " + sink.getName() + " has " + + toString(sink.getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR); + + if (sink.getInputs().front().isConnected()) + throw Exception("Sink for drain has connected input.", ErrorCodes::LOGICAL_ERROR); +} + +static void checkTransform(const IProcessor & transform) +{ + if (transform.getInputs().size() != 1) + throw Exception("Transform for drain should have single input, " + "but " + transform.getName() + " has " + + toString(transform.getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR); + + if (transform.getOutputs().size() != 1) + throw Exception("Transform for drain should have single output, " + "but " + transform.getName() + " has " + + toString(transform.getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); + + if (transform.getInputs().front().isConnected()) + throw Exception("Transform for drain has connected input.", ErrorCodes::LOGICAL_ERROR); + + if (transform.getOutputs().front().isConnected()) + throw Exception("Transform for drain has connected input.", ErrorCodes::LOGICAL_ERROR); +} + +void checkInitialized(const Processors & processors) +{ + if (processors.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Drain is not initialized"); +} + +Drain::Drain(ProcessorPtr processor) +{ + checkSink(*processor); + processors.emplace_back(std::move(processor)); +} + +void Drain::addTransform(ProcessorPtr processor) +{ + checkInitialized(processors); + checkTransform(*processor); + connect(processor->getOutputs().front(), processors.back()->getInputs().front()); + processors.emplace_back(std::move(processor)); +} + +InputPort & Drain::getPort() const +{ + checkInitialized(processors); + return processors.back()->getInputs().front(); +} + +} diff --git a/src/Processors/Drain.h b/src/Processors/Drain.h new file mode 100644 index 00000000000..866d8a0fc52 --- /dev/null +++ b/src/Processors/Drain.h @@ -0,0 +1,23 @@ +#pragma once + +#include + +namespace DB +{ + +class Drain +{ +public: + Drain() = default; + explicit Drain(ProcessorPtr processor); + + void addTransform(ProcessorPtr processor); + + InputPort & getPort() const; + const Block & getHeader() const { return getPort().getHeader(); } + +private: + Processors processors; +}; + +} From d7e78f3ea9d5f482766c74a2a495b3f85e974667 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 23 Aug 2021 13:46:52 +0300 Subject: [PATCH 002/142] Rewrite PushingToViewsBlockOutputStream part 1. --- src/CMakeLists.txt | 1 + .../PushingToViewsBlockOutputStream.cpp | 220 +++++++++++++++--- .../PushingToViewsBlockOutputStream.h | 26 ++- src/Processors/Sinks/SinkToStorage.cpp | 112 +++++++++ src/Processors/Sinks/SinkToStorage.h | 57 ++++- 5 files changed, 378 insertions(+), 38 deletions(-) create mode 100644 src/Processors/Sinks/SinkToStorage.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 796c9eb4d2c..3d2ff00c313 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -210,6 +210,7 @@ add_object_library(clickhouse_processors_formats Processors/Formats) add_object_library(clickhouse_processors_formats_impl Processors/Formats/Impl) add_object_library(clickhouse_processors_transforms Processors/Transforms) add_object_library(clickhouse_processors_sources Processors/Sources) +add_object_library(clickhouse_processors_sinks Processors/Sinks) add_object_library(clickhouse_processors_merges Processors/Merges) add_object_library(clickhouse_processors_merges_algorithms Processors/Merges/Algorithms) add_object_library(clickhouse_processors_queryplan Processors/QueryPlan) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 5e8aca96675..7ea3b8b767b 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -10,6 +10,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -30,6 +33,115 @@ namespace DB { +class ExceptionCollectingTransform : public IProcessor +{ +public: + ExceptionCollectingTransform(const Block & header, size_t num_inputs) + : IProcessor(InputPorts(num_inputs, header), {header}) + , output(outputs.front()) + { + has_exception.assign(num_inputs, false); + } + + Status prepare() override + { + if (output.isFinished()) + { + for (auto & input : inputs) + input.close(); + + return Status::Finished; + } + + if (!output.canPush()) + return Status::PortFull; + + size_t num_finished = 0; + size_t pos = 0; + for (auto & input : inputs) + { + auto i = pos; + ++pos; + + if (input.isFinished()) + { + ++num_finished; + continue; + } + + input.setNeeded(); + if (input.hasData()) + { + auto data = input.pullData(); + if (data.exception) + { + if (i == 0 || !has_exception[i]) + { + has_exception[i] = true; + output.pushData(std::move(data)); + return Status::PortFull; + } + } + + if (input.isFinished()) + ++num_finished; + } + } + + if (num_finished == inputs.size()) + { + output.finish(); + return Status::Finished; + } + + return Status::NeedData; + } + +private: + OutputPort & output; + std::vector has_exception; +}; + +class ExceptionHandlingSink : public IProcessor +{ +public: + explicit ExceptionHandlingSink(Block header) + : IProcessor({std::move(header)}, {}) + , input(inputs.front()) + { + } + + Status prepare() override + { + while (!input.isFinished()) + { + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + + auto data = input.pullData(); + if (data.exception) + exceptions.emplace_back(std::move(data.exception)); + } + + if (!exceptions.empty()) + return Status::Ready; + + return Status::Finished; + } + + void work() override + { + auto exception = std::move(exceptions.at(0)); + exceptions.clear(); + std::rethrow_exception(std::move(exception)); + } + +private: + InputPort & input; + std::vector exceptions; +}; + Drain buildPushingToViewsDrainImpl( const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot, @@ -179,12 +291,11 @@ Drain buildPushingToViewsDrainImpl( /// Do not push to destination table if the flag is set if (!no_destination) { - auto sink = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), getContext()); + auto sink = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), context); - metadata_snapshot->check(sink->getPort().getHeader().getColumnsWithTypeAndName()); + metadata_snapshot->check(sink->getHeader().getColumnsWithTypeAndName()); - replicated_output = dynamic_cast(sink.get()); - output = std::make_shared(std::move(sink)); + auto replicated_output = dynamic_cast(sink.get()); } } @@ -403,9 +514,21 @@ void PushingToViewsBlockOutputStream::flush() view.out->flush(); } -void PushingToViewsBlockOutputStream::process(const Block & block, ViewRuntimeData & view) +static void process(Block & block, ViewRuntimeData & view) { - BlockInputStreamPtr in; + const auto & storage = view.storage; + const auto & metadata_snapshot = view.metadata_snapshot; + const auto & context = view.context; + + /// We create a table with the same name as original table and the same alias columns, + /// but it will contain single block (that is INSERT-ed into main table). + /// InterpreterSelectQuery will do processing of alias columns. + auto local_context = Context::createCopy(context); + local_context->addViewSource(StorageValues::create( + storage->getStorageID(), + metadata_snapshot->getColumns(), + block, + storage->getVirtuals())); /// We need keep InterpreterSelectQuery, until the processing will be finished, since: /// @@ -418,45 +541,76 @@ void PushingToViewsBlockOutputStream::process(const Block & block, ViewRuntimeDa /// (the problem raises only when function uses context from the /// execute*() method, like FunctionDictGet do) /// - These objects live inside query pipeline (DataStreams) and the reference become dangling. - std::optional select; + InterpreterSelectQuery select(view.query, local_context, SelectQueryOptions()); - if (view.runtime_stats.type == QueryViewsLogElement::ViewType::MATERIALIZED) - { - /// We create a table with the same name as original table and the same alias columns, - /// but it will contain single block (that is INSERT-ed into main table). - /// InterpreterSelectQuery will do processing of alias columns. + auto io = select.execute(); + io.pipeline.resize(1); - auto local_context = Context::createCopy(select_context); - local_context->addViewSource( - StorageValues::create(storage->getStorageID(), metadata_snapshot->getColumns(), block, storage->getVirtuals())); - select.emplace(view.query, local_context, SelectQueryOptions()); - in = std::make_shared(select->execute().getInputStream()); + /// Squashing is needed here because the materialized 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). + io.pipeline.addTransform(std::make_shared( + io.pipeline.getHeader(), + context->getSettingsRef().min_insert_block_size_rows, + context->getSettingsRef().min_insert_block_size_bytes)); - /// Squashing is needed here because the materialized 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). - in = std::make_shared( - in, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes); - in = std::make_shared(in, view.out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Name); - } - else - in = std::make_shared(block); + auto converting = ActionsDAG::makeConvertingActions( + io.pipeline.getHeader().getColumnsWithTypeAndName(), + view.sample_block.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); - in->setProgressCallback([this](const Progress & progress) + io.pipeline.addTransform(std::make_shared( + io.pipeline.getHeader(), + std::make_shared(std::move(converting)))); + + io.pipeline.setProgressCallback([context](const Progress & progress) { CurrentThread::updateProgressIn(progress); - this->onProgress(progress); + if (auto callback = context->getProgressCallback()) + callback(progress); }); - in->readPrefix(); + PullingPipelineExecutor executor(io.pipeline); + if (!executor.pull(block)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No nothing is returned from view inner query {}", view.query); - while (Block result_block = in->read()) + if (executor.pull(block)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Single chunk is expected from view inner query {}", view.query); +} + +void ExecutingInnerQueryFromViewTransform::transform(Chunk & chunk) +{ + Stopwatch watch; + + auto * original_thread = current_thread; + SCOPE_EXIT({ current_thread = original_thread; }); + + if (view.runtime_stats.thread_status) { - Nested::validateArraySizes(result_block); - view.out->write(result_block); + /// Change thread context to store individual metrics per view. Once the work in done, go back to the original thread + view.runtime_stats.thread_status->resetPerformanceCountersLastUsage(); + current_thread = view.runtime_stats.thread_status.get(); } - in->readSuffix(); + try + { + auto block = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); + process(block, view); + chunk.setColumns(block.getColumns(), block.rows()); + } + catch (Exception & ex) + { + ex.addMessage("while pushing to view " + view.table_id.getNameForLogs()); + view.setException(std::current_exception()); + } + catch (...) + { + view.setException(std::current_exception()); + } + + if (view.runtime_stats.thread_status) + view.runtime_stats.thread_status->updatePerformanceCounters(); + view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); } void PushingToViewsBlockOutputStream::checkExceptionsInViews() diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index 776398cc107..9466c5ef326 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace Poco { @@ -20,8 +21,14 @@ class ReplicatedMergeTreeSink; struct ViewRuntimeData { const ASTPtr query; + Block sample_block; + StorageID table_id; - Drain out; + StoragePtr storage; + StorageMetadataPtr metadata_snapshot; + + ContextPtr context; + std::exception_ptr exception; QueryViewsLogElement::ViewRuntimeStats runtime_stats; @@ -71,5 +78,22 @@ private: void logQueryViews(); }; +class ExecutingInnerQueryFromViewTransform final : public ExceptionKeepingTransform +{ +public: + ExecutingInnerQueryFromViewTransform(const Block & header, ViewRuntimeData view_data) + : ExceptionKeepingTransform(header, view_data.sample_block) + , view(std::move(view_data)) + { + } + + String getName() const override { return "ExecutingInnerQueryFromView"; } + +protected: + void transform(Chunk & chunk) override; + +private: + ViewRuntimeData view; +}; } diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp new file mode 100644 index 00000000000..f538bb3c8f0 --- /dev/null +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -0,0 +1,112 @@ +#include + +namespace DB +{ + + +ExceptionKeepingTransform::ExceptionKeepingTransform(const Block & in_header, const Block & out_header) + : IProcessor({in_header}, {out_header}) + , input(inputs.front()), output(outputs.front()) +{ +} + +IProcessor::Status ExceptionKeepingTransform::prepare() +{ + if (!was_on_start_called) + return Status::Ready; + + /// Check can output. + + if (output.isFinished()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Output port is finished for {}", getName()); + + if (!output.canPush()) + { + input.setNotNeeded(); + return Status::PortFull; + } + + /// Output if has data. + if (ready_output) + { + output.pushData(std::move(data)); + ready_output = false; + return Status::PortFull; + } + + if (!ready_input) + { + if (input.isFinished()) + { + if (!was_on_finish_called) + return Status::Ready; + + output.finish(); + return Status::Finished; + } + + input.setNeeded(); + + if (!input.hasData()) + return Status::NeedData; + + data = input.pullData(true); + + if (data.exception) + { + output.pushData(std::move(data)); + return Status::PortFull; + } + + ready_input = true; + } + + return Status::Ready; +} + +void ExceptionKeepingTransform::work() +{ + if (!was_on_start_called) + { + was_on_start_called = true; + onStart(); + } + + if (ready_input) + { + ready_input = false; + ready_output = true; + + try + { + transform(data.chunk); + } + catch (...) + { + data.chunk.clear(); + data.exception = std::current_exception(); + } + } + else if (!was_on_finish_called) + { + was_on_finish_called = true; + try + { + onFinish(); + } + catch (...) + { + ready_input = true; + data.exception = std::current_exception(); + } + } +} + +SinkToStorage::SinkToStorage(const Block & header) : ExceptionKeepingTransform(header, header) {} + +void SinkToStorage::transform(Chunk & chunk) +{ + consume(chunk.clone()); +} + +} diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index c57adef568f..3629b8714da 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -1,21 +1,70 @@ #pragma once -#include +#include #include namespace DB { +/// Has one input and one output. +/// Works similarly to ISimpleTransform, but with much care about exceptions. +/// +/// If input contain exception, this exception is pushed directly to output port. +/// If input contain data chunk, transform() is called for it. +/// When transform throws exception itself, data chunk is replaced by caught exception. +/// Transformed chunk or newly caught exception is pushed to output. +/// +/// There may be any number of exceptions read from input, transform keeps the order. +/// It is expected that output port won't be closed from the other side before all data is processed. +/// +/// Method onStart() is called before reading any data. +/// Exception from it is not pushed into pipeline, but thrown immediately. +/// +/// Method onFinish() is called after all data from input is processed. +/// In case of exception, it is additionally pushed into pipeline. +class ExceptionKeepingTransform : public IProcessor +{ +private: + InputPort & input; + OutputPort & output; + Port::Data data; + + bool ready_input = false; + bool ready_output = false; + bool was_on_start_called = false; + bool was_on_finish_called = false; + +protected: + virtual void transform(Chunk & chunk) = 0; + virtual void onStart() {} + virtual void onFinish() {} + +public: + ExceptionKeepingTransform(const Block & in_header, const Block & out_header); + + Status prepare() override; + void work() override; + + InputPort & getInputPort() { return input; } + OutputPort & getOutputPort() { return output; } +}; + + /// Sink which is returned from Storage::read. -/// The same as ISink, but also can hold table lock. -class SinkToStorage : public ISink +class SinkToStorage : public ExceptionKeepingTransform { public: - using ISink::ISink; + explicit SinkToStorage(const Block & header); + const Block & getHeader() const { return inputs.front().getHeader(); } void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } +protected: + virtual void consume(Chunk chunk) = 0; + private: std::vector table_locks; + + void transform(Chunk & chunk) override; }; using SinkToStoragePtr = std::shared_ptr; From 4d821efa153cf63d08439f1e1e6f1746fa2f0e4b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 26 Aug 2021 11:01:26 +0300 Subject: [PATCH 003/142] Rewrite PushingToViewsBlockOutputStream part 2. --- src/Common/ThreadStatus.h | 2 +- .../PushingToViewsBlockOutputStream.cpp | 177 +++++++++++++----- .../PushingToViewsBlockOutputStream.h | 1 + 3 files changed, 130 insertions(+), 50 deletions(-) diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index e4edad45bb5..1ce88e3a6c4 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -226,6 +226,7 @@ public: /// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped void detachQuery(bool exit_if_already_detached = false, bool thread_exits = false); + void logToQueryViewsLog(const ViewRuntimeData & vinfo); protected: void applyQuerySettings(); @@ -238,7 +239,6 @@ protected: void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database, std::chrono::time_point now); - void logToQueryViewsLog(const ViewRuntimeData & vinfo); void assertState(const std::initializer_list & permitted_states, const char * description = nullptr) const; diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 7ea3b8b767b..9f9ed68851b 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -33,25 +33,108 @@ namespace DB { -class ExceptionCollectingTransform : public IProcessor +struct ViewsData +{ + std::vector views; + + /// In case of exception happened while inserting into main table, it is pushed to pipeline. + /// Remember the first one, we should keep them after view processing. + std::atomic_bool has_exception = false; + std::exception_ptr first_exception; +}; + +using ViewsDataPtr = std::shared_ptr; + +class CopyingDataToViewsTransform final : public IProcessor { public: - ExceptionCollectingTransform(const Block & header, size_t num_inputs) - : IProcessor(InputPorts(num_inputs, header), {header}) - , output(outputs.front()) + CopyingDataToViewsTransform(const Block & header, ViewsDataPtr data) + : IProcessor({header}, OutputPorts(data->views.size(), header)) + , input(inputs.front()) + , views_data(std::move(data)) { - has_exception.assign(num_inputs, false); + if (views_data->views.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "CopyingDataToViewsTransform cannot have zero outputs"); } + String getName() const override { return "CopyingDataToViewsTransform"; } + + Status prepare() override + { + bool all_can_push = true; + for (auto & output : outputs) + { + if (output.isFinished()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot push data to view because output port is finished"); + + if (!output.canPush()) + all_can_push = false; + } + + if (!all_can_push) + return Status::PortFull; + + if (input.isFinished()) + { + for (auto & output : outputs) + output.finish(); + + return Status::Finished; + } + + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + + auto data = input.pullData(); + if (data.exception) + { + if (!views_data->has_exception) + { + views_data->first_exception = data.exception; + views_data->has_exception = true; + } + + for (auto & output : outputs) + output.pushException(data.exception); + } + else + { + for (auto & output : outputs) + output.push(data.chunk.clone()); + } + } + + InputPort & getInputPort() { return input; } + +private: + InputPort & input; + ViewsDataPtr views_data; +}; + +class FinalizingViewsTransform final : public IProcessor +{ + struct ExceptionStatus + { + std::exception_ptr exception; + bool is_first = false; + }; + +public: + FinalizingViewsTransform(const Block & header, ViewsDataPtr data) + : IProcessor(InputPorts(data->views.size(), header), {header}) + , output(outputs.front()) + , views_data(std::move(data)) + { + statuses.resize(views_data->views.size()); + } + + String getName() const override { return "FinalizingViewsTransform"; } + Status prepare() override { if (output.isFinished()) - { - for (auto & input : inputs) - input.close(); - - return Status::Finished; - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot finalize views because output port is finished"); if (!output.canPush()) return Status::PortFull; @@ -75,9 +158,13 @@ public: auto data = input.pullData(); if (data.exception) { - if (i == 0 || !has_exception[i]) + if (views_data->has_exception && views_data->first_exception == data.exception) + statuses[i].is_first = true; + else + statuses[i].exception = data.exception; + + if (i == 0 && statuses[0].is_first) { - has_exception[i] = true; output.pushData(std::move(data)); return Status::PortFull; } @@ -90,6 +177,9 @@ public: if (num_finished == inputs.size()) { + if (!statuses.empty()) + return Status::Ready; + output.finish(); return Status::Finished; } @@ -97,9 +187,29 @@ public: return Status::NeedData; } + void work() override + { + size_t num_views = statuses.size(); + for (size_t i = 0; i < num_views; ++i) + { + auto & view = views_data->views[i]; + auto & status = statuses[i]; + if (status.exception) + { + if (!any_exception) + any_exception = status.exception; + + view.setException(std::move(status.exception)); + } + } + statuses.clear(); + } + private: OutputPort & output; - std::vector has_exception; + ViewsDataPtr views_data; + std::vector statuses; + std::exception_ptr any_exception; }; class ExceptionHandlingSink : public IProcessor @@ -580,37 +690,12 @@ static void process(Block & block, ViewRuntimeData & view) void ExecutingInnerQueryFromViewTransform::transform(Chunk & chunk) { - Stopwatch watch; - - auto * original_thread = current_thread; - SCOPE_EXIT({ current_thread = original_thread; }); - - if (view.runtime_stats.thread_status) - { - /// Change thread context to store individual metrics per view. Once the work in done, go back to the original thread - view.runtime_stats.thread_status->resetPerformanceCountersLastUsage(); - current_thread = view.runtime_stats.thread_status.get(); - } - - try + runViewStage(view, "while pushing to view", [&] { auto block = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); process(block, view); chunk.setColumns(block.getColumns(), block.rows()); - } - catch (Exception & ex) - { - ex.addMessage("while pushing to view " + view.table_id.getNameForLogs()); - view.setException(std::current_exception()); - } - catch (...) - { - view.setException(std::current_exception()); - } - - if (view.runtime_stats.thread_status) - view.runtime_stats.thread_status->updatePerformanceCounters(); - view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); + }); } void PushingToViewsBlockOutputStream::checkExceptionsInViews() @@ -625,9 +710,9 @@ void PushingToViewsBlockOutputStream::checkExceptionsInViews() } } -void PushingToViewsBlockOutputStream::logQueryViews() +static void logQueryViews(std::vector & views, ContextPtr context) { - const auto & settings = getContext()->getSettingsRef(); + const auto & settings = context->getSettingsRef(); const UInt64 min_query_duration = settings.log_queries_min_query_duration_ms.totalMilliseconds(); const QueryViewsLogElement::ViewStatus min_status = settings.log_queries_min_type; if (views.empty() || !settings.log_queries || !settings.log_query_views) @@ -650,10 +735,4 @@ void PushingToViewsBlockOutputStream::logQueryViews() } } - -void PushingToViewsBlockOutputStream::onProgress(const Progress & progress) -{ - if (getContext()->getProgressCallback()) - getContext()->getProgressCallback()(progress); -} } diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index 9466c5ef326..daaace1ee8b 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -91,6 +91,7 @@ public: protected: void transform(Chunk & chunk) override; + void onFinish() override; private: ViewRuntimeData view; From eccee47e72df27bc1ef201b69ef85a7ae49bfff6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 30 Aug 2021 14:03:39 +0300 Subject: [PATCH 004/142] Rewrite PushingToViewsBlockOutputStream part 2. --- .../PushingToViewsBlockOutputStream.cpp | 6 ++++++ src/Processors/Sinks/SinkToStorage.cpp | 8 ++++++++ src/Processors/Sinks/SinkToStorage.h | 16 ++++++++++++++++ 3 files changed, 30 insertions(+) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 9f9ed68851b..f12e19bda99 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -36,6 +36,7 @@ namespace DB struct ViewsData { std::vector views; + ContextPtr context; /// In case of exception happened while inserting into main table, it is pushed to pipeline. /// Remember the first one, we should keep them after view processing. @@ -112,6 +113,8 @@ private: ViewsDataPtr views_data; }; +static void logQueryViews(std::vector & views, ContextPtr context); + class FinalizingViewsTransform final : public IProcessor { struct ExceptionStatus @@ -202,6 +205,9 @@ public: view.setException(std::move(status.exception)); } } + + logQueryViews(views_data->views, views_data->context); + statuses.clear(); } diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index f538bb3c8f0..97fdf45884a 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -1,4 +1,6 @@ #include +#include +#include func, ExceptionKeepingTransform::RuntimeData * runtime_data) +{ + auto * original_thread = current_thread; + SCOPE_EXIT({ current_thread = original_thread; }); +} + void ExceptionKeepingTransform::work() { if (!was_on_start_called) diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index 3629b8714da..a5e8aab9226 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -5,6 +5,8 @@ namespace DB { +class ThreadStatus; + /// Has one input and one output. /// Works similarly to ISimpleTransform, but with much care about exceptions. /// @@ -46,6 +48,20 @@ public: InputPort & getInputPort() { return input; } OutputPort & getOutputPort() { return output; } + + struct RuntimeData + { + std::unique_ptr thread_status = nullptr; + UInt64 elapsed_ms = 0; + std::string additional_exception_message; + }; + + using RuntimeDataPtr = std::shared_ptr; + + void setRuntimeData(RuntimeDataPtr runtime_data_) { runtime_data = std::move(runtime_data_); } + +private: + RuntimeDataPtr runtime_data; }; From 5be05c2ef995395167c86ed5ae1ea831c96a21c8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 31 Aug 2021 16:50:07 +0300 Subject: [PATCH 005/142] Rewrite PushingToViewsBlockOutputStream part 3. --- .../PushingToViewsBlockOutputStream.cpp | 65 +++++++++++---- .../PushingToViewsBlockOutputStream.h | 4 +- src/Processors/Sinks/SinkToStorage.cpp | 82 ++++++++++++++----- src/Processors/Sinks/SinkToStorage.h | 6 +- .../MergeTree/ReplicatedMergeTreeSink.h | 6 +- 5 files changed, 121 insertions(+), 42 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index f12e19bda99..1ed72319fb8 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -35,13 +35,18 @@ namespace DB struct ViewsData { - std::vector views; + std::list views; ContextPtr context; /// In case of exception happened while inserting into main table, it is pushed to pipeline. /// Remember the first one, we should keep them after view processing. std::atomic_bool has_exception = false; std::exception_ptr first_exception; + + ViewsData(std::list views_, ContextPtr context_) + : views(std::move(views_)), context(std::move(context_)) + { + } }; using ViewsDataPtr = std::shared_ptr; @@ -113,7 +118,7 @@ private: ViewsDataPtr views_data; }; -static void logQueryViews(std::vector & views, ContextPtr context); +static void logQueryViews(std::list & views, ContextPtr context); class FinalizingViewsTransform final : public IProcessor { @@ -192,11 +197,12 @@ public: void work() override { - size_t num_views = statuses.size(); - for (size_t i = 0; i < num_views; ++i) + size_t i = 0; + for (auto & view : views_data->views) { - auto & view = views_data->views[i]; auto & status = statuses[i]; + ++i; + if (status.exception) { if (!any_exception) @@ -268,6 +274,11 @@ Drain buildPushingToViewsDrainImpl( { checkStackSize(); + /// If we don't write directly to the destination + /// then expect that we're inserting with precalculated virtual columns + auto storage_header = no_destination ? metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals()) + : metadata_snapshot->getSampleBlock(); + /** TODO This is a very important line. At any insertion into the table one of streams should own lock. * Although now any insertion into the table is done via PushingToViewsBlockOutputStream, * but it's clear that here is not the best place for this functionality. @@ -304,7 +315,8 @@ Drain buildPushingToViewsDrainImpl( insert_context->setSetting("min_insert_block_size_bytes", insert_settings.min_insert_block_size_bytes_for_materialized_views.value); } - std::vector views; + std::list views; + std::vector drains; for (const auto & database_table : dependencies) { @@ -349,7 +361,7 @@ Drain buildPushingToViewsDrainImpl( ASTPtr insert_query_ptr(insert.release()); InterpreterInsertQuery interpreter(insert_query_ptr, insert_context); BlockIO io = interpreter.execute(); - out = io.out; + out = std::move(io.out); } else if (const auto * live_view = dynamic_cast(dependent_table.get())) { @@ -393,8 +405,21 @@ Drain buildPushingToViewsDrainImpl( 0, std::chrono::system_clock::now(), QueryViewsLogElement::ViewStatus::EXCEPTION_BEFORE_START}; - views.emplace_back(ViewRuntimeData{std::move(query), database_table, std::move(out), nullptr, std::move(runtime_stats)}); + views.emplace_back(ViewRuntimeData{ + std::move(query), + out.getHeader(), + database_table, + dependent_table, + dependent_metadata_snapshot, + select_context, + nullptr, + std::move(runtime_stats)}); + + auto executing_inner_query = std::make_shared(storage_header, views.back()); + out.addTransform(std::move(executing_inner_query)); + + drains.emplace_back(std::move(out)); /// Add the view to the query access info so it can appear in system.query_log if (!no_destination) @@ -404,14 +429,24 @@ Drain buildPushingToViewsDrainImpl( } } + size_t num_views = views.size(); + if (num_views != 0) + { + auto views_data = std::make_shared(std::move(views), context); + auto copying_data = std::make_shared(storage_header, std::move(views_data)); + auto it = copying_data->getOutputs().begin(); + for (auto & drain : drains) + connect(*it, drain.getPort()); + + + } + /// Do not push to destination table if the flag is set if (!no_destination) { auto sink = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), context); metadata_snapshot->check(sink->getHeader().getColumnsWithTypeAndName()); - - auto replicated_output = dynamic_cast(sink.get()); } } @@ -632,8 +667,6 @@ void PushingToViewsBlockOutputStream::flush() static void process(Block & block, ViewRuntimeData & view) { - const auto & storage = view.storage; - const auto & metadata_snapshot = view.metadata_snapshot; const auto & context = view.context; /// We create a table with the same name as original table and the same alias columns, @@ -641,10 +674,10 @@ static void process(Block & block, ViewRuntimeData & view) /// InterpreterSelectQuery will do processing of alias columns. auto local_context = Context::createCopy(context); local_context->addViewSource(StorageValues::create( - storage->getStorageID(), - metadata_snapshot->getColumns(), + view.table_id, + view.metadata_snapshot->getColumns(), block, - storage->getVirtuals())); + view.storage->getVirtuals())); /// We need keep InterpreterSelectQuery, until the processing will be finished, since: /// @@ -716,7 +749,7 @@ void PushingToViewsBlockOutputStream::checkExceptionsInViews() } } -static void logQueryViews(std::vector & views, ContextPtr context) +static void logQueryViews(std::list & views, ContextPtr context) { const auto & settings = context->getSettingsRef(); const UInt64 min_query_duration = settings.log_queries_min_query_duration_ms.totalMilliseconds(); diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index daaace1ee8b..e7f800406d7 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -81,7 +81,7 @@ private: class ExecutingInnerQueryFromViewTransform final : public ExceptionKeepingTransform { public: - ExecutingInnerQueryFromViewTransform(const Block & header, ViewRuntimeData view_data) + ExecutingInnerQueryFromViewTransform(const Block & header, ViewRuntimeData & view_data) : ExceptionKeepingTransform(header, view_data.sample_block) , view(std::move(view_data)) { @@ -94,7 +94,7 @@ protected: void onFinish() override; private: - ViewRuntimeData view; + ViewRuntimeData & view; }; } diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index 97fdf45884a..8bf5fdf3fd2 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -1,6 +1,7 @@ #include #include -#include +#include namespace DB { @@ -40,7 +41,7 @@ IProcessor::Status ExceptionKeepingTransform::prepare() { if (input.isFinished()) { - if (!was_on_finish_called) + if (!was_on_finish_called && !has_exception) return Status::Ready; output.finish(); @@ -56,6 +57,7 @@ IProcessor::Status ExceptionKeepingTransform::prepare() if (data.exception) { + has_exception = true; output.pushData(std::move(data)); return Status::PortFull; } @@ -66,10 +68,46 @@ IProcessor::Status ExceptionKeepingTransform::prepare() return Status::Ready; } -static std::exception_ptr runStep(std::function func, ExceptionKeepingTransform::RuntimeData * runtime_data) +static std::exception_ptr runStep(std::function step, ExceptionKeepingTransform::RuntimeData * runtime_data) { auto * original_thread = current_thread; SCOPE_EXIT({ current_thread = original_thread; }); + + if (runtime_data && runtime_data->thread_status) + { + /// Change thread context to store individual metrics. Once the work in done, go back to the original thread + runtime_data->thread_status->resetPerformanceCountersLastUsage(); + current_thread = runtime_data->thread_status.get(); + } + + std::exception_ptr res; + Stopwatch watch; + + try + { + step(); + } + catch (Exception & exception) + { + if (runtime_data && !runtime_data->additional_exception_message.empty()) + exception.addMessage(runtime_data->additional_exception_message); + + res = std::current_exception(); + } + catch (...) + { + res = std::current_exception(); + } + + if (runtime_data) + { + if (runtime_data->thread_status) + runtime_data->thread_status->updatePerformanceCounters(); + + runtime_data->elapsed_ms += watch.elapsedMilliseconds(); + } + + return res; } void ExceptionKeepingTransform::work() @@ -77,35 +115,37 @@ void ExceptionKeepingTransform::work() if (!was_on_start_called) { was_on_start_called = true; - onStart(); - } - if (ready_input) + if (auto exception = runStep([this] { onStart(); }, runtime_data.get())) + { + has_exception = true; + ready_output = true; + data.exception = std::move(exception); + } + } + else if (ready_input) { ready_input = false; - ready_output = true; - try - { - transform(data.chunk); - } - catch (...) + if (auto exception = runStep([this] { transform(data.chunk); }, runtime_data.get())) { + has_exception = true; data.chunk.clear(); - data.exception = std::current_exception(); + data.exception = std::move(exception); } + + if (data.chunk || data.exception) + ready_output = true; } else if (!was_on_finish_called) { was_on_finish_called = true; - try + + if (auto exception = runStep([this] { onFinish(); }, runtime_data.get())) { - onFinish(); - } - catch (...) - { - ready_input = true; - data.exception = std::current_exception(); + has_exception = true; + ready_output = true; + data.exception = std::move(exception); } } } @@ -115,6 +155,8 @@ SinkToStorage::SinkToStorage(const Block & header) : ExceptionKeepingTransform(h void SinkToStorage::transform(Chunk & chunk) { consume(chunk.clone()); + if (lastBlockIsDuplicate()) + chunk.clear(); } } diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index a5e8aab9226..d66391833ec 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -19,9 +19,7 @@ class ThreadStatus; /// It is expected that output port won't be closed from the other side before all data is processed. /// /// Method onStart() is called before reading any data. -/// Exception from it is not pushed into pipeline, but thrown immediately. -/// -/// Method onFinish() is called after all data from input is processed. +/// Method onFinish() is called after all data from input is processed, if no exception happened. /// In case of exception, it is additionally pushed into pipeline. class ExceptionKeepingTransform : public IProcessor { @@ -32,6 +30,7 @@ private: bool ready_input = false; bool ready_output = false; + bool has_exception = false; bool was_on_start_called = false; bool was_on_finish_called = false; @@ -76,6 +75,7 @@ public: protected: virtual void consume(Chunk chunk) = 0; + virtual bool lastBlockIsDuplicate() const { return false; } private: std::vector table_locks; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 2a6702736df..75f002c6b42 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -44,8 +44,12 @@ public: void writeExistingPart(MergeTreeData::MutableDataPartPtr & part); /// For proper deduplication in MaterializedViews - bool lastBlockIsDuplicate() const + bool lastBlockIsDuplicate() const override { + /// If MV is responsible for deduplication, block is not considered duplicating. + if (context->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) + return false; + return last_block_is_duplicate; } From 79ecb6667e339f915fbea39bdb6e021eee0a1b64 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 1 Sep 2021 21:41:50 +0300 Subject: [PATCH 006/142] Rewrite PushingToViewsBlockOutputStream part 4. --- .../AddingDefaultBlockOutputStream.cpp | 19 +- .../AddingDefaultBlockOutputStream.h | 15 +- src/DataStreams/BlockIO.h | 3 +- .../CheckConstraintsBlockOutputStream.cpp | 40 +- .../CheckConstraintsBlockOutputStream.h | 19 +- src/DataStreams/CountingBlockOutputStream.cpp | 6 +- src/DataStreams/CountingBlockOutputStream.h | 19 +- .../PushingToViewsBlockOutputStream.cpp | 443 +++++------------- .../PushingToViewsBlockOutputStream.h | 56 +-- src/Interpreters/InterpreterInsertQuery.cpp | 76 +-- src/Interpreters/InterpreterInsertQuery.h | 7 +- src/Processors/Chain.cpp | 108 +++++ src/Processors/Chain.h | 41 ++ src/Processors/Drain.cpp | 75 --- src/Processors/Drain.h | 23 - src/Processors/Pipe.cpp | 35 ++ src/Processors/Pipe.h | 4 + src/Processors/QueryPipeline.cpp | 6 + src/Processors/QueryPipeline.h | 2 + src/Processors/Sinks/SinkToStorage.cpp | 9 +- src/Processors/Sinks/SinkToStorage.h | 27 +- 21 files changed, 446 insertions(+), 587 deletions(-) create mode 100644 src/Processors/Chain.cpp create mode 100644 src/Processors/Chain.h delete mode 100644 src/Processors/Drain.cpp delete mode 100644 src/Processors/Drain.h diff --git a/src/DataStreams/AddingDefaultBlockOutputStream.cpp b/src/DataStreams/AddingDefaultBlockOutputStream.cpp index 6f7975d492d..118926cfdd7 100644 --- a/src/DataStreams/AddingDefaultBlockOutputStream.cpp +++ b/src/DataStreams/AddingDefaultBlockOutputStream.cpp @@ -7,14 +7,14 @@ namespace DB { AddingDefaultBlockOutputStream::AddingDefaultBlockOutputStream( - const BlockOutputStreamPtr & output_, - const Block & header_, + const Block & in_header, + const Block & out_header, const ColumnsDescription & columns_, ContextPtr context_, bool null_as_default_) : output(output_), header(header_) { - auto dag = addMissingDefaults(header_, output->getHeader().getNamesAndTypesList(), columns_, context_, null_as_default_); + auto dag = addMissingDefaults(in_header, output_header.getNamesAndTypesList(), columns_, context_, null_as_default_); adding_defaults_actions = std::make_shared(std::move(dag), ExpressionActionsSettings::fromContext(context_, CompileExpressions::yes)); } @@ -25,19 +25,6 @@ void AddingDefaultBlockOutputStream::write(const Block & block) output->write(copy); } -void AddingDefaultBlockOutputStream::flush() -{ - output->flush(); -} -void AddingDefaultBlockOutputStream::writePrefix() -{ - output->writePrefix(); -} - -void AddingDefaultBlockOutputStream::writeSuffix() -{ - output->writeSuffix(); -} } diff --git a/src/DataStreams/AddingDefaultBlockOutputStream.h b/src/DataStreams/AddingDefaultBlockOutputStream.h index 45ff30a3daa..237bd994613 100644 --- a/src/DataStreams/AddingDefaultBlockOutputStream.h +++ b/src/DataStreams/AddingDefaultBlockOutputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -20,18 +20,17 @@ class Context; * Also the stream can substitute NULL into DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1. * All three types of columns are materialized (not constants). */ -class AddingDefaultBlockOutputStream : public IBlockOutputStream +class AddingMissingDefaultsTransform : public ISimpleTransform { public: - AddingDefaultBlockOutputStream( - const BlockOutputStreamPtr & output_, - const Block & header_, + AddingMissingDefaultsTransform( + const Block & in_header, + const Block & out_header, const ColumnsDescription & columns_, ContextPtr context_, bool null_as_default_ = false); - Block getHeader() const override { return header; } - void write(const Block & block) override; + void transform(Chunk & chunk) override; void flush() override; @@ -39,8 +38,6 @@ public: void writeSuffix() override; private: - BlockOutputStreamPtr output; - const Block header; ExpressionActionsPtr adding_defaults_actions; }; diff --git a/src/DataStreams/BlockIO.h b/src/DataStreams/BlockIO.h index 31a0e1020d2..7c8edc3318a 100644 --- a/src/DataStreams/BlockIO.h +++ b/src/DataStreams/BlockIO.h @@ -5,6 +5,7 @@ #include #include +#include namespace DB @@ -25,7 +26,7 @@ struct BlockIO std::shared_ptr process_list_entry; - BlockOutputStreamPtr out; + Chain out; BlockInputStreamPtr in; QueryPipeline pipeline; diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index fbf4a777032..9dace69d38f 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -22,26 +22,24 @@ namespace ErrorCodes } -CheckConstraintsBlockOutputStream::CheckConstraintsBlockOutputStream( +CheckConstraintsTransform::CheckConstraintsTransform( const StorageID & table_id_, - const BlockOutputStreamPtr & output_, - const Block & header_, + const Block & header, const ConstraintsDescription & constraints_, ContextPtr context_) - : table_id(table_id_), - output(output_), - header(header_), - constraints(constraints_), - expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())) + : ISimpleTransform(header, header, false) + , table_id(table_id_) + , constraints(constraints_) + , expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())) { } -void CheckConstraintsBlockOutputStream::write(const Block & block) +void CheckConstraintsTransform::transform(Chunk & chunk) { - if (block.rows() > 0) + if (chunk.getNumRows() > 0) { - Block block_to_calculate = block; + Block block_to_calculate = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); for (size_t i = 0; i < expressions.size(); ++i) { auto constraint_expr = expressions[i]; @@ -101,7 +99,7 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) column_values_msg.reserve(approx_bytes_for_col * related_columns.size()); for (const auto & name : related_columns) { - const IColumn & column = *block.getByName(name).column; + const IColumn & column = *chunk.getColumns()[getInputPort().getHeader().getPositionByName(name)]; assert(row_idx < column.size()); if (!first) @@ -124,23 +122,7 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) } } - output->write(block); - rows_written += block.rows(); -} - -void CheckConstraintsBlockOutputStream::flush() -{ - output->flush(); -} - -void CheckConstraintsBlockOutputStream::writePrefix() -{ - output->writePrefix(); -} - -void CheckConstraintsBlockOutputStream::writeSuffix() -{ - output->writeSuffix(); + rows_written += chunk.getNumRows(); } } diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.h b/src/DataStreams/CheckConstraintsBlockOutputStream.h index 0f115550eb8..aa6d96aa5d3 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.h +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -12,28 +12,21 @@ namespace DB * Otherwise just pass block to output unchanged. */ -class CheckConstraintsBlockOutputStream : public IBlockOutputStream +class CheckConstraintsTransform final : public ISimpleTransform { public: - CheckConstraintsBlockOutputStream( + CheckConstraintsTransform( const StorageID & table_, - const BlockOutputStreamPtr & output_, - const Block & header_, + const Block & header, const ConstraintsDescription & constraints_, ContextPtr context_); - Block getHeader() const override { return header; } - void write(const Block & block) override; + String getName() const override { return "CheckConstraintsTransform"; } - void flush() override; - - void writePrefix() override; - void writeSuffix() override; + void transform(Chunk & chunk) override; private: StorageID table_id; - BlockOutputStreamPtr output; - Block header; const ConstraintsDescription constraints; const ConstraintsExpressions expressions; size_t rows_written = 0; diff --git a/src/DataStreams/CountingBlockOutputStream.cpp b/src/DataStreams/CountingBlockOutputStream.cpp index 6594b3b2ce1..e17dc80ca13 100644 --- a/src/DataStreams/CountingBlockOutputStream.cpp +++ b/src/DataStreams/CountingBlockOutputStream.cpp @@ -12,11 +12,9 @@ namespace ProfileEvents namespace DB { -void CountingBlockOutputStream::write(const Block & block) +void CountingTransform::transform(Chunk & chunk) { - stream->write(block); - - Progress local_progress(block.rows(), block.bytes(), 0); + Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); progress.incrementPiecewiseAtomically(local_progress); ProfileEvents::increment(ProfileEvents::InsertedRows, local_progress.read_rows); diff --git a/src/DataStreams/CountingBlockOutputStream.h b/src/DataStreams/CountingBlockOutputStream.h index c7247b39945..702a9f190f8 100644 --- a/src/DataStreams/CountingBlockOutputStream.h +++ b/src/DataStreams/CountingBlockOutputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,11 +9,12 @@ namespace DB /// Proxy class which counts number of written block, rows, bytes -class CountingBlockOutputStream : public IBlockOutputStream +class CountingTransform final : public ISimpleTransform { public: - CountingBlockOutputStream(const BlockOutputStreamPtr & stream_) - : stream(stream_) {} + explicit CountingTransform(const Block & header) : ISimpleTransform(header, header, false) {} + + String getName() const override { return "CountingTransform"; } void setProgressCallback(const ProgressCallback & callback) { @@ -30,17 +31,9 @@ public: return progress; } - Block getHeader() const override { return stream->getHeader(); } - void write(const Block & block) override; - - void writePrefix() override { stream->writePrefix(); } - void writeSuffix() override { stream->writeSuffix(); } - void flush() override { stream->flush(); } - void onProgress(const Progress & current_progress) override { stream->onProgress(current_progress); } - String getContentType() const override { return stream->getContentType(); } + void transform(Chunk & chunk) override; protected: - BlockOutputStreamPtr stream; Progress progress; ProgressCallback progress_callback; QueryStatus * process_elem = nullptr; diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 1ed72319fb8..4ad64206442 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -1,10 +1,4 @@ -#include -#include -#include -#include #include -#include -#include #include #include #include @@ -19,12 +13,9 @@ #include #include #include -#include #include #include #include -#include -#include #include #include @@ -128,9 +119,18 @@ class FinalizingViewsTransform final : public IProcessor bool is_first = false; }; + static InputPorts initPorts(std::vector headers) + { + InputPorts res; + for (auto & header : headers) + res.emplace_back(std::move(header)); + + return res; + } + public: - FinalizingViewsTransform(const Block & header, ViewsDataPtr data) - : IProcessor(InputPorts(data->views.size(), header), {header}) + FinalizingViewsTransform(std::vector headers, ViewsDataPtr data) + : IProcessor(initPorts(std::move(headers)), {Block()}) , output(outputs.front()) , views_data(std::move(data)) { @@ -224,53 +224,54 @@ private: std::exception_ptr any_exception; }; -class ExceptionHandlingSink : public IProcessor -{ -public: - explicit ExceptionHandlingSink(Block header) - : IProcessor({std::move(header)}, {}) - , input(inputs.front()) - { - } +//class ExceptionHandlingSink : public IProcessor +//{ +//public: +// explicit ExceptionHandlingSink(Block header) +// : IProcessor({std::move(header)}, {}) +// , input(inputs.front()) +// { +// } +// +// Status prepare() override +// { +// while (!input.isFinished()) +// { +// input.setNeeded(); +// if (!input.hasData()) +// return Status::NeedData; +// +// auto data = input.pullData(); +// if (data.exception) +// exceptions.emplace_back(std::move(data.exception)); +// } +// +// if (!exceptions.empty()) +// return Status::Ready; +// +// return Status::Finished; +// } +// +// void work() override +// { +// auto exception = std::move(exceptions.at(0)); +// exceptions.clear(); +// std::rethrow_exception(std::move(exception)); +// } +// +//private: +// InputPort & input; +// std::vector exceptions; +//}; - Status prepare() override - { - while (!input.isFinished()) - { - input.setNeeded(); - if (!input.hasData()) - return Status::NeedData; - - auto data = input.pullData(); - if (data.exception) - exceptions.emplace_back(std::move(data.exception)); - } - - if (!exceptions.empty()) - return Status::Ready; - - return Status::Finished; - } - - void work() override - { - auto exception = std::move(exceptions.at(0)); - exceptions.clear(); - std::rethrow_exception(std::move(exception)); - } - -private: - InputPort & input; - std::vector exceptions; -}; - -Drain buildPushingToViewsDrainImpl( +Chain buildPushingToViewsDrain( const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, const ASTPtr & query_ptr, bool no_destination, - std::vector & locks) + std::vector & locks, + ExceptionKeepingTransformRuntimeDataPtr runtime_data) { checkStackSize(); @@ -316,7 +317,7 @@ Drain buildPushingToViewsDrainImpl( } std::list views; - std::vector drains; + std::vector chains; for (const auto & database_table : dependencies) { @@ -324,10 +325,38 @@ Drain buildPushingToViewsDrainImpl( auto dependent_metadata_snapshot = dependent_table->getInMemoryMetadataPtr(); ASTPtr query; - Drain out; + Chain out; QueryViewsLogElement::ViewType type = QueryViewsLogElement::ViewType::DEFAULT; String target_name = database_table.getFullTableName(); + /// If the materialized view is executed outside of a query, for example as a result of SYSTEM FLUSH LOGS or + /// SYSTEM FLUSH DISTRIBUTED ..., we can't attach to any thread group and we won't log, so there is no point on collecting metrics + std::unique_ptr thread_status = nullptr; + + ThreadGroupStatusPtr running_group = current_thread && current_thread->getThreadGroup() + ? current_thread->getThreadGroup() + : MainThreadStatus::getInstance().getThreadGroup(); + if (running_group) + { + /// We are creating a ThreadStatus per view to store its metrics individually + /// Since calling ThreadStatus() changes current_thread we save it and restore it after the calls + /// Later on, before doing any task related to a view, we'll switch to its ThreadStatus, do the work, + /// and switch back to the original thread_status. + auto * original_thread = current_thread; + SCOPE_EXIT({ current_thread = original_thread; }); + + thread_status = std::make_unique(); + /// Disable query profiler for this ThreadStatus since the running (main query) thread should already have one + /// If we didn't disable it, then we could end up with N + 1 (N = number of dependencies) profilers which means + /// N times more interruptions + thread_status->disableProfiling(); + thread_status->attachQuery(running_group); + } + + auto view_runtime_data = std::make_shared( + std::move(thread_status), + database_table.getNameForLogs()); + if (auto * materialized_view = dynamic_cast(dependent_table.get())) { type = QueryViewsLogElement::ViewType::MATERIALIZED; @@ -359,7 +388,7 @@ Drain buildPushingToViewsDrainImpl( insert->columns = std::move(list); ASTPtr insert_query_ptr(insert.release()); - InterpreterInsertQuery interpreter(insert_query_ptr, insert_context); + InterpreterInsertQuery interpreter(insert_query_ptr, insert_context, false, false, false, runtime_data); BlockIO io = interpreter.execute(); out = std::move(io.out); } @@ -367,36 +396,12 @@ Drain buildPushingToViewsDrainImpl( { type = QueryViewsLogElement::ViewType::LIVE; query = live_view->getInnerQuery(); // Used only to log in system.query_views_log - out = buildPushingToViewsDrainImpl( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true, locks); + out = buildPushingToViewsDrain( + dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true, locks, view_runtime_data); } else - out = buildPushingToViewsDrainImpl( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), false, locks); - - /// If the materialized view is executed outside of a query, for example as a result of SYSTEM FLUSH LOGS or - /// SYSTEM FLUSH DISTRIBUTED ..., we can't attach to any thread group and we won't log, so there is no point on collecting metrics - std::unique_ptr thread_status = nullptr; - - ThreadGroupStatusPtr running_group = current_thread && current_thread->getThreadGroup() - ? current_thread->getThreadGroup() - : MainThreadStatus::getInstance().getThreadGroup(); - if (running_group) - { - /// We are creating a ThreadStatus per view to store its metrics individually - /// Since calling ThreadStatus() changes current_thread we save it and restore it after the calls - /// Later on, before doing any task related to a view, we'll switch to its ThreadStatus, do the work, - /// and switch back to the original thread_status. - auto * original_thread = current_thread; - SCOPE_EXIT({ current_thread = original_thread; }); - - thread_status = std::make_unique(); - /// Disable query profiler for this ThreadStatus since the running (main query) thread should already have one - /// If we didn't disable it, then we could end up with N + 1 (N = number of dependencies) profilers which means - /// N times more interruptions - thread_status->disableProfiling(); - thread_status->attachQuery(running_group); - } + out = buildPushingToViewsDrain( + dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), false, locks, view_runtime_data); QueryViewsLogElement::ViewRuntimeStats runtime_stats{ target_name, @@ -408,7 +413,7 @@ Drain buildPushingToViewsDrainImpl( views.emplace_back(ViewRuntimeData{ std::move(query), - out.getHeader(), + out.getInputHeader(), database_table, dependent_table, dependent_metadata_snapshot, @@ -417,9 +422,10 @@ Drain buildPushingToViewsDrainImpl( std::move(runtime_stats)}); auto executing_inner_query = std::make_shared(storage_header, views.back()); - out.addTransform(std::move(executing_inner_query)); + executing_inner_query->setRuntimeData(view_runtime_data); - drains.emplace_back(std::move(out)); + out.addSource(std::move(executing_inner_query)); + chains.emplace_back(std::move(out)); /// Add the view to the query access info so it can appear in system.query_log if (!no_destination) @@ -429,240 +435,46 @@ Drain buildPushingToViewsDrainImpl( } } + Chain result_chain; + size_t num_views = views.size(); if (num_views != 0) { + std::vector headers; + headers.reserve(num_views); + for (const auto & chain : chains) + headers.push_back(chain.getOutputHeader()); + auto views_data = std::make_shared(std::move(views), context); - auto copying_data = std::make_shared(storage_header, std::move(views_data)); - auto it = copying_data->getOutputs().begin(); - for (auto & drain : drains) - connect(*it, drain.getPort()); + auto copying_data = std::make_shared(storage_header, views_data); + auto finalizing_views = std::make_shared(std::move(headers), views_data); + auto out = copying_data->getOutputs().begin(); + auto in = finalizing_views->getInputs().begin(); + std::list processors; + for (auto & chain : chains) + { + connect(*out, chain.getInputPort()); + connect(chain.getOutputPort(), *in); + processors.splice(processors.end(), Chain::getProcessors(std::move(chain))); + } + + processors.emplace_front(std::move(copying_data)); + processors.emplace_back(std::move(finalizing_views)); + result_chain = Chain(std::move(processors)); } /// Do not push to destination table if the flag is set if (!no_destination) { auto sink = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), context); - + sink->setRuntimeData(runtime_data); metadata_snapshot->check(sink->getHeader().getColumnsWithTypeAndName()); - } -} - -Block PushingToViewsBlockOutputStream::getHeader() const -{ - /// If we don't write directly to the destination - /// then expect that we're inserting with precalculated virtual columns - if (output) - return metadata_snapshot->getSampleBlock(); - else - return metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals()); -} - -/// Auxiliary function to do the setup and teardown to run a view individually and collect its metrics inside the view ThreadStatus -void inline runViewStage(ViewRuntimeData & view, const std::string & action, std::function stage) -{ - Stopwatch watch; - - auto * original_thread = current_thread; - SCOPE_EXIT({ current_thread = original_thread; }); - - if (view.runtime_stats.thread_status) - { - /// Change thread context to store individual metrics per view. Once the work in done, go back to the original thread - view.runtime_stats.thread_status->resetPerformanceCountersLastUsage(); - current_thread = view.runtime_stats.thread_status.get(); + result_chain.addSource(std::move(sink)); } - try - { - stage(); - } - catch (Exception & ex) - { - ex.addMessage(action + " " + view.table_id.getNameForLogs()); - view.setException(std::current_exception()); - } - catch (...) - { - view.setException(std::current_exception()); - } - - if (view.runtime_stats.thread_status) - view.runtime_stats.thread_status->updatePerformanceCounters(); - view.runtime_stats.elapsed_ms += watch.elapsedMilliseconds(); -} - -void PushingToViewsBlockOutputStream::write(const Block & block) -{ - /** Throw an exception if the sizes of arrays - elements of nested data structures doesn't match. - * We have to make this assertion before writing to table, because storage engine may assume that they have equal sizes. - * NOTE It'd better to do this check in serialization of nested structures (in place when this assumption is required), - * but currently we don't have methods for serialization of nested structures "as a whole". - */ - Nested::validateArraySizes(block); - - if (auto * live_view = dynamic_cast(storage.get())) - { - StorageLiveView::writeIntoLiveView(*live_view, block, getContext()); - } - else - { - if (output) - /// TODO: to support virtual and alias columns inside MVs, we should return here the inserted block extended - /// with additional columns directly from storage and pass it to MVs instead of raw block. - output->write(block); - } - - if (views.empty()) - return; - - /// Don't process materialized views if this block is duplicate - const Settings & settings = getContext()->getSettingsRef(); - if (!settings.deduplicate_blocks_in_dependent_materialized_views && replicated_output && replicated_output->lastBlockIsDuplicate()) - return; - - size_t max_threads = 1; - if (settings.parallel_view_processing) - max_threads = settings.max_threads ? std::min(static_cast(settings.max_threads), views.size()) : views.size(); - if (max_threads > 1) - { - ThreadPool pool(max_threads); - for (auto & view : views) - { - pool.scheduleOrThrowOnError([&] { - setThreadName("PushingToViews"); - runViewStage(view, "while pushing to view", [&]() { process(block, view); }); - }); - } - pool.wait(); - } - else - { - for (auto & view : views) - { - runViewStage(view, "while pushing to view", [&]() { process(block, view); }); - } - } -} - -void PushingToViewsBlockOutputStream::writePrefix() -{ - if (output) - output->writePrefix(); - - for (auto & view : views) - { - runViewStage(view, "while writing prefix to view", [&] { view.out->writePrefix(); }); - if (view.exception) - { - logQueryViews(); - std::rethrow_exception(view.exception); - } - } -} - -void PushingToViewsBlockOutputStream::writeSuffix() -{ - if (output) - output->writeSuffix(); - - if (views.empty()) - return; - - auto process_suffix = [](ViewRuntimeData & view) - { - view.out->writeSuffix(); - view.runtime_stats.setStatus(QueryViewsLogElement::ViewStatus::QUERY_FINISH); - }; - static std::string stage_step = "while writing suffix to view"; - - /// Run writeSuffix() for views in separate thread pool. - /// In could have been done in PushingToViewsBlockOutputStream::process, however - /// it is not good if insert into main table fail but into view succeed. - const Settings & settings = getContext()->getSettingsRef(); - size_t max_threads = 1; - if (settings.parallel_view_processing) - max_threads = settings.max_threads ? std::min(static_cast(settings.max_threads), views.size()) : views.size(); - bool exception_happened = false; - if (max_threads > 1) - { - ThreadPool pool(max_threads); - std::atomic_uint8_t exception_count = 0; - for (auto & view : views) - { - if (view.exception) - { - exception_happened = true; - continue; - } - pool.scheduleOrThrowOnError([&] { - setThreadName("PushingToViews"); - - runViewStage(view, stage_step, [&] { process_suffix(view); }); - if (view.exception) - { - exception_count.fetch_add(1, std::memory_order_relaxed); - } - else - { - LOG_TRACE( - log, - "Pushing (parallel {}) from {} to {} took {} ms.", - max_threads, - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - view.runtime_stats.elapsed_ms); - } - }); - } - pool.wait(); - exception_happened |= exception_count.load(std::memory_order_relaxed) != 0; - } - else - { - for (auto & view : views) - { - if (view.exception) - { - exception_happened = true; - continue; - } - runViewStage(view, stage_step, [&] { process_suffix(view); }); - if (view.exception) - { - exception_happened = true; - } - else - { - LOG_TRACE( - log, - "Pushing (sequentially) from {} to {} took {} ms.", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - view.runtime_stats.elapsed_ms); - } - } - } - if (exception_happened) - checkExceptionsInViews(); - - if (views.size() > 1) - { - UInt64 milliseconds = main_watch.elapsedMilliseconds(); - LOG_DEBUG(log, "Pushing from {} to {} views took {} ms.", storage->getStorageID().getNameForLogs(), views.size(), milliseconds); - } - logQueryViews(); -} - -void PushingToViewsBlockOutputStream::flush() -{ - if (output) - output->flush(); - - for (auto & view : views) - view.out->flush(); + return result_chain; } static void process(Block & block, ViewRuntimeData & view) @@ -729,24 +541,9 @@ static void process(Block & block, ViewRuntimeData & view) void ExecutingInnerQueryFromViewTransform::transform(Chunk & chunk) { - runViewStage(view, "while pushing to view", [&] - { - auto block = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); - process(block, view); - chunk.setColumns(block.getColumns(), block.rows()); - }); -} - -void PushingToViewsBlockOutputStream::checkExceptionsInViews() -{ - for (auto & view : views) - { - if (view.exception) - { - logQueryViews(); - std::rethrow_exception(view.exception); - } - } + auto block = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); + process(block, view); + chunk.setColumns(block.getColumns(), block.rows()); } static void logQueryViews(std::list & views, ContextPtr context) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index e7f800406d7..dfa7a9593d7 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -3,10 +3,11 @@ #include #include #include -#include -#include -#include +#include #include +#include +#include +#include namespace Poco { @@ -18,6 +19,9 @@ namespace DB class ReplicatedMergeTreeSink; +struct ExceptionKeepingTransformRuntimeData; +using ExceptionKeepingTransformRuntimeDataPtr = std::shared_ptr; + struct ViewRuntimeData { const ASTPtr query; @@ -41,49 +45,22 @@ struct ViewRuntimeData /** Writes data to the specified table and to all dependent materialized views. */ -class PushingToViewsBlockOutputStream : public IBlockOutputStream, WithContext -{ -public: - PushingToViewsBlockOutputStream( - const StoragePtr & storage_, - const StorageMetadataPtr & metadata_snapshot_, - ContextPtr context_, - const ASTPtr & query_ptr_, - bool no_destination = false); +Chain buildPushingToViewsDrain( + const StoragePtr & storage, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + const ASTPtr & query_ptr, + bool no_destination, + std::vector & locks, + ExceptionKeepingTransformRuntimeDataPtr runtime_data); - Block getHeader() const override; - void write(const Block & block) override; - - void flush() override; - void writePrefix() override; - void writeSuffix() override; - void onProgress(const Progress & progress) override; - -private: - StoragePtr storage; - StorageMetadataPtr metadata_snapshot; - BlockOutputStreamPtr output; - ReplicatedMergeTreeSink * replicated_output = nullptr; - Poco::Logger * log; - - ASTPtr query_ptr; - Stopwatch main_watch; - - std::vector views; - ContextMutablePtr select_context; - ContextMutablePtr insert_context; - - void process(const Block & block, ViewRuntimeData & view); - void checkExceptionsInViews(); - void logQueryViews(); -}; class ExecutingInnerQueryFromViewTransform final : public ExceptionKeepingTransform { public: ExecutingInnerQueryFromViewTransform(const Block & header, ViewRuntimeData & view_data) : ExceptionKeepingTransform(header, view_data.sample_block) - , view(std::move(view_data)) + , view(view_data) { } @@ -91,7 +68,6 @@ public: protected: void transform(Chunk & chunk) override; - void onFinish() override; private: ViewRuntimeData & view; diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 3589176f231..b150b959f87 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -19,7 +19,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -28,6 +30,7 @@ #include #include #include +#include #include #include @@ -43,12 +46,14 @@ namespace ErrorCodes } InterpreterInsertQuery::InterpreterInsertQuery( - const ASTPtr & query_ptr_, ContextPtr context_, bool allow_materialized_, bool no_squash_, bool no_destination_) + const ASTPtr & query_ptr_, ContextPtr context_, bool allow_materialized_, bool no_squash_, bool no_destination_, + ExceptionKeepingTransformRuntimeDataPtr runtime_data_) : WithContext(context_) , query_ptr(query_ptr_) , allow_materialized(allow_materialized_) , no_squash(no_squash_) , no_destination(no_destination_) + , runtime_data(runtime_data_) { checkStackSize(); } @@ -174,7 +179,7 @@ BlockIO InterpreterInsertQuery::execute() } } - BlockOutputStreams out_streams; + std::vector out_chains; if (!is_distributed_insert_select || query.watch) { size_t out_streams_size = 1; @@ -266,28 +271,45 @@ BlockIO InterpreterInsertQuery::execute() for (size_t i = 0; i < out_streams_size; i++) { /// We create a pipeline of several streams, into which we will write data. - BlockOutputStreamPtr out; + Chain out; /// NOTE: we explicitly ignore bound materialized views when inserting into Kafka Storage. /// Otherwise we'll get duplicates when MV reads same rows again from Kafka. if (table->noPushingToViews() && !no_destination) - out = std::make_shared(table->write(query_ptr, metadata_snapshot, getContext())); + { + auto sink = table->write(query_ptr, metadata_snapshot, getContext()); + sink->setRuntimeData(runtime_data); + out.addSource(std::move(sink)); + } else - out = std::make_shared(table, metadata_snapshot, getContext(), query_ptr, no_destination); + { + std::vector locks; + out = buildPushingToViewsDrain(table, metadata_snapshot, getContext(), query_ptr, no_destination, locks, runtime_data); + for (auto & lock : locks) + res.pipeline.addTableLock(std::move(lock)); + } /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. /// Checking constraints. It must be done after calculation of all defaults, so we can check them on calculated columns. if (const auto & constraints = metadata_snapshot->getConstraints(); !constraints.empty()) - out = std::make_shared( - query.table_id, out, out->getHeader(), metadata_snapshot->getConstraints(), getContext()); + out.addSource(std::make_shared( + query.table_id, out.getInputHeader(), metadata_snapshot->getConstraints(), getContext())); bool null_as_default = query.select && getContext()->getSettingsRef().insert_null_as_default; + auto adding_missing_defaults_dag = addMissingDefaults( + query_sample_block, + out.getInputHeader().getNamesAndTypesList(), + metadata_snapshot->getColumns(), + getContext(), + null_as_default); + + auto adding_missing_defaults_actions = std::make_shared(adding_missing_defaults_dag); + /// Actually we don't know structure of input blocks from query/table, /// because some clients break insertion protocol (columns != header) - out = std::make_shared( - out, query_sample_block, metadata_snapshot->getColumns(), getContext(), null_as_default); + out.addSource(std::make_shared(query_sample_block, adding_missing_defaults_actions)); /// It's important to squash blocks as early as possible (before other transforms), /// because other transforms may work inefficient if block size is small. @@ -298,16 +320,17 @@ BlockIO InterpreterInsertQuery::execute() { bool table_prefers_large_blocks = table->prefersLargeBlocks(); - out = std::make_shared( - out, - out->getHeader(), + out.addSource(std::make_shared( + out.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0); + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0)); } - auto out_wrapper = std::make_shared(out); - out_wrapper->setProcessListElement(getContext()->getProcessListElement()); - out_streams.emplace_back(std::move(out_wrapper)); + auto counting = std::make_shared(out.getInputHeader()); + counting->setProcessListElement(getContext()->getProcessListElement()); + out.addSource(std::move(counting)); + + out_chains.emplace_back(std::move(out)); } } @@ -318,7 +341,7 @@ BlockIO InterpreterInsertQuery::execute() } else if (query.select || query.watch) { - const auto & header = out_streams.at(0)->getHeader(); + const auto & header = out_chains.at(0).getInputHeader(); auto actions_dag = ActionsDAG::makeConvertingActions( res.pipeline.getHeader().getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), @@ -330,15 +353,11 @@ BlockIO InterpreterInsertQuery::execute() return std::make_shared(in_header, actions); }); - res.pipeline.setSinks([&](const Block &, QueryPipeline::StreamType type) -> ProcessorPtr + res.pipeline.addChains(std::move(out_chains)); + + res.pipeline.setSinks([&](const Block & cur_header, QueryPipeline::StreamType) -> ProcessorPtr { - if (type != QueryPipeline::StreamType::Main) - return nullptr; - - auto stream = std::move(out_streams.back()); - out_streams.pop_back(); - - return std::make_shared(std::move(stream)); + return std::make_shared(cur_header); }); if (!allow_materialized) @@ -353,13 +372,14 @@ BlockIO InterpreterInsertQuery::execute() auto pipe = getSourceFromFromASTInsertQuery(query_ptr, nullptr, query_sample_block, getContext(), nullptr); res.pipeline.init(std::move(pipe)); res.pipeline.resize(1); - res.pipeline.setSinks([&](const Block &, Pipe::StreamType) + res.pipeline.addChains(std::move(out_chains)); + res.pipeline.setSinks([&](const Block & cur_header, Pipe::StreamType) { - return std::make_shared(out_streams.at(0)); + return std::make_shared(cur_header); }); } else - res.out = std::move(out_streams.at(0)); + res.out = std::move(out_chains.at(0)); res.pipeline.addStorageHolder(table); if (const auto * mv = dynamic_cast(table.get())) diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 71b8c827702..380d7550ed8 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -9,6 +9,9 @@ namespace DB { +struct ExceptionKeepingTransformRuntimeData; +using ExceptionKeepingTransformRuntimeDataPtr = std::shared_ptr; + /** Interprets the INSERT query. */ class InterpreterInsertQuery : public IInterpreter, WithContext @@ -19,7 +22,8 @@ public: ContextPtr context_, bool allow_materialized_ = false, bool no_squash_ = false, - bool no_destination_ = false); + bool no_destination_ = false, + ExceptionKeepingTransformRuntimeDataPtr runtime_data = nullptr); /** Prepare a request for execution. Return block streams * - the stream into which you can write data to execute the query, if INSERT; @@ -40,6 +44,7 @@ private: const bool allow_materialized; const bool no_squash; const bool no_destination; + ExceptionKeepingTransformRuntimeDataPtr runtime_data; }; diff --git a/src/Processors/Chain.cpp b/src/Processors/Chain.cpp new file mode 100644 index 00000000000..6f0320c86ac --- /dev/null +++ b/src/Processors/Chain.cpp @@ -0,0 +1,108 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +static void checkSingleInput(const IProcessor & transform) +{ + if (transform.getInputs().size() != 1) + throw Exception("Transform for chain should have single input, " + "but " + transform.getName() + " has " + + toString(transform.getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR); + + if (transform.getInputs().front().isConnected()) + throw Exception("Transform for chain has connected input.", ErrorCodes::LOGICAL_ERROR); +} + +static void checkSingleOutput(const IProcessor & transform) +{ + if (transform.getOutputs().size() != 1) + throw Exception("Transform for chain should have single output, " + "but " + transform.getName() + " has " + + toString(transform.getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); + + if (transform.getOutputs().front().isConnected()) + throw Exception("Transform for chain has connected input.", ErrorCodes::LOGICAL_ERROR); +} + +static void checkTransform(const IProcessor & transform) +{ + checkSingleInput(transform); + checkSingleOutput(transform); +} + +static void checkInitialized(const std::list & processors) +{ + if (processors.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Drain is not initialized"); +} + +Chain::Chain(ProcessorPtr processor) +{ + checkTransform(*processor); + processors.emplace_back(std::move(processor)); +} + +Chain::Chain(std::list processors_) : processors(std::move(processors_)) +{ + if (processors.empty()) + return; + + checkSingleInput(*processors.front()); + checkSingleOutput(*processors.back()); + + for (const auto & processor : processors) + { + for (const auto & input : processor->getInputs()) + if (&input != &getInputPort() && !input.isConnected()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot initialize chain because there is a not connected input for {}", + processor->getName()); + + for (const auto & output : processor->getOutputs()) + if (&output != &getOutputPort() && !output.isConnected()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot initialize chain because there is a not connected output for {}", + processor->getName()); + } +} + +void Chain::addSource(ProcessorPtr processor) +{ + checkTransform(*processor); + + if (!processors.empty()) + connect(processor->getOutputs().front(), getInputPort()); + + processors.emplace_front(std::move(processor)); +} + +void Chain::addSink(ProcessorPtr processor) +{ + checkTransform(*processor); + + if (!processors.empty()) + connect(getOutputPort(), processor->getInputs().front()); + + processors.emplace_front(std::move(processor)); +} + +InputPort & Chain::getInputPort() const +{ + checkInitialized(processors); + return processors.front()->getInputs().front(); +} + +OutputPort & Chain::getOutputPort() const +{ + checkInitialized(processors); + return processors.back()->getOutputs().front(); +} + +} diff --git a/src/Processors/Chain.h b/src/Processors/Chain.h new file mode 100644 index 00000000000..459fd9fbc7e --- /dev/null +++ b/src/Processors/Chain.h @@ -0,0 +1,41 @@ +#pragma once + +#include + +namespace DB +{ + +class Chain +{ +public: + Chain() = default; + Chain(Chain &&) = default; + Chain(const Chain &) = delete; + + Chain & operator=(Chain &&) = default; + Chain & operator=(const Chain &) = delete; + + explicit Chain(ProcessorPtr processor); + explicit Chain(std::list processors); + + bool empty() const { return processors.empty(); } + + void addSource(ProcessorPtr processor); + void addSink(ProcessorPtr processor); + + InputPort & getInputPort() const; + OutputPort & getOutputPort() const; + + const Block & getInputHeader() const { return getInputPort().getHeader(); } + const Block & getOutputHeader() const { return getOutputPort().getHeader(); } + + static std::list getProcessors(Chain chain) { return std::move(chain.processors); } + +private: + /// -> source -> transform -> ... -> transform -> sink -> + /// ^ -> -> -> -> ^ + /// input port output port + std::list processors; +}; + +} diff --git a/src/Processors/Drain.cpp b/src/Processors/Drain.cpp deleted file mode 100644 index a6a010f0a42..00000000000 --- a/src/Processors/Drain.cpp +++ /dev/null @@ -1,75 +0,0 @@ -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -static void checkSink(const IProcessor & sink) -{ - if (!sink.getOutputs().empty()) - throw Exception("Sink for drain shouldn't have any output, but " + sink.getName() + " has " + - toString(sink.getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); - - if (sink.getInputs().empty()) - throw Exception("Sink for drain should have single input, but it doesn't have any", - ErrorCodes::LOGICAL_ERROR); - - if (sink.getInputs().size() > 1) - throw Exception("Sink for drain should have single input, but " + sink.getName() + " has " + - toString(sink.getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR); - - if (sink.getInputs().front().isConnected()) - throw Exception("Sink for drain has connected input.", ErrorCodes::LOGICAL_ERROR); -} - -static void checkTransform(const IProcessor & transform) -{ - if (transform.getInputs().size() != 1) - throw Exception("Transform for drain should have single input, " - "but " + transform.getName() + " has " + - toString(transform.getInputs().size()) + " inputs.", ErrorCodes::LOGICAL_ERROR); - - if (transform.getOutputs().size() != 1) - throw Exception("Transform for drain should have single output, " - "but " + transform.getName() + " has " + - toString(transform.getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); - - if (transform.getInputs().front().isConnected()) - throw Exception("Transform for drain has connected input.", ErrorCodes::LOGICAL_ERROR); - - if (transform.getOutputs().front().isConnected()) - throw Exception("Transform for drain has connected input.", ErrorCodes::LOGICAL_ERROR); -} - -void checkInitialized(const Processors & processors) -{ - if (processors.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Drain is not initialized"); -} - -Drain::Drain(ProcessorPtr processor) -{ - checkSink(*processor); - processors.emplace_back(std::move(processor)); -} - -void Drain::addTransform(ProcessorPtr processor) -{ - checkInitialized(processors); - checkTransform(*processor); - connect(processor->getOutputs().front(), processors.back()->getInputs().front()); - processors.emplace_back(std::move(processor)); -} - -InputPort & Drain::getPort() const -{ - checkInitialized(processors); - return processors.back()->getInputs().front(); -} - -} diff --git a/src/Processors/Drain.h b/src/Processors/Drain.h deleted file mode 100644 index 866d8a0fc52..00000000000 --- a/src/Processors/Drain.h +++ /dev/null @@ -1,23 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class Drain -{ -public: - Drain() = default; - explicit Drain(ProcessorPtr processor); - - void addTransform(ProcessorPtr processor); - - InputPort & getPort() const; - const Block & getHeader() const { return getPort().getHeader(); } - -private: - Processors processors; -}; - -} diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index e0da79f148d..d2aecd10f3f 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -667,6 +667,41 @@ void Pipe::addSimpleTransform(const ProcessorGetter & getter) addSimpleTransform([&](const Block & stream_header, StreamType) { return getter(stream_header); }); } +void Pipe::addChains(std::vector chains) +{ + if (output_ports.size() != chains.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot add chains to Pipe because " + "number of output ports ({}) is not equal to the number of chains ({})", + output_ports.size(), chains.size()); + + dropTotals(); + dropExtremes(); + + Block new_header; + for (size_t i = 0; i < output_ports.size(); ++i) + { + if (i == 0) + new_header = chains[i].getOutputHeader(); + else + assertBlocksHaveEqualStructure(new_header, chains[i].getOutputHeader(), "QueryPipeline"); + + connect(*output_ports[i], chains[i].getInputPort()); + output_ports[i] = &chains[i].getOutputPort(); + + auto added_processors = Chain::getProcessors(std::move(chains[i])); + for (auto & transform : added_processors) + { + if (collected_processors) + collected_processors->emplace_back(transform); + + processors.emplace_back(std::move(transform)); + } + } + + header = std::move(new_header); +} + void Pipe::resize(size_t num_streams, bool force, bool strict) { if (output_ports.empty()) diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index dc3be3289fc..0b430631d71 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -86,6 +87,9 @@ public: void addSimpleTransform(const ProcessorGetter & getter); void addSimpleTransform(const ProcessorGetterWithStreamKind & getter); + /// Add chain to every output port. + void addChains(std::vector chains); + /// Changes the number of output ports if needed. Adds ResizeTransform. void resize(size_t num_streams, bool force = false, bool strict = false); diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 2b882ee93ab..c03aacf2e1b 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -103,6 +103,12 @@ void QueryPipeline::addTransform(ProcessorPtr transform, InputPort * totals, Inp pipe.addTransform(std::move(transform), totals, extremes); } +void QueryPipeline::addChains(std::vector chains) +{ + checkInitializedAndNotCompleted(); + pipe.addChains(std::move(chains)); +} + void QueryPipeline::transform(const Transformer & transformer) { checkInitializedAndNotCompleted(); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 358d31a6dff..d382bf21ebe 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -56,6 +56,8 @@ public: void addTransform(ProcessorPtr transform); void addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes); + void addChains(std::vector chains); + using Transformer = std::function; /// Transform pipeline in general way. void transform(const Transformer & transformer); diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index 8bf5fdf3fd2..7fb6f3d6248 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -6,6 +6,13 @@ namespace DB { +ExceptionKeepingTransformRuntimeData::ExceptionKeepingTransformRuntimeData( + std::unique_ptr thread_status_, + std::string additional_exception_message_) + : thread_status(std::move(thread_status_)) + , additional_exception_message(std::move(additional_exception_message_)) +{ +} ExceptionKeepingTransform::ExceptionKeepingTransform(const Block & in_header, const Block & out_header) : IProcessor({in_header}, {out_header}) @@ -68,7 +75,7 @@ IProcessor::Status ExceptionKeepingTransform::prepare() return Status::Ready; } -static std::exception_ptr runStep(std::function step, ExceptionKeepingTransform::RuntimeData * runtime_data) +static std::exception_ptr runStep(std::function step, ExceptionKeepingTransformRuntimeData * runtime_data) { auto * original_thread = current_thread; SCOPE_EXIT({ current_thread = original_thread; }); diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index d66391833ec..380696ed059 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -7,6 +7,20 @@ namespace DB class ThreadStatus; + +struct ExceptionKeepingTransformRuntimeData +{ + std::unique_ptr thread_status = nullptr; + UInt64 elapsed_ms = 0; + std::string additional_exception_message; + + ExceptionKeepingTransformRuntimeData( + std::unique_ptr thread_status_, + std::string additional_exception_message_); +}; + +using ExceptionKeepingTransformRuntimeDataPtr = std::shared_ptr; + /// Has one input and one output. /// Works similarly to ISimpleTransform, but with much care about exceptions. /// @@ -48,19 +62,10 @@ public: InputPort & getInputPort() { return input; } OutputPort & getOutputPort() { return output; } - struct RuntimeData - { - std::unique_ptr thread_status = nullptr; - UInt64 elapsed_ms = 0; - std::string additional_exception_message; - }; - - using RuntimeDataPtr = std::shared_ptr; - - void setRuntimeData(RuntimeDataPtr runtime_data_) { runtime_data = std::move(runtime_data_); } + void setRuntimeData(ExceptionKeepingTransformRuntimeDataPtr runtime_data_) { runtime_data = std::move(runtime_data_); } private: - RuntimeDataPtr runtime_data; + ExceptionKeepingTransformRuntimeDataPtr runtime_data; }; From 358babc31b4679c98cbe9b808d0a77139161e5ae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Sep 2021 14:29:20 +0300 Subject: [PATCH 007/142] Rewrite PushingToViewsBlockOutputStream part 5. --- src/DataStreams/BlockIO.h | 6 +- .../PushingToViewsBlockOutputStream.cpp | 40 ------ src/Interpreters/InterpreterInsertQuery.cpp | 6 +- src/Interpreters/executeQuery.cpp | 35 +++-- src/Processors/Chain.cpp | 12 ++ src/Processors/Chain.h | 3 + .../Executors/PushingPipelineExecutor.cpp | 121 ++++++++++++++++++ .../Executors/PushingPipelineExecutor.h | 58 +++++++++ src/Processors/Sinks/ExceptionHandlingSink.h | 51 ++++++++ src/Server/TCPHandler.cpp | 32 +++-- src/Server/TCPHandler.h | 1 + 11 files changed, 284 insertions(+), 81 deletions(-) create mode 100644 src/Processors/Executors/PushingPipelineExecutor.cpp create mode 100644 src/Processors/Executors/PushingPipelineExecutor.h create mode 100644 src/Processors/Sinks/ExceptionHandlingSink.h diff --git a/src/DataStreams/BlockIO.h b/src/DataStreams/BlockIO.h index 7c8edc3318a..aa4d481f68b 100644 --- a/src/DataStreams/BlockIO.h +++ b/src/DataStreams/BlockIO.h @@ -32,8 +32,8 @@ struct BlockIO QueryPipeline pipeline; /// Callbacks for query logging could be set here. - std::function finish_callback; - std::function exception_callback; + std::function finish_callback; + std::function exception_callback; /// When it is true, don't bother sending any non-empty blocks to the out stream bool null_format = false; @@ -47,7 +47,7 @@ struct BlockIO if (pipeline.initialized()) pipeline_ptr = &pipeline; - finish_callback(in.get(), out.get(), pipeline_ptr); + finish_callback(in.get(), pipeline_ptr); } } diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 4ad64206442..60bbf52269c 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -224,46 +224,6 @@ private: std::exception_ptr any_exception; }; -//class ExceptionHandlingSink : public IProcessor -//{ -//public: -// explicit ExceptionHandlingSink(Block header) -// : IProcessor({std::move(header)}, {}) -// , input(inputs.front()) -// { -// } -// -// Status prepare() override -// { -// while (!input.isFinished()) -// { -// input.setNeeded(); -// if (!input.hasData()) -// return Status::NeedData; -// -// auto data = input.pullData(); -// if (data.exception) -// exceptions.emplace_back(std::move(data.exception)); -// } -// -// if (!exceptions.empty()) -// return Status::Ready; -// -// return Status::Finished; -// } -// -// void work() override -// { -// auto exception = std::move(exceptions.at(0)); -// exceptions.clear(); -// std::rethrow_exception(std::move(exception)); -// } -// -//private: -// InputPort & input; -// std::vector exceptions; -//}; - Chain buildPushingToViewsDrain( const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot, diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index b150b959f87..5e9d29ed7d3 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include #include #include @@ -357,7 +357,7 @@ BlockIO InterpreterInsertQuery::execute() res.pipeline.setSinks([&](const Block & cur_header, QueryPipeline::StreamType) -> ProcessorPtr { - return std::make_shared(cur_header); + return std::make_shared(cur_header); }); if (!allow_materialized) @@ -375,7 +375,7 @@ BlockIO InterpreterInsertQuery::execute() res.pipeline.addChains(std::move(out_chains)); res.pipeline.setSinks([&](const Block & cur_header, Pipe::StreamType) { - return std::make_shared(cur_header); + return std::make_shared(cur_header); }); } else diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 3ebc2eb142c..6ff951277f6 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -54,6 +54,7 @@ #include #include #include +#include namespace ProfileEvents @@ -623,11 +624,11 @@ static std::tuple executeQueryImpl( } } - if (res.out) + if (!res.out.empty()) { - if (auto * stream = dynamic_cast(res.out.get())) + if (auto * counting = dynamic_cast(&res.out.getSource())) { - stream->setProcessListElement(context->getProcessListElement()); + counting->setProcessListElement(context->getProcessListElement()); } } } @@ -737,7 +738,7 @@ static std::tuple executeQueryImpl( log_queries_min_query_duration_ms = settings.log_queries_min_query_duration_ms.totalMilliseconds(), status_info_to_query_log ] - (IBlockInputStream * stream_in, IBlockOutputStream * stream_out, QueryPipeline * query_pipeline) mutable + (IBlockInputStream * stream_in, QueryPipeline * query_pipeline) mutable { QueryStatus * process_list_elem = context->getProcessListElement(); @@ -773,15 +774,6 @@ static std::tuple executeQueryImpl( elem.result_rows = stream_in_info.rows; elem.result_bytes = stream_in_info.bytes; } - else if (stream_out) /// will be used only for ordinary INSERT queries - { - if (const auto * counting_stream = dynamic_cast(stream_out)) - { - /// NOTE: Redundancy. The same values could be extracted from process_list_elem->progress_out.query_settings = process_list_elem->progress_in - elem.result_rows = counting_stream->getProgress().read_rows; - elem.result_bytes = counting_stream->getProgress().read_bytes; - } - } else if (query_pipeline) { if (const auto * output_format = query_pipeline->getOutputFormat()) @@ -790,6 +782,12 @@ static std::tuple executeQueryImpl( elem.result_bytes = output_format->getResultBytes(); } } + else /// will be used only for ordinary INSERT queries + { + auto progress_out = process_list_elem->getProgressOut(); + elem.result_rows = progress_out.read_rows; + elem.result_bytes = progress_out.read_bytes; + } if (elem.read_rows != 0) { @@ -1009,15 +1007,16 @@ void executeQuery( try { - if (streams.out) + if (!streams.out.empty()) { - auto pipe = getSourceFromFromASTInsertQuery(ast, &istr, streams.out->getHeader(), context, nullptr); + auto pipe = getSourceFromFromASTInsertQuery(ast, &istr, streams.out.getInputHeader(), context, nullptr); pipeline.init(std::move(pipe)); pipeline.resize(1); - pipeline.setSinks([&](const Block &, Pipe::StreamType) + pipeline.addChains({std::move(streams.out)}); + pipeline.setSinks([&](const Block & header, Pipe::StreamType) { - return std::make_shared(streams.out); + return std::make_shared(header); }); auto executor = pipeline.execute(); @@ -1154,7 +1153,7 @@ void executeTrivialBlockIO(BlockIO & streams, ContextPtr context) { try { - if (streams.out) + if (!streams.out.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Query stream requires input, but no input buffer provided, it's a bug"); if (streams.in) throw Exception(ErrorCodes::LOGICAL_ERROR, "Query stream requires output, but no output buffer provided, it's a bug"); diff --git a/src/Processors/Chain.cpp b/src/Processors/Chain.cpp index 6f0320c86ac..2c863a9fc23 100644 --- a/src/Processors/Chain.cpp +++ b/src/Processors/Chain.cpp @@ -93,6 +93,18 @@ void Chain::addSink(ProcessorPtr processor) processors.emplace_front(std::move(processor)); } +IProcessor & Chain::getSource() +{ + checkInitialized(processors); + return *processors.front(); +} + +IProcessor & Chain::getSink() +{ + checkInitialized(processors); + return *processors.back(); +} + InputPort & Chain::getInputPort() const { checkInitialized(processors); diff --git a/src/Processors/Chain.h b/src/Processors/Chain.h index 459fd9fbc7e..1ff468f430e 100644 --- a/src/Processors/Chain.h +++ b/src/Processors/Chain.h @@ -23,6 +23,9 @@ public: void addSource(ProcessorPtr processor); void addSink(ProcessorPtr processor); + IProcessor & getSource(); + IProcessor & getSink(); + InputPort & getInputPort() const; OutputPort & getOutputPort() const; diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp new file mode 100644 index 00000000000..478e26c602c --- /dev/null +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -0,0 +1,121 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +class PushingSource : public ISource +{ +public: + explicit PushingSource(const Block & header, std::atomic_bool & need_data_flag_) + : ISource(header) + , need_data_flag(need_data_flag_) + {} + + String getName() const override { return "PullingOutputFormat"; } + + void setData(Chunk chunk) { data = std::move(chunk); } + +protected: + + Status prepare() override + { + auto status = ISource::prepare(); + if (status == Status::Ready) + need_data_flag = true; + + return status; + } + + Chunk generate() override + { + return std::move(data); + } + +private: + Chunk data; + std::atomic_bool & need_data_flag; +}; + + +PushingPipelineExecutor::PushingPipelineExecutor(Chain & chain_) : chain(chain_) +{ + pushing_source = std::make_shared(chain.getInputHeader(), need_data_flag); + auto sink = std::make_shared(chain.getOutputHeader()); + connect(pushing_source->getPort(), chain.getInputPort()); + connect(chain.getOutputPort(), sink->getInputPort()); +} + +PushingPipelineExecutor::~PushingPipelineExecutor() +{ + try + { + cancel(); + } + catch (...) + { + tryLogCurrentException("PullingPipelineExecutor"); + } +} + +const Block & PushingPipelineExecutor::getHeader() const +{ + return pushing_source->getPort().getHeader(); +} + + +void PushingPipelineExecutor::start() +{ + if (started) + return; + + started = true; + executor = std::make_shared(*processors); + + if (!executor->executeStep(&need_data_flag)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Pipeline for PushingPipelineExecutor was finished before all data was inserted"); +} + +void PushingPipelineExecutor::push(Chunk chunk) +{ + if (!started) + start(); + + pushing_source->setData(std::move(chunk)); + + if (!executor->executeStep(&need_data_flag)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Pipeline for PushingPipelineExecutor was finished before all data was inserted"); +} + +void PushingPipelineExecutor::push(Block block) +{ + push(Chunk(block.getColumns(), block.rows())); +} + +void PushingPipelineExecutor::finish() +{ + if (finished) + return; + finished = true; + + if (executor) + executor->executeStep(); +} + +void PushingPipelineExecutor::cancel() +{ + /// Cancel execution if it wasn't finished. + if (executor && !finished) + { + finished = true; + executor->cancel(); + } +} + +} diff --git a/src/Processors/Executors/PushingPipelineExecutor.h b/src/Processors/Executors/PushingPipelineExecutor.h new file mode 100644 index 00000000000..f15a845a233 --- /dev/null +++ b/src/Processors/Executors/PushingPipelineExecutor.h @@ -0,0 +1,58 @@ +#pragma once +#include +#include + +namespace DB +{ + +class Block; +class Chunk; +class Chain; +class PushingSource; + +class PipelineExecutor; +using PipelineExecutorPtr = std::shared_ptr; + +class IProcessor; +using ProcessorPtr = std::shared_ptr; +using Processors = std::vector; + +/// Pushing executor for Chain of processors. Always executed in single thread. +/// Typical usage is: +/// +/// PushingPipelineExecutor executor(chain); +/// executor.start(); +/// while (auto chunk = ...) +/// executor.push(std::move(chunk)); +/// executor.finish(); +class PushingPipelineExecutor +{ +public: + explicit PushingPipelineExecutor(Chain & chain); + ~PushingPipelineExecutor(); + + /// Get structure of returned block or chunk. + const Block & getHeader() const; + + void start(); + + void push(Chunk chunk); + void push(Block block); + + void finish(); + + /// Stop execution. It is not necessary, but helps to stop execution before executor is destroyed. + void cancel(); + +private: + Chain & chain; + std::atomic_bool need_data_flag = false; + std::shared_ptr pushing_source; + + std::unique_ptr processors; + PipelineExecutorPtr executor; + bool started = false; + bool finished = false; +}; + +} diff --git a/src/Processors/Sinks/ExceptionHandlingSink.h b/src/Processors/Sinks/ExceptionHandlingSink.h new file mode 100644 index 00000000000..285ef1de789 --- /dev/null +++ b/src/Processors/Sinks/ExceptionHandlingSink.h @@ -0,0 +1,51 @@ +#pragma once +#include + +namespace DB +{ + +/// For now, this sink throws only last exception if any. +class ExceptionHandlingSink final : public IProcessor +{ +public: + explicit ExceptionHandlingSink(Block header) + : IProcessor({std::move(header)}, {}) + , input(inputs.front()) + { + } + + String getName() const override { return "ExceptionHandlingSink"; } + + Status prepare() override + { + while (!input.isFinished()) + { + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + + auto data = input.pullData(); + if (data.exception) + last_exception = std::move(data.exception); + } + + if (last_exception) + return Status::Ready; + + return Status::Finished; + } + + void work() override + { + if (last_exception) + std::rethrow_exception(std::move(last_exception)); + } + + InputPort & getInputPort() { return input; } + +private: + InputPort & input; + std::exception_ptr last_exception; +}; + +} diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 0339b144f09..b557476f34c 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -35,6 +35,7 @@ #include #include +#include #include "Core/Protocol.h" #include "TCPHandler.h" @@ -339,7 +340,7 @@ void TCPHandler::runImpl() after_check_cancelled.restart(); after_send_progress.restart(); - if (state.io.out) + if (!state.io.out.empty()) { state.need_receive_data_for_insert = true; processInsertQuery(connection_settings); @@ -568,10 +569,11 @@ void TCPHandler::readData(const Settings & connection_settings) void TCPHandler::processInsertQuery(const Settings & connection_settings) { + PushingPipelineExecutor executor(state.io.out); /** Made above the rest of the lines, so that in case of `writePrefix` function throws an exception, * client receive exception before sending data. */ - state.io.out->writePrefix(); + executor.start(); /// Send ColumnsDescription for insertion table if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA) @@ -588,19 +590,15 @@ void TCPHandler::processInsertQuery(const Settings & connection_settings) } /// Send block to the client - table structure. - sendData(state.io.out->getHeader()); + sendData(executor.getHeader()); - try - { - readData(connection_settings); - } - catch (...) - { - /// To avoid flushing from the destructor, that may lead to uncaught exception. - state.io.out->writeSuffix(); - throw; - } - state.io.out->writeSuffix(); + auto [poll_interval, receive_timeout] = getReadTimeouts(connection_settings); + sendLogs(); + + while (readDataNext(poll_interval, receive_timeout)) + executor.push(std::move(state.block_for_insert)); + + executor.finish(); } @@ -1364,7 +1362,7 @@ bool TCPHandler::receiveData(bool scalar) else { /// INSERT query. - state.io.out->write(block); + state.block_for_insert = block; } return true; } @@ -1406,8 +1404,8 @@ void TCPHandler::initBlockInput() state.maybe_compressed_in = in; Block header; - if (state.io.out) - header = state.io.out->getHeader(); + if (!state.io.out.empty()) + header = state.io.out.getInputHeader(); else if (state.need_receive_data_for_input) header = state.input_header; diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 086c1f7d5e5..db6de1bc30d 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -50,6 +50,7 @@ struct QueryState /// Where to write result data. std::shared_ptr maybe_compressed_out; BlockOutputStreamPtr block_out; + Block block_for_insert; /// Query text. String query; From 66a76ab70f32d3ef1ee99e8f2d74ced0a1b2f402 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 3 Sep 2021 20:29:36 +0300 Subject: [PATCH 008/142] Rewrite PushingToViewsBlockOutputStream part 6 --- programs/copier/ClusterCopier.cpp | 41 +++++-- src/Core/ExternalTable.cpp | 6 +- .../AddingDefaultBlockOutputStream.cpp | 30 ----- .../AddingDefaultBlockOutputStream.h | 45 ------- src/DataStreams/BlockIO.cpp | 4 +- .../PushingToSinkBlockOutputStream.h | 114 ------------------ .../PushingToViewsBlockOutputStream.cpp | 50 ++++++-- .../PushingToViewsBlockOutputStream.h | 13 +- src/DataStreams/RemoteBlockOutputStream.cpp | 24 ++-- src/DataStreams/RemoteBlockOutputStream.h | 46 +++++-- .../MySQL/MaterializedMySQLSyncThread.cpp | 46 ++++--- src/Interpreters/GlobalSubqueriesVisitor.h | 7 +- src/Interpreters/InterpreterExplainQuery.cpp | 60 +++++---- src/Interpreters/InterpreterInsertQuery.cpp | 2 - src/Interpreters/ProcessList.cpp | 9 +- src/Interpreters/ProcessList.h | 3 +- src/Interpreters/SystemLog.h | 9 +- src/Interpreters/executeQuery.cpp | 2 +- src/Parsers/ParserExplainQuery.cpp | 5 +- src/Processors/Chain.h | 1 + .../Executors/PushingPipelineExecutor.cpp | 11 +- src/Processors/QueryPipeline.cpp | 9 ++ src/Processors/QueryPipeline.h | 1 + src/Processors/Sinks/ExceptionHandlingSink.h | 2 +- src/Processors/Sinks/SinkToStorage.cpp | 2 + .../Sources/SourceFromSingleChunk.h | 1 + .../Transforms/CreatingSetsTransform.cpp | 24 ++-- .../Transforms/CreatingSetsTransform.h | 8 +- src/Server/GRPCServer.cpp | 39 +++--- src/Server/TCPHandler.cpp | 11 +- src/Storages/Distributed/DirectoryMonitor.cpp | 21 ++-- src/Storages/Distributed/DistributedSink.cpp | 42 ++++--- src/Storages/Distributed/DistributedSink.h | 5 +- src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/Kafka/KafkaBlockOutputStream.cpp | 6 +- src/Storages/Kafka/StorageKafka.cpp | 18 ++- src/Storages/LiveView/LiveViewSink.h | 2 +- src/Storages/MergeTree/MergeTreeSink.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- .../MaterializedPostgreSQLConsumer.cpp | 19 ++- .../PostgreSQLReplicationHandler.cpp | 17 +-- src/Storages/RabbitMQ/RabbitMQSink.cpp | 5 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 13 +- src/Storages/RocksDB/EmbeddedRocksDBSink.cpp | 4 +- src/Storages/StorageBuffer.cpp | 12 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageLog.cpp | 4 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMySQL.cpp | 2 +- src/Storages/StoragePostgreSQL.cpp | 2 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageSQLite.cpp | 2 +- src/Storages/StorageSet.cpp | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- src/Storages/StorageTinyLog.cpp | 4 +- src/Storages/StorageURL.cpp | 2 +- src/Storages/StorageValues.cpp | 2 + 57 files changed, 413 insertions(+), 410 deletions(-) delete mode 100644 src/DataStreams/AddingDefaultBlockOutputStream.cpp delete mode 100644 src/DataStreams/AddingDefaultBlockOutputStream.h delete mode 100644 src/DataStreams/PushingToSinkBlockOutputStream.h diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index cf0b6cc76a4..d8d6cf6e867 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -9,6 +9,10 @@ #include #include #include +#include +#include +#include +#include #include namespace DB @@ -1523,25 +1527,27 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( context_insert->setSettings(task_cluster->settings_push); /// Custom INSERT SELECT implementation - BlockInputStreamPtr input; - BlockOutputStreamPtr output; + QueryPipeline input; + Chain output; { BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); - auto pure_input = io_select.getInputStream(); - output = io_insert.out; + output = std::move(io_insert.out); /// Add converting actions to make it possible to copy blocks with slightly different schema - const auto & select_block = pure_input->getHeader(); - const auto & insert_block = output->getHeader(); + const auto & select_block = io_select.pipeline.getHeader(); + const auto & insert_block = output.getInputHeader(); auto actions_dag = ActionsDAG::makeConvertingActions( select_block.getColumnsWithTypeAndName(), insert_block.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Position); auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext())); - input = std::make_shared(pure_input, actions); + input.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, actions); + }); } /// Fail-fast optimization to abort copying when the current clean state expires @@ -1587,7 +1593,26 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( }; /// Main work is here - copyData(*input, *output, cancel_check, update_stats); + PullingPipelineExecutor pulling_executor(input); + PushingPipelineExecutor pushing_executor(output); + + Block data; + bool is_cancelled = false; + while (pulling_executor.pull(data)) + { + if (cancel_check()) + { + is_cancelled = true; + pushing_executor.cancel(); + pushing_executor.cancel(); + break; + } + pushing_executor.push(data); + update_stats(data); + } + + if (!is_cancelled) + pushing_executor.finish(); // Just in case if (future_is_dirty_checker.valid()) diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 9b53cd79a84..240179d6e7a 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -161,14 +162,17 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, auto storage = temporary_table.getTable(); getContext()->addExternalTable(data->table_name, std::move(temporary_table)); auto sink = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), getContext()); + auto exception_handling = std::make_shared(sink->getOutputPort().getHeader()); /// Write data data->pipe->resize(1); - connect(*data->pipe->getOutputPort(0), sink->getPort()); + connect(*data->pipe->getOutputPort(0), sink->getInputPort()); + connect(sink->getOutputPort(), exception_handling->getPort()); auto processors = Pipe::detachProcessors(std::move(*data->pipe)); processors.push_back(std::move(sink)); + processors.push_back(std::move(exception_handling)); auto executor = std::make_shared(processors); executor->execute(/*num_threads = */ 1); diff --git a/src/DataStreams/AddingDefaultBlockOutputStream.cpp b/src/DataStreams/AddingDefaultBlockOutputStream.cpp deleted file mode 100644 index 118926cfdd7..00000000000 --- a/src/DataStreams/AddingDefaultBlockOutputStream.cpp +++ /dev/null @@ -1,30 +0,0 @@ -#include -#include -#include - - -namespace DB -{ - -AddingDefaultBlockOutputStream::AddingDefaultBlockOutputStream( - const Block & in_header, - const Block & out_header, - const ColumnsDescription & columns_, - ContextPtr context_, - bool null_as_default_) - : output(output_), header(header_) -{ - auto dag = addMissingDefaults(in_header, output_header.getNamesAndTypesList(), columns_, context_, null_as_default_); - adding_defaults_actions = std::make_shared(std::move(dag), ExpressionActionsSettings::fromContext(context_, CompileExpressions::yes)); -} - -void AddingDefaultBlockOutputStream::write(const Block & block) -{ - auto copy = block; - adding_defaults_actions->execute(copy); - output->write(copy); -} - - - -} diff --git a/src/DataStreams/AddingDefaultBlockOutputStream.h b/src/DataStreams/AddingDefaultBlockOutputStream.h deleted file mode 100644 index 237bd994613..00000000000 --- a/src/DataStreams/AddingDefaultBlockOutputStream.h +++ /dev/null @@ -1,45 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ - -class ExpressionActions; -using ExpressionActionsPtr = std::shared_ptr; - -class Context; - -/** This stream adds three types of columns into block - * 1. Columns, that are missed inside request, but present in table without defaults (missed columns) - * 2. Columns, that are missed inside request, but present in table with defaults (columns with default values) - * 3. Columns that materialized from other columns (materialized columns) - * Also the stream can substitute NULL into DEFAULT value in case of INSERT SELECT query (null_as_default) if according setting is 1. - * All three types of columns are materialized (not constants). - */ -class AddingMissingDefaultsTransform : public ISimpleTransform -{ -public: - AddingMissingDefaultsTransform( - const Block & in_header, - const Block & out_header, - const ColumnsDescription & columns_, - ContextPtr context_, - bool null_as_default_ = false); - - void transform(Chunk & chunk) override; - - void flush() override; - - void writePrefix() override; - void writeSuffix() override; - -private: - ExpressionActionsPtr adding_defaults_actions; -}; - - -} diff --git a/src/DataStreams/BlockIO.cpp b/src/DataStreams/BlockIO.cpp index 3bbef4471db..79e1bd209a2 100644 --- a/src/DataStreams/BlockIO.cpp +++ b/src/DataStreams/BlockIO.cpp @@ -12,7 +12,7 @@ namespace ErrorCodes BlockInputStreamPtr BlockIO::getInputStream() { - if (out) + if (!out.empty()) throw Exception("Cannot get input stream from BlockIO because output stream is not empty", ErrorCodes::LOGICAL_ERROR); @@ -38,7 +38,7 @@ void BlockIO::reset() */ /// TODO simplify it all - out.reset(); + out = {}; in.reset(); if (process_list_entry) process_list_entry->get().releaseQueryStreams(); diff --git a/src/DataStreams/PushingToSinkBlockOutputStream.h b/src/DataStreams/PushingToSinkBlockOutputStream.h deleted file mode 100644 index eeca8506d8e..00000000000 --- a/src/DataStreams/PushingToSinkBlockOutputStream.h +++ /dev/null @@ -1,114 +0,0 @@ -#pragma once -#include -#include -#include -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -class PushingToSinkBlockOutputStream : public IBlockOutputStream -{ -public: - explicit PushingToSinkBlockOutputStream(SinkToStoragePtr sink_) - : sink(std::move(sink_)), port(sink->getPort().getHeader(), sink.get()) {} - - Block getHeader() const override { return sink->getPort().getHeader(); } - - void write(const Block & block) override - { - /// In case writePrefix was not called. - if (!port.isConnected()) - writePrefix(); - - if (!block) - return; - - size_t num_rows = block.rows(); - Chunk chunk(block.getColumns(), num_rows); - port.push(std::move(chunk)); - - while (true) - { - auto status = sink->prepare(); - switch (status) - { - case IProcessor::Status::Ready: - sink->work(); - continue; - case IProcessor::Status::NeedData: - return; - case IProcessor::Status::Async: [[fallthrough]]; - case IProcessor::Status::ExpandPipeline: [[fallthrough]]; - case IProcessor::Status::Finished: [[fallthrough]]; - case IProcessor::Status::PortFull: - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Status {} in not expected in PushingToSinkBlockOutputStream::writePrefix", - IProcessor::statusToName(status)); - } - } - } - - void writePrefix() override - { - connect(port, sink->getPort()); - - while (true) - { - auto status = sink->prepare(); - switch (status) - { - case IProcessor::Status::Ready: - sink->work(); - continue; - case IProcessor::Status::NeedData: - return; - case IProcessor::Status::Async: [[fallthrough]]; - case IProcessor::Status::ExpandPipeline: [[fallthrough]]; - case IProcessor::Status::Finished: [[fallthrough]]; - case IProcessor::Status::PortFull: - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Status {} in not expected in PushingToSinkBlockOutputStream::writePrefix", - IProcessor::statusToName(status)); - } - } - } - - void writeSuffix() override - { - port.finish(); - while (true) - { - auto status = sink->prepare(); - switch (status) - { - case IProcessor::Status::Ready: - sink->work(); - continue; - case IProcessor::Status::Finished: - - ///flush(); - return; - case IProcessor::Status::NeedData: - case IProcessor::Status::Async: - case IProcessor::Status::ExpandPipeline: - case IProcessor::Status::PortFull: - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Status {} in not expected in PushingToSinkBlockOutputStream::writeSuffix", - IProcessor::statusToName(status)); - } - } - } - -private: - SinkToStoragePtr sink; - OutputPort port; -}; - -} diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 60bbf52269c..fa91ba552fa 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -17,6 +17,7 @@ #include #include #include +#include "Processors/printPipeline.h" #include #include @@ -28,14 +29,20 @@ struct ViewsData { std::list views; ContextPtr context; + StorageID source_storage_id; + StorageMetadataPtr source_metadata_snapshot; + StoragePtr source_storage; /// In case of exception happened while inserting into main table, it is pushed to pipeline. /// Remember the first one, we should keep them after view processing. std::atomic_bool has_exception = false; std::exception_ptr first_exception; - ViewsData(std::list views_, ContextPtr context_) - : views(std::move(views_)), context(std::move(context_)) + ViewsData(ContextPtr context_, StorageID source_storage_id_, StorageMetadataPtr source_metadata_snapshot_ , StoragePtr source_storage_) + : context(std::move(context_)) + , source_storage_id(std::move(source_storage_id_)) + , source_metadata_snapshot(std::move(source_metadata_snapshot_)) + , source_storage(std::move(source_storage_)) { } }; @@ -100,6 +107,8 @@ public: for (auto & output : outputs) output.push(data.chunk.clone()); } + + return Status::PortFull; } InputPort & getInputPort() { return input; } @@ -276,8 +285,8 @@ Chain buildPushingToViewsDrain( insert_context->setSetting("min_insert_block_size_bytes", insert_settings.min_insert_block_size_bytes_for_materialized_views.value); } - std::list views; std::vector chains; + auto views_data = std::make_shared(context, table_id, metadata_snapshot, storage); for (const auto & database_table : dependencies) { @@ -371,7 +380,7 @@ Chain buildPushingToViewsDrain( std::chrono::system_clock::now(), QueryViewsLogElement::ViewStatus::EXCEPTION_BEFORE_START}; - views.emplace_back(ViewRuntimeData{ + views_data->views.emplace_back(ViewRuntimeData{ std::move(query), out.getInputHeader(), database_table, @@ -381,7 +390,8 @@ Chain buildPushingToViewsDrain( nullptr, std::move(runtime_stats)}); - auto executing_inner_query = std::make_shared(storage_header, views.back()); + auto executing_inner_query = std::make_shared( + storage_header, views_data->views.back(), views_data->source_storage_id, views_data->source_metadata_snapshot, views_data->source_storage); executing_inner_query->setRuntimeData(view_runtime_data); out.addSource(std::move(executing_inner_query)); @@ -397,7 +407,7 @@ Chain buildPushingToViewsDrain( Chain result_chain; - size_t num_views = views.size(); + size_t num_views = views_data->views.size(); if (num_views != 0) { std::vector headers; @@ -405,7 +415,6 @@ Chain buildPushingToViewsDrain( for (const auto & chain : chains) headers.push_back(chain.getOutputHeader()); - auto views_data = std::make_shared(std::move(views), context); auto copying_data = std::make_shared(storage_header, views_data); auto finalizing_views = std::make_shared(std::move(headers), views_data); auto out = copying_data->getOutputs().begin(); @@ -417,6 +426,8 @@ Chain buildPushingToViewsDrain( { connect(*out, chain.getInputPort()); connect(chain.getOutputPort(), *in); + ++in; + ++out; processors.splice(processors.end(), Chain::getProcessors(std::move(chain))); } @@ -434,10 +445,14 @@ Chain buildPushingToViewsDrain( result_chain.addSource(std::move(sink)); } + /// TODO: add pushing to live view + if (result_chain.empty()) + result_chain.addSink(std::make_shared(storage_header)); + return result_chain; } -static void process(Block & block, ViewRuntimeData & view) +static void process(Block & block, ViewRuntimeData & view, const StorageID & source_storage_id, const StorageMetadataPtr & source_metadata_snapshot, const StoragePtr & source_storage) { const auto & context = view.context; @@ -446,10 +461,10 @@ static void process(Block & block, ViewRuntimeData & view) /// InterpreterSelectQuery will do processing of alias columns. auto local_context = Context::createCopy(context); local_context->addViewSource(StorageValues::create( - view.table_id, - view.metadata_snapshot->getColumns(), + source_storage_id, + source_metadata_snapshot->getColumns(), block, - view.storage->getVirtuals())); + source_storage->getVirtuals())); /// We need keep InterpreterSelectQuery, until the processing will be finished, since: /// @@ -491,9 +506,18 @@ static void process(Block & block, ViewRuntimeData & view) callback(progress); }); + PullingPipelineExecutor executor(io.pipeline); if (!executor.pull(block)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No nothing is returned from view inner query {}", view.query); + { + block.clear(); + return; + } + + WriteBufferFromOwnString buf; + auto pipe = QueryPipeline::getPipe(std::move(io.pipeline)); + const auto & processors = pipe.getProcessors(); + printPipeline(processors, buf); if (executor.pull(block)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Single chunk is expected from view inner query {}", view.query); @@ -502,7 +526,7 @@ static void process(Block & block, ViewRuntimeData & view) void ExecutingInnerQueryFromViewTransform::transform(Chunk & chunk) { auto block = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); - process(block, view); + process(block, view, source_storage_id, source_metadata_snapshot, source_storage); chunk.setColumns(block.getColumns(), block.rows()); } diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index dfa7a9593d7..94c87b63c5a 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -58,9 +58,17 @@ Chain buildPushingToViewsDrain( class ExecutingInnerQueryFromViewTransform final : public ExceptionKeepingTransform { public: - ExecutingInnerQueryFromViewTransform(const Block & header, ViewRuntimeData & view_data) + ExecutingInnerQueryFromViewTransform( + const Block & header, + ViewRuntimeData & view_data, + const StorageID & source_storage_id_, + const StorageMetadataPtr & source_metadata_snapshot_, + const StoragePtr & source_storage_) : ExceptionKeepingTransform(header, view_data.sample_block) , view(view_data) + , source_storage_id(source_storage_id_) + , source_metadata_snapshot(source_metadata_snapshot_) + , source_storage(source_storage_) { } @@ -71,6 +79,9 @@ protected: private: ViewRuntimeData & view; + const StorageID & source_storage_id; + const StorageMetadataPtr & source_metadata_snapshot; + const StoragePtr & source_storage; }; } diff --git a/src/DataStreams/RemoteBlockOutputStream.cpp b/src/DataStreams/RemoteBlockOutputStream.cpp index 976c4671652..106add68c5c 100644 --- a/src/DataStreams/RemoteBlockOutputStream.cpp +++ b/src/DataStreams/RemoteBlockOutputStream.cpp @@ -18,11 +18,12 @@ namespace ErrorCodes } -RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, - const ConnectionTimeouts & timeouts, - const String & query_, - const Settings & settings_, - const ClientInfo & client_info_) +RemoteInserter::RemoteInserter( + Connection & connection_, + const ConnectionTimeouts & timeouts, + const String & query_, + const Settings & settings_, + const ClientInfo & client_info_) : connection(connection_), query(query_) { ClientInfo modified_client_info = client_info_; @@ -70,11 +71,8 @@ RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, } -void RemoteBlockOutputStream::write(const Block & block) +void RemoteInserter::write(Block block) { - if (header) - assertBlocksHaveEqualStructure(block, header, "RemoteBlockOutputStream"); - try { connection.sendData(block); @@ -94,14 +92,14 @@ void RemoteBlockOutputStream::write(const Block & block) } -void RemoteBlockOutputStream::writePrepared(ReadBuffer & input, size_t size) +void RemoteInserter::writePrepared(ReadBuffer & buf, size_t size) { /// We cannot use 'header'. Input must contain block with proper structure. - connection.sendPreparedData(input, size); + connection.sendPreparedData(buf, size); } -void RemoteBlockOutputStream::writeSuffix() +void RemoteInserter::onFinish() { /// Empty block means end of data. connection.sendData(Block()); @@ -127,7 +125,7 @@ void RemoteBlockOutputStream::writeSuffix() finished = true; } -RemoteBlockOutputStream::~RemoteBlockOutputStream() +RemoteInserter::~RemoteInserter() { /// If interrupted in the middle of the loop of communication with the server, then interrupt the connection, /// to not leave the connection in unsynchronized state. diff --git a/src/DataStreams/RemoteBlockOutputStream.h b/src/DataStreams/RemoteBlockOutputStream.h index 2c89a7358ad..f1f49015c9d 100644 --- a/src/DataStreams/RemoteBlockOutputStream.h +++ b/src/DataStreams/RemoteBlockOutputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include @@ -17,24 +17,25 @@ struct Settings; /** Allow to execute INSERT query on remote server and send data for it. */ -class RemoteBlockOutputStream : public IBlockOutputStream +class RemoteInserter { public: - RemoteBlockOutputStream(Connection & connection_, - const ConnectionTimeouts & timeouts, - const String & query_, - const Settings & settings_, - const ClientInfo & client_info_); + RemoteInserter( + Connection & connection_, + const ConnectionTimeouts & timeouts, + const String & query_, + const Settings & settings_, + const ClientInfo & client_info_); - Block getHeader() const override { return header; } - - void write(const Block & block) override; - void writeSuffix() override; + void write(Block block); + void onFinish(); /// Send pre-serialized and possibly pre-compressed block of data, that will be read from 'input'. - void writePrepared(ReadBuffer & input, size_t size = 0); + void writePrepared(ReadBuffer & buf, size_t size = 0); - ~RemoteBlockOutputStream() override; + ~RemoteInserter(); + + const Block & getHeader() const { return header; } private: Connection & connection; @@ -43,4 +44,23 @@ private: bool finished = false; }; +class RemoteSink final : public RemoteInserter, public SinkToStorage +{ +public: + explicit RemoteSink( + Connection & connection_, + const ConnectionTimeouts & timeouts, + const String & query_, + const Settings & settings_, + const ClientInfo & client_info_) + : RemoteInserter(connection_, timeouts, query_, settings_, client_info_) + , SinkToStorage(RemoteInserter::getHeader()) + { + } + + String getName() const override { return "RemoteSink"; } + void consume (Chunk chunk) override { write(RemoteInserter::getHeader().cloneWithColumns(chunk.detachColumns())); } + void onFinish() override { RemoteInserter::onFinish(); } +}; + } diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 53495aa3cb1..baa9ee14cea 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -11,6 +11,8 @@ # include # include # include +# include +# include # include # include # include @@ -287,7 +289,7 @@ static inline void cleanOutdatedTables(const String & database_name, ContextPtr } } -static inline BlockOutputStreamPtr +static inline Chain getTableOutput(const String & database_name, const String & table_name, ContextMutablePtr query_context, bool insert_materialized = false) { const StoragePtr & storage = DatabaseCatalog::instance().getTable(StorageID(database_name, table_name), query_context); @@ -311,10 +313,10 @@ getTableOutput(const String & database_name, const String & table_name, ContextM BlockIO res = tryToExecuteQuery("INSERT INTO " + backQuoteIfNeed(table_name) + "(" + insert_columns_str.str() + ")" + " VALUES", query_context, database_name, comment); - if (!res.out) + if (res.out.empty()) throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR); - return res.out; + return std::move(res.out); } static inline void dumpDataForTables( @@ -332,25 +334,28 @@ static inline void dumpDataForTables( String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data"; tryToExecuteQuery(query_prefix + " " + iterator->second, query_context, database_name, comment); /// create table. - auto out = std::make_shared(getTableOutput(database_name, table_name, query_context)); + auto chain = getTableOutput(database_name, table_name, query_context); + auto counting = std::make_shared(chain.getInputHeader()); + chain.addSource(counting); StreamSettings mysql_input_stream_settings(context->getSettingsRef()); auto input = std::make_unique( connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), - out->getHeader(), mysql_input_stream_settings); + chain.getInputHeader(), mysql_input_stream_settings); QueryPipeline pipeline; pipeline.init(Pipe(std::move(input))); - PullingPipelineExecutor executor(pipeline); + pipeline.addChain(std::move(chain)); + pipeline.setSinks([&](const Block & header, Pipe::StreamType) + { + return std::make_shared(header); + }); + + auto executor = pipeline.execute(); Stopwatch watch; + executor->execute(1); - out->writePrefix(); - Block block; - while (executor.pull(block)) - out->write(block); - out->writeSuffix(); - - const Progress & progress = out->getProgress(); + const Progress & progress = counting->getProgress(); LOG_INFO(&Poco::Logger::get("MaterializedMySQLSyncThread(" + database_name + ")"), "Materialize MySQL step 1: dump {}, {} rows, {} in {} sec., {} rows/sec., {}/sec." , table_name, formatReadableQuantity(progress.written_rows), formatReadableSizeWithBinarySuffix(progress.written_bytes) @@ -800,9 +805,18 @@ void MaterializedMySQLSyncThread::Buffers::commit(ContextPtr context) for (auto & table_name_and_buffer : data) { auto query_context = createQueryContext(context); - OneBlockInputStream input(table_name_and_buffer.second->first); - BlockOutputStreamPtr out = getTableOutput(database, table_name_and_buffer.first, query_context, true); - copyData(input, *out); + auto input = std::make_shared(table_name_and_buffer.second->first); + auto out = getTableOutput(database, table_name_and_buffer.first, query_context, true); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); + pipeline.addChain(std::move(out)); + pipeline.setSinks([&](const Block & header, Pipe::StreamType) + { + return std::make_shared(header); + }); + + auto executor = pipeline.execute(); + executor->execute(1); } data.clear(); diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 6a87527dc9c..b5edaa0d53f 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -17,6 +17,7 @@ #include #include #include +#include #include namespace DB @@ -158,10 +159,10 @@ public: auto external_table = external_storage_holder->getTable(); auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext()); auto io = interpreter->execute(); - io.pipeline.resize(1); - io.pipeline.setSinks([&](const Block &, Pipe::StreamType) -> ProcessorPtr + io.pipeline.addChain(Chain(std::move(table_out))); + io.pipeline.setSinks([&](const Block & header, Pipe::StreamType) -> ProcessorPtr { - return table_out; + return std::make_shared(header); }); auto executor = io.pipeline.execute(); executor->execute(io.pipeline.getNumStreams()); diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 37650f5caa7..0373c165c5b 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -303,33 +303,51 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() } else if (ast.getKind() == ASTExplainQuery::QueryPipeline) { - if (!dynamic_cast(ast.getExplainedQuery().get())) - throw Exception("Only SELECT is supported for EXPLAIN query", ErrorCodes::INCORRECT_QUERY); - - auto settings = checkAndGetSettings(ast.getSettings()); - QueryPlan plan; - - InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); - interpreter.buildQueryPlan(plan); - auto pipeline = plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(getContext()), - BuildQueryPipelineSettings::fromContext(getContext())); - - if (settings.graph) + if (dynamic_cast(ast.getExplainedQuery().get())) { - /// Pipe holds QueryPlan, should not go out-of-scope - auto pipe = QueryPipeline::getPipe(std::move(*pipeline)); - const auto & processors = pipe.getProcessors(); + auto settings = checkAndGetSettings(ast.getSettings()); + QueryPlan plan; - if (settings.compact) - printPipelineCompact(processors, buf, settings.query_pipeline_options.header); + InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); + interpreter.buildQueryPlan(plan); + auto pipeline = plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(getContext()), + BuildQueryPipelineSettings::fromContext(getContext())); + + if (settings.graph) + { + /// Pipe holds QueryPlan, should not go out-of-scope + auto pipe = QueryPipeline::getPipe(std::move(*pipeline)); + const auto & processors = pipe.getProcessors(); + + if (settings.compact) + printPipelineCompact(processors, buf, settings.query_pipeline_options.header); + else + printPipeline(processors, buf); + } else + { + plan.explainPipeline(buf, settings.query_pipeline_options); + } + } + else if (dynamic_cast(ast.getExplainedQuery().get())) + { + InterpreterInsertQuery insert(ast.getExplainedQuery(), getContext()); + auto io = insert.execute(); + if (io.pipeline.initialized()) + { + auto pipe = QueryPipeline::getPipe(std::move(io.pipeline)); + const auto & processors = pipe.getProcessors(); printPipeline(processors, buf); + } + else + { + const auto & processors = io.out.getProcessors(); + printPipeline(processors, buf); + } } else - { - plan.explainPipeline(buf, settings.query_pipeline_options); - } + throw Exception("Only SELECT and INSERT is supported for EXPLAIN PIPELINE query", ErrorCodes::INCORRECT_QUERY); } else if (ast.getKind() == ASTExplainQuery::QueryEstimates) { diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 5e9d29ed7d3..c873a0cde44 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -1,14 +1,12 @@ #include #include -#include #include #include #include #include #include #include -#include #include #include #include diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 06320f00dfa..42e8234adc1 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -308,21 +308,18 @@ void QueryStatus::setQueryStreams(const BlockIO & io) std::lock_guard lock(query_streams_mutex); query_stream_in = io.in; - query_stream_out = io.out; query_streams_status = QueryStreamsStatus::Initialized; } void QueryStatus::releaseQueryStreams() { BlockInputStreamPtr in; - BlockOutputStreamPtr out; { std::lock_guard lock(query_streams_mutex); query_streams_status = QueryStreamsStatus::Released; in = std::move(query_stream_in); - out = std::move(query_stream_out); } /// Destroy streams outside the mutex lock @@ -335,7 +332,7 @@ bool QueryStatus::streamsAreReleased() return query_streams_status == QueryStreamsStatus::Released; } -bool QueryStatus::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStreamPtr & out) const +bool QueryStatus::tryGetQueryStreams(BlockInputStreamPtr & in) const { std::lock_guard lock(query_streams_mutex); @@ -343,7 +340,6 @@ bool QueryStatus::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStream return false; in = query_stream_in; - out = query_stream_out; return true; } @@ -354,14 +350,13 @@ CancellationCode QueryStatus::cancelQuery(bool kill) return CancellationCode::CancelSent; BlockInputStreamPtr input_stream; - BlockOutputStreamPtr output_stream; SCOPE_EXIT({ std::lock_guard lock(query_streams_mutex); for (auto * e : executors) e->cancel(); }); - if (tryGetQueryStreams(input_stream, output_stream)) + if (tryGetQueryStreams(input_stream)) { if (input_stream) { diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 1adad84c040..e37309b6a6c 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -109,7 +109,6 @@ protected: /// This declaration is compatible with notes about BlockIO::process_list_entry: /// there are no cyclic dependencies: BlockIO::in,out point to objects inside ProcessListElement (not whole object) BlockInputStreamPtr query_stream_in; - BlockOutputStreamPtr query_stream_out; /// Array of PipelineExecutors to be cancelled when a cancelQuery is received std::vector executors; @@ -183,7 +182,7 @@ public: bool streamsAreReleased(); /// Get query in/out pointers from BlockIO - bool tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStreamPtr & out) const; + bool tryGetQueryStreams(BlockInputStreamPtr & in) const; CancellationCode cancelQuery(bool kill); diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index a332245439b..c9f3b90877c 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -485,9 +486,11 @@ void SystemLog::flushImpl(const std::vector & to_flush, InterpreterInsertQuery interpreter(query_ptr, insert_context); BlockIO io = interpreter.execute(); - io.out->writePrefix(); - io.out->write(block); - io.out->writeSuffix(); + PushingPipelineExecutor executor(io.out); + + executor.start(); + executor.push(block); + executor.finish(); } catch (...) { diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 6ff951277f6..7fed15b16b0 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1013,7 +1013,7 @@ void executeQuery( pipeline.init(std::move(pipe)); pipeline.resize(1); - pipeline.addChains({std::move(streams.out)}); + pipeline.addChain(std::move(streams.out)); pipeline.setSinks([&](const Block & header, Pipe::StreamType) { return std::make_shared(header); diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp index b4ba0523239..ffaab0f2b6d 100644 --- a/src/Parsers/ParserExplainQuery.cpp +++ b/src/Parsers/ParserExplainQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -54,6 +55,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserCreateTableQuery create_p; ParserSelectWithUnionQuery select_p; + ParserInsertQuery insert_p(end); ASTPtr query; if (kind == ASTExplainQuery::ExplainKind::ParsedAST) { @@ -64,7 +66,8 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; } else if (select_p.parse(pos, query, expected) || - create_p.parse(pos, query, expected)) + create_p.parse(pos, query, expected) || + insert_p.parse(pos, query, expected)) explain_query->setExplainedQuery(std::move(query)); else return false; diff --git a/src/Processors/Chain.h b/src/Processors/Chain.h index 1ff468f430e..fd3c72221b9 100644 --- a/src/Processors/Chain.h +++ b/src/Processors/Chain.h @@ -32,6 +32,7 @@ public: const Block & getInputHeader() const { return getInputPort().getHeader(); } const Block & getOutputHeader() const { return getOutputPort().getHeader(); } + const std::list & getProcessors() const { return processors; } static std::list getProcessors(Chain chain) { return std::move(chain.processors); } private: diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index 478e26c602c..61081a5dc05 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -47,14 +47,21 @@ PushingPipelineExecutor::PushingPipelineExecutor(Chain & chain_) : chain(chain_) pushing_source = std::make_shared(chain.getInputHeader(), need_data_flag); auto sink = std::make_shared(chain.getOutputHeader()); connect(pushing_source->getPort(), chain.getInputPort()); - connect(chain.getOutputPort(), sink->getInputPort()); + connect(chain.getOutputPort(), sink->getPort()); + + processors = std::make_unique(); + processors->reserve(chain.getProcessors().size() + 2); + for (const auto & processor : chain.getProcessors()) + processors->push_back(processor); + processors->push_back(pushing_source); + processors->push_back(std::move(sink)); } PushingPipelineExecutor::~PushingPipelineExecutor() { try { - cancel(); + finish(); } catch (...) { diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index c03aacf2e1b..b97193ff4d6 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -109,6 +109,15 @@ void QueryPipeline::addChains(std::vector chains) pipe.addChains(std::move(chains)); } +void QueryPipeline::addChain(Chain chain) +{ + checkInitializedAndNotCompleted(); + std::vector chains; + chains.emplace_back(std::move(chain)); + pipe.resize(1); + pipe.addChains(std::move(chains)); +} + void QueryPipeline::transform(const Transformer & transformer) { checkInitializedAndNotCompleted(); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index d382bf21ebe..e2c66a677c8 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -57,6 +57,7 @@ public: void addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes); void addChains(std::vector chains); + void addChain(Chain chain); using Transformer = std::function; /// Transform pipeline in general way. diff --git a/src/Processors/Sinks/ExceptionHandlingSink.h b/src/Processors/Sinks/ExceptionHandlingSink.h index 285ef1de789..a16574dfbd8 100644 --- a/src/Processors/Sinks/ExceptionHandlingSink.h +++ b/src/Processors/Sinks/ExceptionHandlingSink.h @@ -41,7 +41,7 @@ public: std::rethrow_exception(std::move(last_exception)); } - InputPort & getInputPort() { return input; } + InputPort & getPort() { return input; } private: InputPort & input; diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index 7fb6f3d6248..61da1a38f21 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -161,6 +162,7 @@ SinkToStorage::SinkToStorage(const Block & header) : ExceptionKeepingTransform(h void SinkToStorage::transform(Chunk & chunk) { + std::cerr << "--- sink to storage rows " << chunk.getNumRows() << std::endl; consume(chunk.clone()); if (lastBlockIsDuplicate()) chunk.clear(); diff --git a/src/Processors/Sources/SourceFromSingleChunk.h b/src/Processors/Sources/SourceFromSingleChunk.h index d304bdbab93..8268fa5b0a6 100644 --- a/src/Processors/Sources/SourceFromSingleChunk.h +++ b/src/Processors/Sources/SourceFromSingleChunk.h @@ -9,6 +9,7 @@ class SourceFromSingleChunk : public SourceWithProgress { public: explicit SourceFromSingleChunk(Block header, Chunk chunk_) : SourceWithProgress(std::move(header)), chunk(std::move(chunk_)) {} + explicit SourceFromSingleChunk(Block data) : SourceWithProgress(data.cloneEmpty()), chunk(data.getColumns(), data.rows()) {} String getName() const override { return "SourceFromSingleChunk"; } protected: diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 6f69765ee23..127c83efff2 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -1,6 +1,6 @@ #include -#include - +#include +#include #include #include @@ -20,6 +20,7 @@ namespace ErrorCodes extern const int SET_SIZE_LIMIT_EXCEEDED; } +CreatingSetsTransform::~CreatingSetsTransform() = default; CreatingSetsTransform::CreatingSetsTransform( Block in_header_, @@ -50,7 +51,8 @@ void CreatingSetsTransform::startSubquery() LOG_TRACE(log, "Filling temporary table."); if (subquery.table) - table_out = std::make_shared(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext())); + /// TODO: make via port + table_out = Chain(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext())); done_with_set = !subquery.set; done_with_table = !subquery.table; @@ -58,8 +60,11 @@ void CreatingSetsTransform::startSubquery() if (done_with_set /*&& done_with_join*/ && done_with_table) throw Exception("Logical error: nothing to do with subquery", ErrorCodes::LOGICAL_ERROR); - if (table_out) - table_out->writePrefix(); + if (!table_out.empty()) + { + executor = std::make_unique(table_out); + executor->start(); + } } void CreatingSetsTransform::finishSubquery() @@ -104,7 +109,7 @@ void CreatingSetsTransform::consume(Chunk chunk) if (!done_with_table) { block = materializeBlock(block); - table_out->write(block); + executor->push(block); rows_to_transfer += block.rows(); bytes_to_transfer += block.bytes(); @@ -123,8 +128,11 @@ Chunk CreatingSetsTransform::generate() if (subquery.set) subquery.set->finishInsert(); - if (table_out) - table_out->writeSuffix(); + if (!table_out.empty()) + { + executor->finish(); + executor.reset(); + } finishSubquery(); return {}; diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index a847582a988..23eeca4f7c5 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -16,6 +17,8 @@ class QueryStatus; struct Progress; using ProgressCallback = std::function; +class PushingPipelineExecutor; + /// This processor creates set during execution. /// Don't return any data. Sets are created when Finish status is returned. /// In general, several work() methods need to be called to finish. @@ -30,6 +33,8 @@ public: SizeLimits network_transfer_limits_, ContextPtr context_); + ~CreatingSetsTransform() override; + String getName() const override { return "CreatingSetsTransform"; } void work() override; @@ -39,7 +44,8 @@ public: private: SubqueryForSet subquery; - BlockOutputStreamPtr table_out; + std::unique_ptr executor; + Chain table_out; UInt64 read_rows = 0; Stopwatch watch; diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index b90b0c33f17..c6cbea14209 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include @@ -23,7 +22,10 @@ #include #include #include +#include #include +#include +#include #include #include #include @@ -819,7 +821,7 @@ namespace void Call::processInput() { - if (!io.out) + if (io.out.empty()) return; bool has_data_to_insert = (insert_query && insert_query->data) @@ -834,18 +836,19 @@ namespace /// This is significant, because parallel parsing may be used. /// So we mustn't touch the input stream from other thread. - initializeBlockInputStream(io.out->getHeader()); + initializeBlockInputStream(io.out.getInputHeader()); - io.out->writePrefix(); + PushingPipelineExecutor executor(io.out); + executor.start(); Block block; while (pipeline_executor->pull(block)) { if (block) - io.out->write(block); + executor.push(block); } - io.out->writeSuffix(); + executor.finish(); } void Call::initializeBlockInputStream(const Block & header) @@ -977,7 +980,7 @@ namespace { /// The data will be written directly to the table. auto metadata_snapshot = storage->getInMemoryMetadataPtr(); - auto out_stream = std::make_shared(storage->write(ASTPtr(), metadata_snapshot, query_context)); + auto sink = storage->write(ASTPtr(), metadata_snapshot, query_context); ReadBufferFromMemory data(external_table.data().data(), external_table.data().size()); String format = external_table.format(); if (format.empty()) @@ -994,14 +997,20 @@ namespace external_table_context->checkSettingsConstraints(settings_changes); external_table_context->applySettingsChanges(settings_changes); } - auto in_stream = external_table_context->getInputFormat( - format, data, metadata_snapshot->getSampleBlock(), external_table_context->getSettings().max_insert_block_size); - in_stream->readPrefix(); - out_stream->writePrefix(); - while (auto block = in_stream->read()) - out_stream->write(block); - in_stream->readSuffix(); - out_stream->writeSuffix(); + auto in = FormatFactory::instance().getInput( + format, data, metadata_snapshot->getSampleBlock(), + external_table_context, external_table_context->getSettings().max_insert_block_size); + + QueryPipeline cur_pipeline; + cur_pipeline.init(Pipe(std::move(in))); + cur_pipeline.addTransform(std::move(sink)); + cur_pipeline.setSinks([&](const Block & header, Pipe::StreamType) + { + return std::make_shared(header); + }); + + auto executor = cur_pipeline.execute(); + executor->execute(1); } } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b557476f34c..23683256bed 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include @@ -36,6 +35,7 @@ #include #include +#include #include "Core/Protocol.h" #include "TCPHandler.h" @@ -1349,10 +1349,11 @@ bool TCPHandler::receiveData(bool scalar) } auto metadata_snapshot = storage->getInMemoryMetadataPtr(); /// The data will be written directly to the table. - auto temporary_table_out = std::make_shared(storage->write(ASTPtr(), metadata_snapshot, query_context)); - temporary_table_out->write(block); - temporary_table_out->writeSuffix(); - + Chain temporary_table_out(storage->write(ASTPtr(), metadata_snapshot, query_context)); + PushingPipelineExecutor executor(temporary_table_out); + executor.start(); + executor.push(block); + executor.finish(); } else if (state.need_receive_data_for_input) { diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index c674a705de1..f20cfc85b8b 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -266,7 +266,7 @@ namespace return nullptr; } - void writeAndConvert(RemoteBlockOutputStream & remote, ReadBufferFromFile & in) + void writeAndConvert(RemoteInserter & remote, ReadBufferFromFile & in) { CompressedReadBuffer decompressing_in(in); NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); @@ -287,7 +287,7 @@ namespace void writeRemoteConvert( const DistributedHeader & distributed_header, - RemoteBlockOutputStream & remote, + RemoteInserter & remote, bool compression_expected, ReadBufferFromFile & in, Poco::Logger * log) @@ -619,14 +619,13 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa formatReadableSizeWithBinarySuffix(distributed_header.bytes)); auto connection = pool->get(timeouts, &distributed_header.insert_settings); - RemoteBlockOutputStream remote{*connection, timeouts, + RemoteInserter remote{*connection, timeouts, distributed_header.insert_query, distributed_header.insert_settings, distributed_header.client_info}; - remote.writePrefix(); bool compression_expected = connection->getCompression() == Protocol::Compression::Enable; writeRemoteConvert(distributed_header, remote, compression_expected, in, log); - remote.writeSuffix(); + remote.onFinish(); } catch (Exception & e) { @@ -833,7 +832,7 @@ struct StorageDistributedDirectoryMonitor::Batch private: void sendBatch(Connection & connection, const ConnectionTimeouts & timeouts) { - std::unique_ptr remote; + std::unique_ptr remote; for (UInt64 file_idx : file_indices) { @@ -847,18 +846,17 @@ private: if (!remote) { - remote = std::make_unique(connection, timeouts, + remote = std::make_unique(connection, timeouts, distributed_header.insert_query, distributed_header.insert_settings, distributed_header.client_info); - remote->writePrefix(); } bool compression_expected = connection.getCompression() == Protocol::Compression::Enable; writeRemoteConvert(distributed_header, *remote, compression_expected, in, parent.log); } if (remote) - remote->writeSuffix(); + remote->onFinish(); } void sendSeparateFiles(Connection & connection, const ConnectionTimeouts & timeouts) @@ -880,14 +878,13 @@ private: ReadBufferFromFile in(file_path->second); const auto & distributed_header = readDistributedHeader(in, parent.log); - RemoteBlockOutputStream remote(connection, timeouts, + RemoteInserter remote(connection, timeouts, distributed_header.insert_query, distributed_header.insert_settings, distributed_header.client_info); - remote.writePrefix(); bool compression_expected = connection.getCompression() == Protocol::Compression::Enable; writeRemoteConvert(distributed_header, remote, compression_expected, in, parent.log); - remote.writeSuffix(); + remote.onFinish(); } catch (Exception & e) { diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index ec3f82d914c..be1a64be926 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -78,11 +79,11 @@ static Block adoptBlock(const Block & header, const Block & block, Poco::Logger } -static void writeBlockConvert(const BlockOutputStreamPtr & out, const Block & block, size_t repeats, Poco::Logger * log) +static void writeBlockConvert(PushingPipelineExecutor & executor, const Block & block, size_t repeats, Poco::Logger * log) { - Block adopted_block = adoptBlock(out->getHeader(), block, log); + Block adopted_block = adoptBlock(executor.getHeader(), block, log); for (size_t i = 0; i < repeats; ++i) - out->write(adopted_block); + executor.push(adopted_block); } @@ -124,7 +125,7 @@ void DistributedSink::consume(Chunk chunk) is_first_chunk = false; } - auto ordinary_block = getPort().getHeader().cloneWithColumns(chunk.detachColumns()); + auto ordinary_block = getHeader().cloneWithColumns(chunk.detachColumns()); if (!allow_materialized) { @@ -322,7 +323,7 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si if (!job.is_local_job || !settings.prefer_localhost_replica) { - if (!job.stream) + if (!job.executor) { auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings); if (shard_info.hasInternalReplication()) @@ -354,19 +355,20 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si if (throttler) job.connection_entry->setThrottler(throttler); - job.stream = std::make_shared( - *job.connection_entry, timeouts, query_string, settings, context->getClientInfo()); - job.stream->writePrefix(); + job.chain.addSource(std::make_shared( + *job.connection_entry, timeouts, query_string, settings, context->getClientInfo())); + job.executor = std::make_unique(job.chain); + job.executor->start(); } CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; - Block adopted_shard_block = adoptBlock(job.stream->getHeader(), shard_block, log); - job.stream->write(adopted_shard_block); + Block adopted_shard_block = adoptBlock(job.executor->getHeader(), shard_block, log); + job.executor->push(adopted_shard_block); } else // local { - if (!job.stream) + if (!job.executor) { /// Forward user settings job.local_context = Context::createCopy(context); @@ -382,11 +384,12 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si InterpreterInsertQuery interp(copy_query_ast, job.local_context, allow_materialized); auto block_io = interp.execute(); - job.stream = block_io.out; - job.stream->writePrefix(); + job.chain = std::move(block_io.out); + job.executor = std::make_unique(job.chain); + job.executor->start(); } - writeBlockConvert(job.stream, shard_block, shard_info.getLocalNodeCount(), log); + writeBlockConvert(*job.executor, shard_block, shard_info.getLocalNodeCount(), log); } job.blocks_written += 1; @@ -498,11 +501,11 @@ void DistributedSink::onFinish() { for (JobReplica & job : shard_jobs.replicas_jobs) { - if (job.stream) + if (job.executor) { pool->scheduleOrThrowOnError([&job]() { - job.stream->writeSuffix(); + job.executor->finish(); }); } } @@ -618,10 +621,11 @@ void DistributedSink::writeToLocal(const Block & block, size_t repeats) InterpreterInsertQuery interp(query_ast, context, allow_materialized); auto block_io = interp.execute(); + PushingPipelineExecutor executor(block_io.out); - block_io.out->writePrefix(); - writeBlockConvert(block_io.out, block, repeats, log); - block_io.out->writeSuffix(); + executor.start(); + writeBlockConvert(executor, block, repeats, log); + executor.finish(); } diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index af04f8c8aac..14498c57364 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -24,6 +25,7 @@ namespace DB class Context; class StorageDistributed; +class PushingPipelineExecutor; /** If insert_sync_ is true, the write is synchronous. Uses insert_timeout_ if it is not zero. * Otherwise, the write is asynchronous - the data is first written to the local filesystem, and then sent to the remote servers. @@ -119,7 +121,8 @@ private: ConnectionPool::Entry connection_entry; ContextPtr local_context; - BlockOutputStreamPtr stream; + Chain chain; + std::unique_ptr executor; UInt64 blocks_written = 0; UInt64 rows_written = 0; diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 9600eb975b4..ae66f258d5b 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -195,7 +195,7 @@ public: writer->writePrefix(); is_first_chunk = false; } - writer->write(getPort().getHeader().cloneWithColumns(chunk.detachColumns())); + writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } void onFinish() override diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.cpp b/src/Storages/Kafka/KafkaBlockOutputStream.cpp index c7fe71f42c1..395065de084 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockOutputStream.cpp @@ -19,13 +19,13 @@ KafkaSink::KafkaSink( void KafkaSink::onStart() { - buffer = storage.createWriteBuffer(getPort().getHeader()); + buffer = storage.createWriteBuffer(getHeader()); auto format_settings = getFormatSettings(context); format_settings.protobuf.allow_multiple_rows_without_delimiter = true; child = FormatFactory::instance().getOutputStream(storage.getFormatName(), *buffer, - getPort().getHeader(), context, + getHeader(), context, [this](const Columns & columns, size_t row) { buffer->countRow(columns, row); @@ -35,7 +35,7 @@ void KafkaSink::onStart() void KafkaSink::consume(Chunk chunk) { - child->write(getPort().getHeader().cloneWithColumns(chunk.detachColumns())); + child->write(getHeader().cloneWithColumns(chunk.detachColumns())); } void KafkaSink::onFinish() diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index cba67bc3bcb..50411c826f7 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -36,6 +36,7 @@ #include #include #include +#include #include #include @@ -616,7 +617,7 @@ bool StorageKafka::streamToViews() streams.reserve(stream_count); for (size_t i = 0; i < stream_count; ++i) { - auto stream = std::make_shared(*this, metadata_snapshot, kafka_context, block_io.out->getHeader().getNames(), log, block_size, false); + auto stream = std::make_shared(*this, metadata_snapshot, kafka_context, block_io.out.getInputHeader().getNames(), log, block_size, false); streams.emplace_back(stream); // Limit read batch to maximum block size to allow DDL @@ -639,12 +640,21 @@ bool StorageKafka::streamToViews() // We can't cancel during copyData, as it's not aware of commits and other kafka-related stuff. // It will be cancelled on underlying layer (kafka buffer) - std::atomic stub = {false}; + size_t rows = 0; - copyData(*in, *block_io.out, [&rows](const Block & block) + PushingPipelineExecutor executor(block_io.out); + + in->readPrefix(); + executor.start(); + + while (auto block = in->read()) { rows += block.rows(); - }, &stub); + executor.push(std::move(block)); + } + + executor.finish(); + in->readSuffix(); bool some_stream_is_stalled = false; for (auto & stream : streams) diff --git a/src/Storages/LiveView/LiveViewSink.h b/src/Storages/LiveView/LiveViewSink.h index 433a5554152..bbb8bf02c45 100644 --- a/src/Storages/LiveView/LiveViewSink.h +++ b/src/Storages/LiveView/LiveViewSink.h @@ -76,7 +76,7 @@ public: void consume(Chunk chunk) override { - auto block = getPort().getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); block.updateHash(*new_hash); new_blocks->push_back(std::move(block)); } diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 73c753386a4..99b8e83b464 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -17,7 +17,7 @@ void MergeTreeSink::onStart() void MergeTreeSink::consume(Chunk chunk) { - auto block = getPort().getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context); for (auto & current_block : part_blocks) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index c81f587cbbc..75308f872dc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -118,7 +118,7 @@ void ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & zoo void ReplicatedMergeTreeSink::consume(Chunk chunk) { - auto block = getPort().getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); last_block_is_duplicate = false; diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index b43e7656084..e70996c8b3b 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -8,7 +8,8 @@ #include #include #include - +#include +#include namespace DB { @@ -485,10 +486,20 @@ void MaterializedPostgreSQLConsumer::syncTables() InterpreterInsertQuery interpreter(insert, insert_context, true); auto block_io = interpreter.execute(); - OneBlockInputStream input(result_rows); + auto input = std::make_shared( + result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); - assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "postgresql replica table sync"); - copyData(input, *block_io.out); + assertBlocksHaveEqualStructure(input->getPort().getHeader(), block_io.out.getInputHeader(), "postgresql replica table sync"); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); + pipeline.addChain(std::move(block_io.out)); + pipeline.setSinks([&](const Block & header, Pipe::StreamType) + { + return std::make_shared(header); + }); + + auto executor = pipeline.execute(); + executor->execute(1); buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 3477397adb7..97a0f17b506 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -231,14 +231,15 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(String & snapshot_name auto input = std::make_unique>(tx, query_str, sample_block, DEFAULT_BLOCK_SIZE); QueryPipeline pipeline; pipeline.init(Pipe(std::move(input))); - assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out->getHeader(), "postgresql replica load from snapshot"); + assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out.getInputHeader(), "postgresql replica load from snapshot"); + pipeline.addChain(std::move(block_io.out)); + pipeline.setSinks([&](const Block & header, Pipe::StreamType) + { + return std::make_shared(header); + }); - PullingPipelineExecutor executor(pipeline); - Block block; - block_io.out->writePrefix(); - while (executor.pull(block)) - block_io.out->write(block); - block_io.out->writeSuffix(); + auto executor = pipeline.execute(); + executor->execute(1); nested_storage = materialized_storage->prepare(); auto nested_table_id = nested_storage->getStorageID(); diff --git a/src/Storages/RabbitMQ/RabbitMQSink.cpp b/src/Storages/RabbitMQ/RabbitMQSink.cpp index 9c556ee0832..b71f3ce86b7 100644 --- a/src/Storages/RabbitMQ/RabbitMQSink.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSink.cpp @@ -31,8 +31,7 @@ void RabbitMQSink::onStart() auto format_settings = getFormatSettings(context); format_settings.protobuf.allow_multiple_rows_without_delimiter = true; - child = FormatFactory::instance().getOutputStream(storage.getFormatName(), *buffer, - getPort().getHeader(), context, + child = FormatFactory::instance().getOutputStream(storage.getFormatName(), *buffer, getHeader(), context, [this](const Columns & /* columns */, size_t /* rows */) { buffer->countRow(); @@ -43,7 +42,7 @@ void RabbitMQSink::onStart() void RabbitMQSink::consume(Chunk chunk) { - child->write(getPort().getHeader().cloneWithColumns(chunk.detachColumns())); + child->write(getHeader().cloneWithColumns(chunk.detachColumns())); } diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 44622f106f4..18a2ef57c9e 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include namespace DB @@ -918,7 +919,7 @@ bool StorageRabbitMQ::streamToViews() auto block_io = interpreter.execute(); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto column_names = block_io.out->getHeader().getNames(); + auto column_names = block_io.out.getInputHeader().getNames(); auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); auto block_size = getMaxBlockSize(); @@ -952,15 +953,19 @@ bool StorageRabbitMQ::streamToViews() else in = streams[0]; - std::atomic stub = {false}; - if (!event_handler->loopRunning()) { event_handler->updateLoopState(Loop::RUN); looping_task->activateAndSchedule(); } - copyData(*in, *block_io.out, &stub); + PushingPipelineExecutor executor(block_io.out); + executor.start(); + in->readPrefix(); + while (auto block = in->read()) + executor.push(std::move(block)); + executor.finish(); + in->readSuffix(); /* Note: sending ack() with loop running in another thread will lead to a lot of data races inside the library, but only in case * error occurs or connection is lost while ack is being sent diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp index ddf839b6427..1c918c15775 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp @@ -20,7 +20,7 @@ EmbeddedRocksDBSink::EmbeddedRocksDBSink( , storage(storage_) , metadata_snapshot(metadata_snapshot_) { - for (const auto & elem : getPort().getHeader()) + for (const auto & elem : getHeader()) { if (elem.name == storage.primary_key) break; @@ -31,7 +31,7 @@ EmbeddedRocksDBSink::EmbeddedRocksDBSink( void EmbeddedRocksDBSink::consume(Chunk chunk) { auto rows = chunk.getNumRows(); - auto block = getPort().getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); WriteBufferFromOwnString wb_key; WriteBufferFromOwnString wb_value; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index e9136bb5d05..57f16ac9b7d 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -31,6 +31,7 @@ #include #include #include +#include namespace ProfileEvents @@ -525,7 +526,7 @@ public: , metadata_snapshot(metadata_snapshot_) { // Check table structure. - metadata_snapshot->check(getPort().getHeader(), true); + metadata_snapshot->check(getHeader(), true); } String getName() const override { return "BufferSink"; } @@ -536,7 +537,7 @@ public: if (!rows) return; - auto block = getPort().getHeader().cloneWithColumns(chunk.getColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); StoragePtr destination; if (storage.destination_id) @@ -960,9 +961,10 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl InterpreterInsertQuery interpreter{insert, insert_context, allow_materialized}; auto block_io = interpreter.execute(); - block_io.out->writePrefix(); - block_io.out->write(block_to_write); - block_io.out->writeSuffix(); + PushingPipelineExecutor executor(block_io.out); + executor.start(); + executor.push(std::move(block_to_write)); + executor.finish(); } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index cc8e397b668..b73f20b45db 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -580,7 +580,7 @@ public: void consume(Chunk chunk) override { - writer->write(getPort().getHeader().cloneWithColumns(chunk.detachColumns())); + writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } void onFinish() override diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 0e156f24cc2..e030b8d8784 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -306,7 +306,7 @@ private: void LogSink::consume(Chunk chunk) { - auto block = getPort().getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); metadata_snapshot->check(block, true); /// The set of written offset columns so that you do not write shared offsets of columns for nested structures multiple times @@ -332,7 +332,7 @@ void LogSink::onFinish() WrittenStreams written_streams; ISerialization::SerializeBinaryBulkSettings settings; - for (const auto & column : getPort().getHeader()) + for (const auto & column : getHeader()) { auto it = serialize_states.find(column.name); if (it != serialize_states.end()) diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 6823f661984..6caae3bd02d 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -115,7 +115,7 @@ public: void consume(Chunk chunk) override { - auto block = getPort().getHeader().cloneWithColumns(chunk.getColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); metadata_snapshot->check(block, true); if (storage.compress) diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 79bb1f59cc7..20e1993a00c 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -132,7 +132,7 @@ public: void consume(Chunk chunk) override { - auto block = getPort().getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); auto blocks = splitBlocks(block, max_batch_rows); mysqlxx::Transaction trans(entry); try diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 5f8b81a47cf..81f41424f5a 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -114,7 +114,7 @@ public: void consume(Chunk chunk) override { - auto block = getPort().getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); if (!inserter) inserter = std::make_unique(connection_holder->get(), remote_table_schema.empty() ? pqxx::table_path({remote_table_name}) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index fc3ce3a10ed..552cf739ef7 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -296,7 +296,7 @@ public: writer->writePrefix(); is_first_chunk = false; } - writer->write(getPort().getHeader().cloneWithColumns(chunk.detachColumns())); + writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } // void flush() override diff --git a/src/Storages/StorageSQLite.cpp b/src/Storages/StorageSQLite.cpp index 758284e8d50..e4dcc1af8d5 100644 --- a/src/Storages/StorageSQLite.cpp +++ b/src/Storages/StorageSQLite.cpp @@ -102,7 +102,7 @@ public: void consume(Chunk chunk) override { - auto block = getPort().getHeader().cloneWithColumns(chunk.getColumns()); + auto block = getHeader().cloneWithColumns(chunk.getColumns()); WriteBufferFromOwnString sqlbuf; sqlbuf << "INSERT INTO "; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 67fd89f5098..21fe5e23993 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -79,7 +79,7 @@ SetOrJoinSink::SetOrJoinSink( void SetOrJoinSink::consume(Chunk chunk) { /// Sort columns in the block. This is necessary, since Set and Join count on the same column order in different blocks. - Block sorted_block = getPort().getHeader().cloneWithColumns(chunk.detachColumns()).sortColumns(); + Block sorted_block = getHeader().cloneWithColumns(chunk.detachColumns()).sortColumns(); table.insertBlock(sorted_block); if (persistent) diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 6bf91a145ed..196a5ab367b 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -208,7 +208,7 @@ public: void consume(Chunk chunk) override { - block_out.write(getPort().getHeader().cloneWithColumns(chunk.detachColumns())); + block_out.write(getHeader().cloneWithColumns(chunk.detachColumns())); } void onFinish() override diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 53ae74c4e00..2e3160a553f 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -332,7 +332,7 @@ void TinyLogSink::onFinish() WrittenStreams written_streams; ISerialization::SerializeBinaryBulkSettings settings; - for (const auto & column : getPort().getHeader()) + for (const auto & column : getHeader()) { auto it = serialize_states.find(column.name); if (it != serialize_states.end()) @@ -367,7 +367,7 @@ void TinyLogSink::onFinish() void TinyLogSink::consume(Chunk chunk) { - auto block = getPort().getHeader().cloneWithColumns(chunk.detachColumns()); + auto block = getHeader().cloneWithColumns(chunk.detachColumns()); metadata_snapshot->check(block, true); /// The set of written offset columns so that you do not write shared columns for nested structures multiple times diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 1aa5ac7f236..68d9d64297a 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -170,7 +170,7 @@ void StorageURLSink::consume(Chunk chunk) is_first_chunk = false; } - writer->write(getPort().getHeader().cloneWithColumns(chunk.detachColumns())); + writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } void StorageURLSink::onFinish() diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index ace5ca3667c..fd1652b2b04 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -36,6 +36,8 @@ Pipe StorageValues::read( for (const auto & name : column_names) block.insert(res_block.getByName(name)); + std::cerr << "=== Reading from Values " << block.rows() << " rows\n"; + Chunk chunk(block.getColumns(), block.rows()); return Pipe(std::make_shared(block.cloneEmpty(), std::move(chunk))); } From a208ee5f4cff4206ed9147250d88ed626424a499 Mon Sep 17 00:00:00 2001 From: Roman Zhukov Date: Mon, 6 Sep 2021 17:58:17 +0000 Subject: [PATCH 009/142] Add touch drafts. --- .../getting-started/example-datasets/index.md | 3 + .../example-datasets/opensky.md | 384 ++++++++++++ .../example-datasets/uk-price-paid.md | 581 ++++++++++++++++++ 3 files changed, 968 insertions(+) create mode 100644 docs/ru/getting-started/example-datasets/opensky.md create mode 100644 docs/ru/getting-started/example-datasets/uk-price-paid.md diff --git a/docs/ru/getting-started/example-datasets/index.md b/docs/ru/getting-started/example-datasets/index.md index 756b3a75dee..e046fae5fa5 100644 --- a/docs/ru/getting-started/example-datasets/index.md +++ b/docs/ru/getting-started/example-datasets/index.md @@ -16,5 +16,8 @@ toc_title: "Введение" - [AMPLab Big Data Benchmark](amplab-benchmark.md) - [Данные о такси в Нью-Йорке](nyc-taxi.md) - [OnTime](ontime.md) +- [OpenSky](../../getting-started/example-datasets/opensky.md) +- [Данные о недвижимости в Великобритании](../../getting-started/example-datasets/uk-price-paid.md) - [Вышки сотовой связи](../../getting-started/example-datasets/cell-towers.md) +[Оригинальная статья](https://clickhouse.tech/docs/ru/getting_started/example_datasets) diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md new file mode 100644 index 00000000000..c9f6729cdd7 --- /dev/null +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -0,0 +1,384 @@ +--- +toc_priority: 20 +toc_title: OpenSky +--- + +# Crowdsourced air traffic data from The OpenSky Network 2020 + +"The data in this dataset is derived and cleaned from the full OpenSky dataset to illustrate the development of air traffic during the COVID-19 pandemic. It spans all flights seen by the network's more than 2500 members since 1 January 2019. More data will be periodically included in the dataset until the end of the COVID-19 pandemic". + +Source: https://zenodo.org/record/5092942#.YRBCyTpRXYd + +Martin Strohmeier, Xavier Olive, Jannis Lübbe, Matthias Schäfer, and Vincent Lenders +"Crowdsourced air traffic data from the OpenSky Network 2019–2020" +Earth System Science Data 13(2), 2021 +https://doi.org/10.5194/essd-13-357-2021 + +## Download the Dataset + +``` +wget -O- https://zenodo.org/record/5092942 | grep -oP 'https://zenodo.org/record/5092942/files/flightlist_\d+_\d+\.csv\.gz' | xargs wget +``` + +Download will take about 2 minutes with good internet connection. There are 30 files with total size of 4.3 GB. + +## Create the Table + +``` +CREATE TABLE opensky +( + callsign String, + number String, + icao24 String, + registration String, + typecode String, + origin String, + destination String, + firstseen DateTime, + lastseen DateTime, + day DateTime, + latitude_1 Float64, + longitude_1 Float64, + altitude_1 Float64, + latitude_2 Float64, + longitude_2 Float64, + altitude_2 Float64 +) ENGINE = MergeTree ORDER BY (origin, destination, callsign); +``` + +## Import Data + +Upload data into ClickHouse in parallel: + +``` +ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c ' + gzip -c -d "{}" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"' +``` + +Here we pass the list of files (`ls -1 flightlist_*.csv.gz`) to `xargs` for parallel processing. +`xargs -P100` specifies to use up to 100 parallel workers but as we only have 30 files, the number of workers will be only 30. + +For every file, `xargs` will run a script with `bash -c`. The script has substitution in form of `{}` and the `xargs` command will substitute the filename to it (we have asked it for xargs with `-I{}`). + +The script will decompress the file (`gzip -c -d "{}"`) to standard output (`-c` parameter) and the output is redirected to `clickhouse-client`. + +Finally, `clickhouse-client` will do insertion. It will read input data in `CSVWithNames` format. We also asked to parse DateTime fields with extended parser (`--date_time_input_format best_effort`) to recognize ISO-8601 format with timezone offsets. + +Parallel upload takes 24 seconds. + +If you don't like parallel upload, here is sequential variant: +``` +for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"; done +``` + +## Validate the Data + +``` +SELECT count() FROM opensky +66010819 +``` + +The size of dataset in ClickHouse is just 2.64 GiB: + +``` +SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' +2.64 GiB +``` + +## Run Some Queries + +Total distance travelled is 68 billion kilometers: + +``` +SELECT formatReadableQuantity(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) / 1000) FROM opensky + +┌─formatReadableQuantity(divide(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)), 1000))─┐ +│ 68.72 billion │ +└──────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +Average flight distance is around 1000 km. +``` +SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM opensky + +┌─avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2))─┐ +│ 1041090.6465708319 │ +└────────────────────────────────────────────────────────────────────┘ +``` + +### Most busy origin airports and the average distance seen: + +``` +SELECT + origin, + count(), + round(avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2))) AS distance, + bar(distance, 0, 10000000, 100) AS bar +FROM opensky +WHERE origin != '' +GROUP BY origin +ORDER BY count() DESC +LIMIT 100 + +Query id: f9010ea5-97d0-45a3-a5bd-9657906cd105 + + ┌─origin─┬─count()─┬─distance─┬─bar────────────────────────────────────┐ + 1. │ KORD │ 745007 │ 1546108 │ ███████████████▍ │ + 2. │ KDFW │ 696702 │ 1358721 │ █████████████▌ │ + 3. │ KATL │ 667286 │ 1169661 │ ███████████▋ │ + 4. │ KDEN │ 582709 │ 1287742 │ ████████████▊ │ + 5. │ KLAX │ 581952 │ 2628393 │ ██████████████████████████▎ │ + 6. │ KLAS │ 447789 │ 1336967 │ █████████████▎ │ + 7. │ KPHX │ 428558 │ 1345635 │ █████████████▍ │ + 8. │ KSEA │ 412592 │ 1757317 │ █████████████████▌ │ + 9. │ KCLT │ 404612 │ 880355 │ ████████▋ │ + 10. │ VIDP │ 363074 │ 1445052 │ ██████████████▍ │ + 11. │ EDDF │ 362643 │ 2263960 │ ██████████████████████▋ │ + 12. │ KSFO │ 361869 │ 2445732 │ ████████████████████████▍ │ + 13. │ KJFK │ 349232 │ 2996550 │ █████████████████████████████▊ │ + 14. │ KMSP │ 346010 │ 1287328 │ ████████████▋ │ + 15. │ LFPG │ 344748 │ 2206203 │ ██████████████████████ │ + 16. │ EGLL │ 341370 │ 3216593 │ ████████████████████████████████▏ │ + 17. │ EHAM │ 340272 │ 2116425 │ █████████████████████▏ │ + 18. │ KEWR │ 337696 │ 1826545 │ ██████████████████▎ │ + 19. │ KPHL │ 320762 │ 1291761 │ ████████████▊ │ + 20. │ OMDB │ 308855 │ 2855706 │ ████████████████████████████▌ │ + 21. │ UUEE │ 307098 │ 1555122 │ ███████████████▌ │ + 22. │ KBOS │ 304416 │ 1621675 │ ████████████████▏ │ + 23. │ LEMD │ 291787 │ 1695097 │ ████████████████▊ │ + 24. │ YSSY │ 272979 │ 1875298 │ ██████████████████▋ │ + 25. │ KMIA │ 265121 │ 1923542 │ ███████████████████▏ │ + 26. │ ZGSZ │ 263497 │ 745086 │ ███████▍ │ + 27. │ EDDM │ 256691 │ 1361453 │ █████████████▌ │ + 28. │ WMKK │ 254264 │ 1626688 │ ████████████████▎ │ + 29. │ CYYZ │ 251192 │ 2175026 │ █████████████████████▋ │ + 30. │ KLGA │ 248699 │ 1106935 │ ███████████ │ + 31. │ VHHH │ 248473 │ 3457658 │ ██████████████████████████████████▌ │ + 32. │ RJTT │ 243477 │ 1272744 │ ████████████▋ │ + 33. │ KBWI │ 241440 │ 1187060 │ ███████████▋ │ + 34. │ KIAD │ 239558 │ 1683485 │ ████████████████▋ │ + 35. │ KIAH │ 234202 │ 1538335 │ ███████████████▍ │ + 36. │ KFLL │ 223447 │ 1464410 │ ██████████████▋ │ + 37. │ KDAL │ 212055 │ 1082339 │ ██████████▋ │ + 38. │ KDCA │ 207883 │ 1013359 │ ██████████▏ │ + 39. │ LIRF │ 207047 │ 1427965 │ ██████████████▎ │ + 40. │ PANC │ 206007 │ 2525359 │ █████████████████████████▎ │ + 41. │ LTFJ │ 205415 │ 860470 │ ████████▌ │ + 42. │ KDTW │ 204020 │ 1106716 │ ███████████ │ + 43. │ VABB │ 201679 │ 1300865 │ █████████████ │ + 44. │ OTHH │ 200797 │ 3759544 │ █████████████████████████████████████▌ │ + 45. │ KMDW │ 200796 │ 1232551 │ ████████████▎ │ + 46. │ KSAN │ 198003 │ 1495195 │ ██████████████▊ │ + 47. │ KPDX │ 197760 │ 1269230 │ ████████████▋ │ + 48. │ SBGR │ 197624 │ 2041697 │ ████████████████████▍ │ + 49. │ VOBL │ 189011 │ 1040180 │ ██████████▍ │ + 50. │ LEBL │ 188956 │ 1283190 │ ████████████▋ │ + 51. │ YBBN │ 188011 │ 1253405 │ ████████████▌ │ + 52. │ LSZH │ 187934 │ 1572029 │ ███████████████▋ │ + 53. │ YMML │ 187643 │ 1870076 │ ██████████████████▋ │ + 54. │ RCTP │ 184466 │ 2773976 │ ███████████████████████████▋ │ + 55. │ KSNA │ 180045 │ 778484 │ ███████▋ │ + 56. │ EGKK │ 176420 │ 1694770 │ ████████████████▊ │ + 57. │ LOWW │ 176191 │ 1274833 │ ████████████▋ │ + 58. │ UUDD │ 176099 │ 1368226 │ █████████████▋ │ + 59. │ RKSI │ 173466 │ 3079026 │ ██████████████████████████████▋ │ + 60. │ EKCH │ 172128 │ 1229895 │ ████████████▎ │ + 61. │ KOAK │ 171119 │ 1114447 │ ███████████▏ │ + 62. │ RPLL │ 170122 │ 1440735 │ ██████████████▍ │ + 63. │ KRDU │ 167001 │ 830521 │ ████████▎ │ + 64. │ KAUS │ 164524 │ 1256198 │ ████████████▌ │ + 65. │ KBNA │ 163242 │ 1022726 │ ██████████▏ │ + 66. │ KSDF │ 162655 │ 1380867 │ █████████████▋ │ + 67. │ ENGM │ 160732 │ 910108 │ █████████ │ + 68. │ LIMC │ 160696 │ 1564620 │ ███████████████▋ │ + 69. │ KSJC │ 159278 │ 1081125 │ ██████████▋ │ + 70. │ KSTL │ 157984 │ 1026699 │ ██████████▎ │ + 71. │ UUWW │ 156811 │ 1261155 │ ████████████▌ │ + 72. │ KIND │ 153929 │ 987944 │ █████████▊ │ + 73. │ ESSA │ 153390 │ 1203439 │ ████████████ │ + 74. │ KMCO │ 153351 │ 1508657 │ ███████████████ │ + 75. │ KDVT │ 152895 │ 74048 │ ▋ │ + 76. │ VTBS │ 152645 │ 2255591 │ ██████████████████████▌ │ + 77. │ CYVR │ 149574 │ 2027413 │ ████████████████████▎ │ + 78. │ EIDW │ 148723 │ 1503985 │ ███████████████ │ + 79. │ LFPO │ 143277 │ 1152964 │ ███████████▌ │ + 80. │ EGSS │ 140830 │ 1348183 │ █████████████▍ │ + 81. │ KAPA │ 140776 │ 420441 │ ████▏ │ + 82. │ KHOU │ 138985 │ 1068806 │ ██████████▋ │ + 83. │ KTPA │ 138033 │ 1338223 │ █████████████▍ │ + 84. │ KFFZ │ 137333 │ 55397 │ ▌ │ + 85. │ NZAA │ 136092 │ 1581264 │ ███████████████▋ │ + 86. │ YPPH │ 133916 │ 1271550 │ ████████████▋ │ + 87. │ RJBB │ 133522 │ 1805623 │ ██████████████████ │ + 88. │ EDDL │ 133018 │ 1265919 │ ████████████▋ │ + 89. │ ULLI │ 130501 │ 1197108 │ ███████████▊ │ + 90. │ KIWA │ 127195 │ 250876 │ ██▌ │ + 91. │ KTEB │ 126969 │ 1189414 │ ███████████▊ │ + 92. │ VOMM │ 125616 │ 1127757 │ ███████████▎ │ + 93. │ LSGG │ 123998 │ 1049101 │ ██████████▍ │ + 94. │ LPPT │ 122733 │ 1779187 │ █████████████████▋ │ + 95. │ WSSS │ 120493 │ 3264122 │ ████████████████████████████████▋ │ + 96. │ EBBR │ 118539 │ 1579939 │ ███████████████▋ │ + 97. │ VTBD │ 118107 │ 661627 │ ██████▌ │ + 98. │ KVNY │ 116326 │ 692960 │ ██████▊ │ + 99. │ EDDT │ 115122 │ 941740 │ █████████▍ │ +100. │ EFHK │ 114860 │ 1629143 │ ████████████████▎ │ + └────────┴─────────┴──────────┴────────────────────────────────────────┘ + +100 rows in set. Elapsed: 0.186 sec. Processed 48.31 million rows, 2.17 GB (259.27 million rows/s., 11.67 GB/s.) +``` + +### Number of flights from three major Moscow airports, weekly: + +``` +SELECT + toMonday(day) AS k, + count() AS c, + bar(c, 0, 10000, 100) AS bar +FROM opensky +WHERE origin IN ('UUEE', 'UUDD', 'UUWW') +GROUP BY k +ORDER BY k ASC + +Query id: 1b446157-9519-4cc4-a1cb-178dfcc15a8e + + ┌──────────k─┬────c─┬─bar──────────────────────────────────────────────────────────────────────────┐ + 1. │ 2018-12-31 │ 5248 │ ████████████████████████████████████████████████████▍ │ + 2. │ 2019-01-07 │ 6302 │ ███████████████████████████████████████████████████████████████ │ + 3. │ 2019-01-14 │ 5701 │ █████████████████████████████████████████████████████████ │ + 4. │ 2019-01-21 │ 5638 │ ████████████████████████████████████████████████████████▍ │ + 5. │ 2019-01-28 │ 5731 │ █████████████████████████████████████████████████████████▎ │ + 6. │ 2019-02-04 │ 5683 │ ████████████████████████████████████████████████████████▋ │ + 7. │ 2019-02-11 │ 5759 │ █████████████████████████████████████████████████████████▌ │ + 8. │ 2019-02-18 │ 5736 │ █████████████████████████████████████████████████████████▎ │ + 9. │ 2019-02-25 │ 5873 │ ██████████████████████████████████████████████████████████▋ │ + 10. │ 2019-03-04 │ 5965 │ ███████████████████████████████████████████████████████████▋ │ + 11. │ 2019-03-11 │ 5900 │ ███████████████████████████████████████████████████████████ │ + 12. │ 2019-03-18 │ 5823 │ ██████████████████████████████████████████████████████████▏ │ + 13. │ 2019-03-25 │ 5899 │ ██████████████████████████████████████████████████████████▊ │ + 14. │ 2019-04-01 │ 6043 │ ████████████████████████████████████████████████████████████▍ │ + 15. │ 2019-04-08 │ 6098 │ ████████████████████████████████████████████████████████████▊ │ + 16. │ 2019-04-15 │ 6196 │ █████████████████████████████████████████████████████████████▊ │ + 17. │ 2019-04-22 │ 6486 │ ████████████████████████████████████████████████████████████████▋ │ + 18. │ 2019-04-29 │ 6682 │ ██████████████████████████████████████████████████████████████████▋ │ + 19. │ 2019-05-06 │ 6739 │ ███████████████████████████████████████████████████████████████████▍ │ + 20. │ 2019-05-13 │ 6600 │ ██████████████████████████████████████████████████████████████████ │ + 21. │ 2019-05-20 │ 6575 │ █████████████████████████████████████████████████████████████████▋ │ + 22. │ 2019-05-27 │ 6786 │ ███████████████████████████████████████████████████████████████████▋ │ + 23. │ 2019-06-03 │ 6872 │ ████████████████████████████████████████████████████████████████████▋ │ + 24. │ 2019-06-10 │ 7045 │ ██████████████████████████████████████████████████████████████████████▍ │ + 25. │ 2019-06-17 │ 7045 │ ██████████████████████████████████████████████████████████████████████▍ │ + 26. │ 2019-06-24 │ 6852 │ ████████████████████████████████████████████████████████████████████▌ │ + 27. │ 2019-07-01 │ 7248 │ ████████████████████████████████████████████████████████████████████████▍ │ + 28. │ 2019-07-08 │ 7284 │ ████████████████████████████████████████████████████████████████████████▋ │ + 29. │ 2019-07-15 │ 7142 │ ███████████████████████████████████████████████████████████████████████▍ │ + 30. │ 2019-07-22 │ 7108 │ ███████████████████████████████████████████████████████████████████████ │ + 31. │ 2019-07-29 │ 7251 │ ████████████████████████████████████████████████████████████████████████▌ │ + 32. │ 2019-08-05 │ 7403 │ ██████████████████████████████████████████████████████████████████████████ │ + 33. │ 2019-08-12 │ 7457 │ ██████████████████████████████████████████████████████████████████████████▌ │ + 34. │ 2019-08-19 │ 7502 │ ███████████████████████████████████████████████████████████████████████████ │ + 35. │ 2019-08-26 │ 7540 │ ███████████████████████████████████████████████████████████████████████████▍ │ + 36. │ 2019-09-02 │ 7237 │ ████████████████████████████████████████████████████████████████████████▎ │ + 37. │ 2019-09-09 │ 7328 │ █████████████████████████████████████████████████████████████████████████▎ │ + 38. │ 2019-09-16 │ 5566 │ ███████████████████████████████████████████████████████▋ │ + 39. │ 2019-09-23 │ 7049 │ ██████████████████████████████████████████████████████████████████████▍ │ + 40. │ 2019-09-30 │ 6880 │ ████████████████████████████████████████████████████████████████████▋ │ + 41. │ 2019-10-07 │ 6518 │ █████████████████████████████████████████████████████████████████▏ │ + 42. │ 2019-10-14 │ 6688 │ ██████████████████████████████████████████████████████████████████▊ │ + 43. │ 2019-10-21 │ 6667 │ ██████████████████████████████████████████████████████████████████▋ │ + 44. │ 2019-10-28 │ 6303 │ ███████████████████████████████████████████████████████████████ │ + 45. │ 2019-11-04 │ 6298 │ ██████████████████████████████████████████████████████████████▊ │ + 46. │ 2019-11-11 │ 6137 │ █████████████████████████████████████████████████████████████▎ │ + 47. │ 2019-11-18 │ 6051 │ ████████████████████████████████████████████████████████████▌ │ + 48. │ 2019-11-25 │ 5820 │ ██████████████████████████████████████████████████████████▏ │ + 49. │ 2019-12-02 │ 5942 │ ███████████████████████████████████████████████████████████▍ │ + 50. │ 2019-12-09 │ 4891 │ ████████████████████████████████████████████████▊ │ + 51. │ 2019-12-16 │ 5682 │ ████████████████████████████████████████████████████████▋ │ + 52. │ 2019-12-23 │ 6111 │ █████████████████████████████████████████████████████████████ │ + 53. │ 2019-12-30 │ 5870 │ ██████████████████████████████████████████████████████████▋ │ + 54. │ 2020-01-06 │ 5953 │ ███████████████████████████████████████████████████████████▌ │ + 55. │ 2020-01-13 │ 5698 │ ████████████████████████████████████████████████████████▊ │ + 56. │ 2020-01-20 │ 5339 │ █████████████████████████████████████████████████████▍ │ + 57. │ 2020-01-27 │ 5566 │ ███████████████████████████████████████████████████████▋ │ + 58. │ 2020-02-03 │ 5801 │ ██████████████████████████████████████████████████████████ │ + 59. │ 2020-02-10 │ 5692 │ ████████████████████████████████████████████████████████▊ │ + 60. │ 2020-02-17 │ 5912 │ ███████████████████████████████████████████████████████████ │ + 61. │ 2020-02-24 │ 6031 │ ████████████████████████████████████████████████████████████▎ │ + 62. │ 2020-03-02 │ 6105 │ █████████████████████████████████████████████████████████████ │ + 63. │ 2020-03-09 │ 5823 │ ██████████████████████████████████████████████████████████▏ │ + 64. │ 2020-03-16 │ 4659 │ ██████████████████████████████████████████████▌ │ + 65. │ 2020-03-23 │ 3720 │ █████████████████████████████████████▏ │ + 66. │ 2020-03-30 │ 1720 │ █████████████████▏ │ + 67. │ 2020-04-06 │ 849 │ ████████▍ │ + 68. │ 2020-04-13 │ 710 │ ███████ │ + 69. │ 2020-04-20 │ 725 │ ███████▏ │ + 70. │ 2020-04-27 │ 920 │ █████████▏ │ + 71. │ 2020-05-04 │ 859 │ ████████▌ │ + 72. │ 2020-05-11 │ 1047 │ ██████████▍ │ + 73. │ 2020-05-18 │ 1135 │ ███████████▎ │ + 74. │ 2020-05-25 │ 1266 │ ████████████▋ │ + 75. │ 2020-06-01 │ 1793 │ █████████████████▊ │ + 76. │ 2020-06-08 │ 1979 │ ███████████████████▋ │ + 77. │ 2020-06-15 │ 2297 │ ██████████████████████▊ │ + 78. │ 2020-06-22 │ 2788 │ ███████████████████████████▊ │ + 79. │ 2020-06-29 │ 3389 │ █████████████████████████████████▊ │ + 80. │ 2020-07-06 │ 3545 │ ███████████████████████████████████▍ │ + 81. │ 2020-07-13 │ 3569 │ ███████████████████████████████████▋ │ + 82. │ 2020-07-20 │ 3784 │ █████████████████████████████████████▋ │ + 83. │ 2020-07-27 │ 3960 │ ███████████████████████████████████████▌ │ + 84. │ 2020-08-03 │ 4323 │ ███████████████████████████████████████████▏ │ + 85. │ 2020-08-10 │ 4581 │ █████████████████████████████████████████████▋ │ + 86. │ 2020-08-17 │ 4791 │ ███████████████████████████████████████████████▊ │ + 87. │ 2020-08-24 │ 4928 │ █████████████████████████████████████████████████▎ │ + 88. │ 2020-08-31 │ 4687 │ ██████████████████████████████████████████████▋ │ + 89. │ 2020-09-07 │ 4643 │ ██████████████████████████████████████████████▍ │ + 90. │ 2020-09-14 │ 4594 │ █████████████████████████████████████████████▊ │ + 91. │ 2020-09-21 │ 4478 │ ████████████████████████████████████████████▋ │ + 92. │ 2020-09-28 │ 4382 │ ███████████████████████████████████████████▋ │ + 93. │ 2020-10-05 │ 4261 │ ██████████████████████████████████████████▌ │ + 94. │ 2020-10-12 │ 4243 │ ██████████████████████████████████████████▍ │ + 95. │ 2020-10-19 │ 3941 │ ███████████████████████████████████████▍ │ + 96. │ 2020-10-26 │ 3616 │ ████████████████████████████████████▏ │ + 97. │ 2020-11-02 │ 3586 │ ███████████████████████████████████▋ │ + 98. │ 2020-11-09 │ 3403 │ ██████████████████████████████████ │ + 99. │ 2020-11-16 │ 3336 │ █████████████████████████████████▎ │ +100. │ 2020-11-23 │ 3230 │ ████████████████████████████████▎ │ +101. │ 2020-11-30 │ 3183 │ ███████████████████████████████▋ │ +102. │ 2020-12-07 │ 3285 │ ████████████████████████████████▋ │ +103. │ 2020-12-14 │ 3367 │ █████████████████████████████████▋ │ +104. │ 2020-12-21 │ 3748 │ █████████████████████████████████████▍ │ +105. │ 2020-12-28 │ 3986 │ ███████████████████████████████████████▋ │ +106. │ 2021-01-04 │ 3906 │ ███████████████████████████████████████ │ +107. │ 2021-01-11 │ 3425 │ ██████████████████████████████████▎ │ +108. │ 2021-01-18 │ 3144 │ ███████████████████████████████▍ │ +109. │ 2021-01-25 │ 3115 │ ███████████████████████████████▏ │ +110. │ 2021-02-01 │ 3285 │ ████████████████████████████████▋ │ +111. │ 2021-02-08 │ 3321 │ █████████████████████████████████▏ │ +112. │ 2021-02-15 │ 3475 │ ██████████████████████████████████▋ │ +113. │ 2021-02-22 │ 3549 │ ███████████████████████████████████▍ │ +114. │ 2021-03-01 │ 3755 │ █████████████████████████████████████▌ │ +115. │ 2021-03-08 │ 3080 │ ██████████████████████████████▋ │ +116. │ 2021-03-15 │ 3789 │ █████████████████████████████████████▊ │ +117. │ 2021-03-22 │ 3804 │ ██████████████████████████████████████ │ +118. │ 2021-03-29 │ 4238 │ ██████████████████████████████████████████▍ │ +119. │ 2021-04-05 │ 4307 │ ███████████████████████████████████████████ │ +120. │ 2021-04-12 │ 4225 │ ██████████████████████████████████████████▎ │ +121. │ 2021-04-19 │ 4391 │ ███████████████████████████████████████████▊ │ +122. │ 2021-04-26 │ 4868 │ ████████████████████████████████████████████████▋ │ +123. │ 2021-05-03 │ 4977 │ █████████████████████████████████████████████████▋ │ +124. │ 2021-05-10 │ 5164 │ ███████████████████████████████████████████████████▋ │ +125. │ 2021-05-17 │ 4986 │ █████████████████████████████████████████████████▋ │ +126. │ 2021-05-24 │ 5024 │ ██████████████████████████████████████████████████▏ │ +127. │ 2021-05-31 │ 4824 │ ████████████████████████████████████████████████▏ │ +128. │ 2021-06-07 │ 5652 │ ████████████████████████████████████████████████████████▌ │ +129. │ 2021-06-14 │ 5613 │ ████████████████████████████████████████████████████████▏ │ +130. │ 2021-06-21 │ 6061 │ ████████████████████████████████████████████████████████████▌ │ +131. │ 2021-06-28 │ 2554 │ █████████████████████████▌ │ + └────────────┴──────┴──────────────────────────────────────────────────────────────────────────────┘ + +131 rows in set. Elapsed: 0.014 sec. Processed 655.36 thousand rows, 11.14 MB (47.56 million rows/s., 808.48 MB/s.) +``` + +### Test it in Playground + +The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICBvcmlnaW4sCiAgICBjb3VudCgpLAogICAgcm91bmQoYXZnKGdlb0Rpc3RhbmNlKGxvbmdpdHVkZV8xLCBsYXRpdHVkZV8xLCBsb25naXR1ZGVfMiwgbGF0aXR1ZGVfMikpKSBBUyBkaXN0YW5jZSwKICAgIGJhcihkaXN0YW5jZSwgMCwgMTAwMDAwMDAsIDEwMCkgQVMgYmFyCkZST00gb3BlbnNreQpXSEVSRSBvcmlnaW4gIT0gJycKR1JPVVAgQlkgb3JpZ2luCk9SREVSIEJZIGNvdW50KCkgREVTQwpMSU1JVCAxMDA=). diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md new file mode 100644 index 00000000000..1f8013ed64a --- /dev/null +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -0,0 +1,581 @@ +--- +toc_priority: 20 +toc_title: Данные о недвижимости в Великобритании +--- + +# UK Property Price Paid + +The dataset contains data about prices paid for real-estate property in England and Wales. The data is available since year 1995. +The size of the dataset in uncompressed form is about 4 GiB and it will take about 226 MiB in ClickHouse. + +Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads +Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data + +Contains HM Land Registry data © Crown copyright and database right 2021. This data is licensed under the Open Government Licence v3.0. + +## Download the Dataset + +``` +wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv +``` + +Download will take about 2 minutes with good internet connection. + +## Create the Table + +``` +CREATE TABLE uk_price_paid +( + price UInt32, + date Date, + postcode1 LowCardinality(String), + postcode2 LowCardinality(String), + type Enum8('terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4, 'other' = 0), + is_new UInt8, + duration Enum8('freehold' = 1, 'leasehold' = 2, 'unknown' = 0), + addr1 String, + addr2 String, + street LowCardinality(String), + locality LowCardinality(String), + town LowCardinality(String), + district LowCardinality(String), + county LowCardinality(String), + category UInt8 +) ENGINE = MergeTree ORDER BY (postcode1, postcode2, addr1, addr2); +``` + +## Preprocess and Import Data + +We will use `clickhouse-local` tool for data preprocessing and `clickhouse-client` to upload it. + +In this example, we define the structure of source data from the CSV file and specify a query to preprocess the data with `clickhouse-local`. + +The preprocessing is: +- splitting the postcode to two different columns `postcode1` and `postcode2` that is better for storage and queries; +- coverting the `time` field to date as it only contains 00:00 time; +- ignoring the `uuid` field because we don't need it for analysis; +- transforming `type` and `duration` to more readable Enum fields with function `transform`; +- transforming `is_new` and `category` fields from single-character string (`Y`/`N` and `A`/`B`) to UInt8 field with 0 and 1. + +Preprocessed data is piped directly to `clickhouse-client` to be inserted into ClickHouse table in streaming fashion. + +``` +clickhouse-local --input-format CSV --structure ' + uuid String, + price UInt32, + time DateTime, + postcode String, + a String, + b String, + c String, + addr1 String, + addr2 String, + street String, + locality String, + town String, + district String, + county String, + d String, + e String +' --query " + WITH splitByChar(' ', postcode) AS p + SELECT + price, + toDate(time) AS date, + p[1] AS postcode1, + p[2] AS postcode2, + transform(a, ['T', 'S', 'D', 'F', 'O'], ['terraced', 'semi-detached', 'detached', 'flat', 'other']) AS type, + b = 'Y' AS is_new, + transform(c, ['F', 'L', 'U'], ['freehold', 'leasehold', 'unknown']) AS duration, + addr1, + addr2, + street, + locality, + town, + district, + county, + d = 'B' AS category + FROM table" --date_time_input_format best_effort < pp-complete.csv | clickhouse-client --query "INSERT INTO uk_price_paid FORMAT TSV" +``` + +It will take about 40 seconds. + +## Validate the Data + +``` +SELECT count() FROM uk_price_paid +26248711 +``` + +The size of dataset in ClickHouse is just 226 MiB: + +``` +SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price_paid' +226.40 MiB +``` + +## Run Some Queries + +### Average price per year: + +``` +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, 80) FROM uk_price_paid GROUP BY year ORDER BY year + +┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ +│ 1995 │ 67932 │ █████▍ │ +│ 1996 │ 71505 │ █████▋ │ +│ 1997 │ 78532 │ ██████▎ │ +│ 1998 │ 85435 │ ██████▋ │ +│ 1999 │ 96036 │ ███████▋ │ +│ 2000 │ 107478 │ ████████▌ │ +│ 2001 │ 118886 │ █████████▌ │ +│ 2002 │ 137940 │ ███████████ │ +│ 2003 │ 155888 │ ████████████▍ │ +│ 2004 │ 178885 │ ██████████████▎ │ +│ 2005 │ 189350 │ ███████████████▏ │ +│ 2006 │ 203528 │ ████████████████▎ │ +│ 2007 │ 219377 │ █████████████████▌ │ +│ 2008 │ 217056 │ █████████████████▎ │ +│ 2009 │ 213419 │ █████████████████ │ +│ 2010 │ 236110 │ ██████████████████▊ │ +│ 2011 │ 232804 │ ██████████████████▌ │ +│ 2012 │ 238366 │ ███████████████████ │ +│ 2013 │ 256931 │ ████████████████████▌ │ +│ 2014 │ 279917 │ ██████████████████████▍ │ +│ 2015 │ 297264 │ ███████████████████████▋ │ +│ 2016 │ 313197 │ █████████████████████████ │ +│ 2017 │ 346070 │ ███████████████████████████▋ │ +│ 2018 │ 350117 │ ████████████████████████████ │ +│ 2019 │ 351010 │ ████████████████████████████ │ +│ 2020 │ 368974 │ █████████████████████████████▌ │ +│ 2021 │ 384351 │ ██████████████████████████████▋ │ +└──────┴────────┴────────────────────────────────────────┘ + +27 rows in set. Elapsed: 0.027 sec. Processed 26.25 million rows, 157.49 MB (955.96 million rows/s., 5.74 GB/s.) +``` + +### Average price per year in London: + +``` +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, 100) FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year + +┌─year─┬───price─┬─bar(round(avg(price)), 0, 2000000, 100)───────────────┐ +│ 1995 │ 109112 │ █████▍ │ +│ 1996 │ 118667 │ █████▊ │ +│ 1997 │ 136518 │ ██████▋ │ +│ 1998 │ 152983 │ ███████▋ │ +│ 1999 │ 180633 │ █████████ │ +│ 2000 │ 215830 │ ██████████▋ │ +│ 2001 │ 232996 │ ███████████▋ │ +│ 2002 │ 263672 │ █████████████▏ │ +│ 2003 │ 278394 │ █████████████▊ │ +│ 2004 │ 304665 │ ███████████████▏ │ +│ 2005 │ 322875 │ ████████████████▏ │ +│ 2006 │ 356192 │ █████████████████▋ │ +│ 2007 │ 404055 │ ████████████████████▏ │ +│ 2008 │ 420741 │ █████████████████████ │ +│ 2009 │ 427754 │ █████████████████████▍ │ +│ 2010 │ 480306 │ ████████████████████████ │ +│ 2011 │ 496274 │ ████████████████████████▋ │ +│ 2012 │ 519441 │ █████████████████████████▊ │ +│ 2013 │ 616209 │ ██████████████████████████████▋ │ +│ 2014 │ 724144 │ ████████████████████████████████████▏ │ +│ 2015 │ 792112 │ ███████████████████████████████████████▌ │ +│ 2016 │ 843568 │ ██████████████████████████████████████████▏ │ +│ 2017 │ 982566 │ █████████████████████████████████████████████████▏ │ +│ 2018 │ 1016845 │ ██████████████████████████████████████████████████▋ │ +│ 2019 │ 1043277 │ ████████████████████████████████████████████████████▏ │ +│ 2020 │ 1003963 │ ██████████████████████████████████████████████████▏ │ +│ 2021 │ 940794 │ ███████████████████████████████████████████████ │ +└──────┴─────────┴───────────────────────────────────────────────────────┘ + +27 rows in set. Elapsed: 0.024 sec. Processed 26.25 million rows, 76.88 MB (1.08 billion rows/s., 3.15 GB/s.) +``` + +Something happened in 2013. I don't have a clue. Maybe you have a clue what happened in 2020? + +### The most expensive neighborhoods: + +``` +SELECT + town, + district, + count() AS c, + round(avg(price)) AS price, + bar(price, 0, 5000000, 100) +FROM uk_price_paid +WHERE date >= '2020-01-01' +GROUP BY + town, + district +HAVING c >= 100 +ORDER BY price DESC +LIMIT 100 + +┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ +│ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ +│ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ +│ LONDON │ KENSINGTON AND CHELSEA │ 2367 │ 2342422 │ ██████████████████████████████████████████████▋ │ +│ LEATHERHEAD │ ELMBRIDGE │ 108 │ 1927143 │ ██████████████████████████████████████▌ │ +│ VIRGINIA WATER │ RUNNYMEDE │ 142 │ 1868819 │ █████████████████████████████████████▍ │ +│ LONDON │ CAMDEN │ 2815 │ 1736788 │ ██████████████████████████████████▋ │ +│ THORNTON HEATH │ CROYDON │ 521 │ 1733051 │ ██████████████████████████████████▋ │ +│ WINDLESHAM │ SURREY HEATH │ 103 │ 1717255 │ ██████████████████████████████████▎ │ +│ BARNET │ ENFIELD │ 115 │ 1503458 │ ██████████████████████████████ │ +│ OXFORD │ SOUTH OXFORDSHIRE │ 298 │ 1275200 │ █████████████████████████▌ │ +│ LONDON │ ISLINGTON │ 2458 │ 1274308 │ █████████████████████████▍ │ +│ COBHAM │ ELMBRIDGE │ 364 │ 1260005 │ █████████████████████████▏ │ +│ LONDON │ HOUNSLOW │ 618 │ 1215682 │ ████████████████████████▎ │ +│ ASCOT │ WINDSOR AND MAIDENHEAD │ 379 │ 1215146 │ ████████████████████████▎ │ +│ LONDON │ RICHMOND UPON THAMES │ 654 │ 1207551 │ ████████████████████████▏ │ +│ BEACONSFIELD │ BUCKINGHAMSHIRE │ 307 │ 1186220 │ ███████████████████████▋ │ +│ RICHMOND │ RICHMOND UPON THAMES │ 805 │ 1100420 │ ██████████████████████ │ +│ LONDON │ HAMMERSMITH AND FULHAM │ 2888 │ 1062959 │ █████████████████████▎ │ +│ WEYBRIDGE │ ELMBRIDGE │ 607 │ 1027161 │ ████████████████████▌ │ +│ RADLETT │ HERTSMERE │ 265 │ 1015896 │ ████████████████████▎ │ +│ SALCOMBE │ SOUTH HAMS │ 124 │ 1014393 │ ████████████████████▎ │ +│ BURFORD │ WEST OXFORDSHIRE │ 102 │ 993100 │ ███████████████████▋ │ +│ ESHER │ ELMBRIDGE │ 454 │ 969770 │ ███████████████████▍ │ +│ HINDHEAD │ WAVERLEY │ 128 │ 967786 │ ███████████████████▎ │ +│ BROCKENHURST │ NEW FOREST │ 121 │ 967046 │ ███████████████████▎ │ +│ LEATHERHEAD │ GUILDFORD │ 191 │ 964489 │ ███████████████████▎ │ +│ GERRARDS CROSS │ BUCKINGHAMSHIRE │ 376 │ 958555 │ ███████████████████▏ │ +│ EAST MOLESEY │ ELMBRIDGE │ 181 │ 943457 │ ██████████████████▋ │ +│ OLNEY │ MILTON KEYNES │ 220 │ 942892 │ ██████████████████▋ │ +│ CHALFONT ST GILES │ BUCKINGHAMSHIRE │ 135 │ 926950 │ ██████████████████▌ │ +│ HENLEY-ON-THAMES │ SOUTH OXFORDSHIRE │ 509 │ 905732 │ ██████████████████ │ +│ KINGSTON UPON THAMES │ KINGSTON UPON THAMES │ 889 │ 899689 │ █████████████████▊ │ +│ BELVEDERE │ BEXLEY │ 313 │ 895336 │ █████████████████▊ │ +│ CRANBROOK │ TUNBRIDGE WELLS │ 404 │ 888190 │ █████████████████▋ │ +│ LONDON │ EALING │ 2460 │ 865893 │ █████████████████▎ │ +│ MAIDENHEAD │ BUCKINGHAMSHIRE │ 114 │ 863814 │ █████████████████▎ │ +│ LONDON │ MERTON │ 1958 │ 857192 │ █████████████████▏ │ +│ GUILDFORD │ WAVERLEY │ 131 │ 854447 │ █████████████████ │ +│ LONDON │ HACKNEY │ 3088 │ 846571 │ ████████████████▊ │ +│ LYMM │ WARRINGTON │ 285 │ 839920 │ ████████████████▋ │ +│ HARPENDEN │ ST ALBANS │ 606 │ 836994 │ ████████████████▋ │ +│ LONDON │ WANDSWORTH │ 6113 │ 832292 │ ████████████████▋ │ +│ LONDON │ SOUTHWARK │ 3612 │ 831319 │ ████████████████▋ │ +│ BERKHAMSTED │ DACORUM │ 502 │ 830356 │ ████████████████▌ │ +│ KINGS LANGLEY │ DACORUM │ 137 │ 821358 │ ████████████████▍ │ +│ TONBRIDGE │ TUNBRIDGE WELLS │ 339 │ 806736 │ ████████████████▏ │ +│ EPSOM │ REIGATE AND BANSTEAD │ 157 │ 805903 │ ████████████████ │ +│ WOKING │ GUILDFORD │ 161 │ 803283 │ ████████████████ │ +│ STOCKBRIDGE │ TEST VALLEY │ 168 │ 801973 │ ████████████████ │ +│ TEDDINGTON │ RICHMOND UPON THAMES │ 539 │ 798591 │ ███████████████▊ │ +│ OXFORD │ VALE OF WHITE HORSE │ 329 │ 792907 │ ███████████████▋ │ +│ LONDON │ BARNET │ 3624 │ 789583 │ ███████████████▋ │ +│ TWICKENHAM │ RICHMOND UPON THAMES │ 1090 │ 787760 │ ███████████████▋ │ +│ LUTON │ CENTRAL BEDFORDSHIRE │ 196 │ 786051 │ ███████████████▋ │ +│ TONBRIDGE │ MAIDSTONE │ 277 │ 785746 │ ███████████████▋ │ +│ TOWCESTER │ WEST NORTHAMPTONSHIRE │ 186 │ 783532 │ ███████████████▋ │ +│ LONDON │ LAMBETH │ 4832 │ 783422 │ ███████████████▋ │ +│ LUTTERWORTH │ HARBOROUGH │ 515 │ 781775 │ ███████████████▋ │ +│ WOODSTOCK │ WEST OXFORDSHIRE │ 135 │ 777499 │ ███████████████▌ │ +│ ALRESFORD │ WINCHESTER │ 196 │ 775577 │ ███████████████▌ │ +│ LONDON │ NEWHAM │ 2942 │ 768551 │ ███████████████▎ │ +│ ALDERLEY EDGE │ CHESHIRE EAST │ 168 │ 768280 │ ███████████████▎ │ +│ MARLOW │ BUCKINGHAMSHIRE │ 301 │ 762784 │ ███████████████▎ │ +│ BILLINGSHURST │ CHICHESTER │ 134 │ 760920 │ ███████████████▏ │ +│ LONDON │ TOWER HAMLETS │ 4183 │ 759635 │ ███████████████▏ │ +│ MIDHURST │ CHICHESTER │ 245 │ 759101 │ ███████████████▏ │ +│ THAMES DITTON │ ELMBRIDGE │ 227 │ 753347 │ ███████████████ │ +│ POTTERS BAR │ WELWYN HATFIELD │ 163 │ 752926 │ ███████████████ │ +│ REIGATE │ REIGATE AND BANSTEAD │ 555 │ 740961 │ ██████████████▋ │ +│ TADWORTH │ REIGATE AND BANSTEAD │ 477 │ 738997 │ ██████████████▋ │ +│ SEVENOAKS │ SEVENOAKS │ 1074 │ 734658 │ ██████████████▋ │ +│ PETWORTH │ CHICHESTER │ 138 │ 732432 │ ██████████████▋ │ +│ BOURNE END │ BUCKINGHAMSHIRE │ 127 │ 730742 │ ██████████████▌ │ +│ PURLEY │ CROYDON │ 540 │ 727721 │ ██████████████▌ │ +│ OXTED │ TANDRIDGE │ 320 │ 726078 │ ██████████████▌ │ +│ LONDON │ HARINGEY │ 2988 │ 724573 │ ██████████████▍ │ +│ BANSTEAD │ REIGATE AND BANSTEAD │ 373 │ 713834 │ ██████████████▎ │ +│ PINNER │ HARROW │ 480 │ 712166 │ ██████████████▏ │ +│ MALMESBURY │ WILTSHIRE │ 293 │ 707747 │ ██████████████▏ │ +│ RICKMANSWORTH │ THREE RIVERS │ 732 │ 705400 │ ██████████████ │ +│ SLOUGH │ BUCKINGHAMSHIRE │ 359 │ 705002 │ ██████████████ │ +│ GREAT MISSENDEN │ BUCKINGHAMSHIRE │ 214 │ 704904 │ ██████████████ │ +│ READING │ SOUTH OXFORDSHIRE │ 295 │ 701697 │ ██████████████ │ +│ HYTHE │ FOLKESTONE AND HYTHE │ 457 │ 700334 │ ██████████████ │ +│ WELWYN │ WELWYN HATFIELD │ 217 │ 699649 │ █████████████▊ │ +│ CHIGWELL │ EPPING FOREST │ 242 │ 697869 │ █████████████▊ │ +│ BARNET │ BARNET │ 906 │ 695680 │ █████████████▊ │ +│ HASLEMERE │ CHICHESTER │ 120 │ 694028 │ █████████████▊ │ +│ LEATHERHEAD │ MOLE VALLEY │ 748 │ 692026 │ █████████████▋ │ +│ LONDON │ BRENT │ 1945 │ 690799 │ █████████████▋ │ +│ HASLEMERE │ WAVERLEY │ 258 │ 690765 │ █████████████▋ │ +│ NORTHWOOD │ HILLINGDON │ 252 │ 690753 │ █████████████▋ │ +│ WALTON-ON-THAMES │ ELMBRIDGE │ 871 │ 689431 │ █████████████▋ │ +│ INGATESTONE │ BRENTWOOD │ 150 │ 688345 │ █████████████▋ │ +│ OXFORD │ OXFORD │ 1761 │ 686114 │ █████████████▋ │ +│ CHISLEHURST │ BROMLEY │ 410 │ 682892 │ █████████████▋ │ +│ KINGS LANGLEY │ THREE RIVERS │ 109 │ 682320 │ █████████████▋ │ +│ ASHTEAD │ MOLE VALLEY │ 280 │ 680483 │ █████████████▌ │ +│ WOKING │ SURREY HEATH │ 269 │ 679035 │ █████████████▌ │ +│ ASCOT │ BRACKNELL FOREST │ 160 │ 678632 │ █████████████▌ │ +└──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ + +100 rows in set. Elapsed: 0.039 sec. Processed 26.25 million rows, 278.03 MB (674.32 million rows/s., 7.14 GB/s.) +``` + +### Test it in Playground + +The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). + +## Let's speed up queries using projections + +[Projections](https://../../sql-reference/statements/alter/projection/) allow to improve queries speed by storing pre-aggregated data. + +### Build a projection + +``` +-- create an aggregate projection by dimensions (toYear(date), district, town) + +ALTER TABLE uk_price_paid + ADD PROJECTION projection_by_year_district_town + ( + SELECT + toYear(date), + district, + town, + avg(price), + sum(price), + count() + GROUP BY + toYear(date), + district, + town + ); + +-- populate the projection for existing data (without it projection will be +-- created for only newly inserted data) + +ALTER TABLE uk_price_paid + MATERIALIZE PROJECTION projection_by_year_district_town +SETTINGS mutations_sync = 1; +``` + +## Test performance + +Let's run the same 3 queries. + +``` +-- enable projections for selects +set allow_experimental_projection_optimization=1; + +-- Q1) Average price per year: + +SELECT + toYear(date) AS year, + round(avg(price)) AS price, + bar(price, 0, 1000000, 80) +FROM uk_price_paid +GROUP BY year +ORDER BY year ASC; + +┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ +│ 1995 │ 67932 │ █████▍ │ +│ 1996 │ 71505 │ █████▋ │ +│ 1997 │ 78532 │ ██████▎ │ +│ 1998 │ 85435 │ ██████▋ │ +│ 1999 │ 96036 │ ███████▋ │ +│ 2000 │ 107478 │ ████████▌ │ +│ 2001 │ 118886 │ █████████▌ │ +│ 2002 │ 137940 │ ███████████ │ +│ 2003 │ 155888 │ ████████████▍ │ +│ 2004 │ 178885 │ ██████████████▎ │ +│ 2005 │ 189350 │ ███████████████▏ │ +│ 2006 │ 203528 │ ████████████████▎ │ +│ 2007 │ 219377 │ █████████████████▌ │ +│ 2008 │ 217056 │ █████████████████▎ │ +│ 2009 │ 213419 │ █████████████████ │ +│ 2010 │ 236110 │ ██████████████████▊ │ +│ 2011 │ 232804 │ ██████████████████▌ │ +│ 2012 │ 238366 │ ███████████████████ │ +│ 2013 │ 256931 │ ████████████████████▌ │ +│ 2014 │ 279917 │ ██████████████████████▍ │ +│ 2015 │ 297264 │ ███████████████████████▋ │ +│ 2016 │ 313197 │ █████████████████████████ │ +│ 2017 │ 346070 │ ███████████████████████████▋ │ +│ 2018 │ 350117 │ ████████████████████████████ │ +│ 2019 │ 351010 │ ████████████████████████████ │ +│ 2020 │ 368974 │ █████████████████████████████▌ │ +│ 2021 │ 384351 │ ██████████████████████████████▋ │ +└──────┴────────┴────────────────────────────────────────┘ + +27 rows in set. Elapsed: 0.003 sec. Processed 106.87 thousand rows, 3.21 MB (31.92 million rows/s., 959.03 MB/s.) + +-- Q2) Average price per year in London: + +SELECT + toYear(date) AS year, + round(avg(price)) AS price, + bar(price, 0, 2000000, 100) +FROM uk_price_paid +WHERE town = 'LONDON' +GROUP BY year +ORDER BY year ASC; + +┌─year─┬───price─┬─bar(round(avg(price)), 0, 2000000, 100)───────────────┐ +│ 1995 │ 109112 │ █████▍ │ +│ 1996 │ 118667 │ █████▊ │ +│ 1997 │ 136518 │ ██████▋ │ +│ 1998 │ 152983 │ ███████▋ │ +│ 1999 │ 180633 │ █████████ │ +│ 2000 │ 215830 │ ██████████▋ │ +│ 2001 │ 232996 │ ███████████▋ │ +│ 2002 │ 263672 │ █████████████▏ │ +│ 2003 │ 278394 │ █████████████▊ │ +│ 2004 │ 304665 │ ███████████████▏ │ +│ 2005 │ 322875 │ ████████████████▏ │ +│ 2006 │ 356192 │ █████████████████▋ │ +│ 2007 │ 404055 │ ████████████████████▏ │ +│ 2008 │ 420741 │ █████████████████████ │ +│ 2009 │ 427754 │ █████████████████████▍ │ +│ 2010 │ 480306 │ ████████████████████████ │ +│ 2011 │ 496274 │ ████████████████████████▋ │ +│ 2012 │ 519441 │ █████████████████████████▊ │ +│ 2013 │ 616209 │ ██████████████████████████████▋ │ +│ 2014 │ 724144 │ ████████████████████████████████████▏ │ +│ 2015 │ 792112 │ ███████████████████████████████████████▌ │ +│ 2016 │ 843568 │ ██████████████████████████████████████████▏ │ +│ 2017 │ 982566 │ █████████████████████████████████████████████████▏ │ +│ 2018 │ 1016845 │ ██████████████████████████████████████████████████▋ │ +│ 2019 │ 1043277 │ ████████████████████████████████████████████████████▏ │ +│ 2020 │ 1003963 │ ██████████████████████████████████████████████████▏ │ +│ 2021 │ 940794 │ ███████████████████████████████████████████████ │ +└──────┴─────────┴───────────────────────────────────────────────────────┘ + +27 rows in set. Elapsed: 0.005 sec. Processed 106.87 thousand rows, 3.53 MB (23.49 million rows/s., 775.95 MB/s.) + +-- Q3) The most expensive neighborhoods: +-- the condition (date >= '2020-01-01') needs to be modified to match projection dimension (toYear(date) >= 2020) + +SELECT + town, + district, + count() AS c, + round(avg(price)) AS price, + bar(price, 0, 5000000, 100) +FROM uk_price_paid +WHERE toYear(date) >= 2020 +GROUP BY + town, + district +HAVING c >= 100 +ORDER BY price DESC +LIMIT 100 + +┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ +│ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ +│ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ +│ LONDON │ KENSINGTON AND CHELSEA │ 2367 │ 2342422 │ ██████████████████████████████████████████████▋ │ +│ LEATHERHEAD │ ELMBRIDGE │ 108 │ 1927143 │ ██████████████████████████████████████▌ │ +│ VIRGINIA WATER │ RUNNYMEDE │ 142 │ 1868819 │ █████████████████████████████████████▍ │ +│ LONDON │ CAMDEN │ 2815 │ 1736788 │ ██████████████████████████████████▋ │ +│ THORNTON HEATH │ CROYDON │ 521 │ 1733051 │ ██████████████████████████████████▋ │ +│ WINDLESHAM │ SURREY HEATH │ 103 │ 1717255 │ ██████████████████████████████████▎ │ +│ BARNET │ ENFIELD │ 115 │ 1503458 │ ██████████████████████████████ │ +│ OXFORD │ SOUTH OXFORDSHIRE │ 298 │ 1275200 │ █████████████████████████▌ │ +│ LONDON │ ISLINGTON │ 2458 │ 1274308 │ █████████████████████████▍ │ +│ COBHAM │ ELMBRIDGE │ 364 │ 1260005 │ █████████████████████████▏ │ +│ LONDON │ HOUNSLOW │ 618 │ 1215682 │ ████████████████████████▎ │ +│ ASCOT │ WINDSOR AND MAIDENHEAD │ 379 │ 1215146 │ ████████████████████████▎ │ +│ LONDON │ RICHMOND UPON THAMES │ 654 │ 1207551 │ ████████████████████████▏ │ +│ BEACONSFIELD │ BUCKINGHAMSHIRE │ 307 │ 1186220 │ ███████████████████████▋ │ +│ RICHMOND │ RICHMOND UPON THAMES │ 805 │ 1100420 │ ██████████████████████ │ +│ LONDON │ HAMMERSMITH AND FULHAM │ 2888 │ 1062959 │ █████████████████████▎ │ +│ WEYBRIDGE │ ELMBRIDGE │ 607 │ 1027161 │ ████████████████████▌ │ +│ RADLETT │ HERTSMERE │ 265 │ 1015896 │ ████████████████████▎ │ +│ SALCOMBE │ SOUTH HAMS │ 124 │ 1014393 │ ████████████████████▎ │ +│ BURFORD │ WEST OXFORDSHIRE │ 102 │ 993100 │ ███████████████████▋ │ +│ ESHER │ ELMBRIDGE │ 454 │ 969770 │ ███████████████████▍ │ +│ HINDHEAD │ WAVERLEY │ 128 │ 967786 │ ███████████████████▎ │ +│ BROCKENHURST │ NEW FOREST │ 121 │ 967046 │ ███████████████████▎ │ +│ LEATHERHEAD │ GUILDFORD │ 191 │ 964489 │ ███████████████████▎ │ +│ GERRARDS CROSS │ BUCKINGHAMSHIRE │ 376 │ 958555 │ ███████████████████▏ │ +│ EAST MOLESEY │ ELMBRIDGE │ 181 │ 943457 │ ██████████████████▋ │ +│ OLNEY │ MILTON KEYNES │ 220 │ 942892 │ ██████████████████▋ │ +│ CHALFONT ST GILES │ BUCKINGHAMSHIRE │ 135 │ 926950 │ ██████████████████▌ │ +│ HENLEY-ON-THAMES │ SOUTH OXFORDSHIRE │ 509 │ 905732 │ ██████████████████ │ +│ KINGSTON UPON THAMES │ KINGSTON UPON THAMES │ 889 │ 899689 │ █████████████████▊ │ +│ BELVEDERE │ BEXLEY │ 313 │ 895336 │ █████████████████▊ │ +│ CRANBROOK │ TUNBRIDGE WELLS │ 404 │ 888190 │ █████████████████▋ │ +│ LONDON │ EALING │ 2460 │ 865893 │ █████████████████▎ │ +│ MAIDENHEAD │ BUCKINGHAMSHIRE │ 114 │ 863814 │ █████████████████▎ │ +│ LONDON │ MERTON │ 1958 │ 857192 │ █████████████████▏ │ +│ GUILDFORD │ WAVERLEY │ 131 │ 854447 │ █████████████████ │ +│ LONDON │ HACKNEY │ 3088 │ 846571 │ ████████████████▊ │ +│ LYMM │ WARRINGTON │ 285 │ 839920 │ ████████████████▋ │ +│ HARPENDEN │ ST ALBANS │ 606 │ 836994 │ ████████████████▋ │ +│ LONDON │ WANDSWORTH │ 6113 │ 832292 │ ████████████████▋ │ +│ LONDON │ SOUTHWARK │ 3612 │ 831319 │ ████████████████▋ │ +│ BERKHAMSTED │ DACORUM │ 502 │ 830356 │ ████████████████▌ │ +│ KINGS LANGLEY │ DACORUM │ 137 │ 821358 │ ████████████████▍ │ +│ TONBRIDGE │ TUNBRIDGE WELLS │ 339 │ 806736 │ ████████████████▏ │ +│ EPSOM │ REIGATE AND BANSTEAD │ 157 │ 805903 │ ████████████████ │ +│ WOKING │ GUILDFORD │ 161 │ 803283 │ ████████████████ │ +│ STOCKBRIDGE │ TEST VALLEY │ 168 │ 801973 │ ████████████████ │ +│ TEDDINGTON │ RICHMOND UPON THAMES │ 539 │ 798591 │ ███████████████▊ │ +│ OXFORD │ VALE OF WHITE HORSE │ 329 │ 792907 │ ███████████████▋ │ +│ LONDON │ BARNET │ 3624 │ 789583 │ ███████████████▋ │ +│ TWICKENHAM │ RICHMOND UPON THAMES │ 1090 │ 787760 │ ███████████████▋ │ +│ LUTON │ CENTRAL BEDFORDSHIRE │ 196 │ 786051 │ ███████████████▋ │ +│ TONBRIDGE │ MAIDSTONE │ 277 │ 785746 │ ███████████████▋ │ +│ TOWCESTER │ WEST NORTHAMPTONSHIRE │ 186 │ 783532 │ ███████████████▋ │ +│ LONDON │ LAMBETH │ 4832 │ 783422 │ ███████████████▋ │ +│ LUTTERWORTH │ HARBOROUGH │ 515 │ 781775 │ ███████████████▋ │ +│ WOODSTOCK │ WEST OXFORDSHIRE │ 135 │ 777499 │ ███████████████▌ │ +│ ALRESFORD │ WINCHESTER │ 196 │ 775577 │ ███████████████▌ │ +│ LONDON │ NEWHAM │ 2942 │ 768551 │ ███████████████▎ │ +│ ALDERLEY EDGE │ CHESHIRE EAST │ 168 │ 768280 │ ███████████████▎ │ +│ MARLOW │ BUCKINGHAMSHIRE │ 301 │ 762784 │ ███████████████▎ │ +│ BILLINGSHURST │ CHICHESTER │ 134 │ 760920 │ ███████████████▏ │ +│ LONDON │ TOWER HAMLETS │ 4183 │ 759635 │ ███████████████▏ │ +│ MIDHURST │ CHICHESTER │ 245 │ 759101 │ ███████████████▏ │ +│ THAMES DITTON │ ELMBRIDGE │ 227 │ 753347 │ ███████████████ │ +│ POTTERS BAR │ WELWYN HATFIELD │ 163 │ 752926 │ ███████████████ │ +│ REIGATE │ REIGATE AND BANSTEAD │ 555 │ 740961 │ ██████████████▋ │ +│ TADWORTH │ REIGATE AND BANSTEAD │ 477 │ 738997 │ ██████████████▋ │ +│ SEVENOAKS │ SEVENOAKS │ 1074 │ 734658 │ ██████████████▋ │ +│ PETWORTH │ CHICHESTER │ 138 │ 732432 │ ██████████████▋ │ +│ BOURNE END │ BUCKINGHAMSHIRE │ 127 │ 730742 │ ██████████████▌ │ +│ PURLEY │ CROYDON │ 540 │ 727721 │ ██████████████▌ │ +│ OXTED │ TANDRIDGE │ 320 │ 726078 │ ██████████████▌ │ +│ LONDON │ HARINGEY │ 2988 │ 724573 │ ██████████████▍ │ +│ BANSTEAD │ REIGATE AND BANSTEAD │ 373 │ 713834 │ ██████████████▎ │ +│ PINNER │ HARROW │ 480 │ 712166 │ ██████████████▏ │ +│ MALMESBURY │ WILTSHIRE │ 293 │ 707747 │ ██████████████▏ │ +│ RICKMANSWORTH │ THREE RIVERS │ 732 │ 705400 │ ██████████████ │ +│ SLOUGH │ BUCKINGHAMSHIRE │ 359 │ 705002 │ ██████████████ │ +│ GREAT MISSENDEN │ BUCKINGHAMSHIRE │ 214 │ 704904 │ ██████████████ │ +│ READING │ SOUTH OXFORDSHIRE │ 295 │ 701697 │ ██████████████ │ +│ HYTHE │ FOLKESTONE AND HYTHE │ 457 │ 700334 │ ██████████████ │ +│ WELWYN │ WELWYN HATFIELD │ 217 │ 699649 │ █████████████▊ │ +│ CHIGWELL │ EPPING FOREST │ 242 │ 697869 │ █████████████▊ │ +│ BARNET │ BARNET │ 906 │ 695680 │ █████████████▊ │ +│ HASLEMERE │ CHICHESTER │ 120 │ 694028 │ █████████████▊ │ +│ LEATHERHEAD │ MOLE VALLEY │ 748 │ 692026 │ █████████████▋ │ +│ LONDON │ BRENT │ 1945 │ 690799 │ █████████████▋ │ +│ HASLEMERE │ WAVERLEY │ 258 │ 690765 │ █████████████▋ │ +│ NORTHWOOD │ HILLINGDON │ 252 │ 690753 │ █████████████▋ │ +│ WALTON-ON-THAMES │ ELMBRIDGE │ 871 │ 689431 │ █████████████▋ │ +│ INGATESTONE │ BRENTWOOD │ 150 │ 688345 │ █████████████▋ │ +│ OXFORD │ OXFORD │ 1761 │ 686114 │ █████████████▋ │ +│ CHISLEHURST │ BROMLEY │ 410 │ 682892 │ █████████████▋ │ +│ KINGS LANGLEY │ THREE RIVERS │ 109 │ 682320 │ █████████████▋ │ +│ ASHTEAD │ MOLE VALLEY │ 280 │ 680483 │ █████████████▌ │ +│ WOKING │ SURREY HEATH │ 269 │ 679035 │ █████████████▌ │ +│ ASCOT │ BRACKNELL FOREST │ 160 │ 678632 │ █████████████▌ │ +└──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ + +100 rows in set. Elapsed: 0.005 sec. Processed 12.85 thousand rows, 813.40 KB (2.73 million rows/s., 172.95 MB/s.) +``` + +All 3 queries work much faster and read fewer rows. + +``` +Q1) +no projection: 27 rows in set. Elapsed: 0.027 sec. Processed 26.25 million rows, 157.49 MB (955.96 million rows/s., 5.74 GB/s.) + projection: 27 rows in set. Elapsed: 0.003 sec. Processed 106.87 thousand rows, 3.21 MB (31.92 million rows/s., 959.03 MB/s.) +``` From 4b46774b4c2910459e09ae0463edba8568a6a4e1 Mon Sep 17 00:00:00 2001 From: Roman Zhukov Date: Mon, 6 Sep 2021 18:17:31 +0000 Subject: [PATCH 010/142] Add touch drafts. --- .../example-datasets/opensky.md | 58 ++++++++--- .../example-datasets/uk-price-paid.md | 93 ++++++++++++++---- .../example-datasets/opensky.md | 58 ++++++++--- .../example-datasets/uk-price-paid.md | 95 ++++++++++++++----- 4 files changed, 235 insertions(+), 69 deletions(-) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md index c9f6729cdd7..f30d29e91df 100644 --- a/docs/en/getting-started/example-datasets/opensky.md +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -24,7 +24,7 @@ Download will take about 2 minutes with good internet connection. There are 30 f ## Create the Table -``` +```sql CREATE TABLE opensky ( callsign String, @@ -50,7 +50,7 @@ CREATE TABLE opensky Upload data into ClickHouse in parallel: -``` +```bash ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c ' gzip -c -d "{}" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"' ``` @@ -73,15 +73,29 @@ for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --dat ## Validate the Data -``` +Query: + +```sql SELECT count() FROM opensky +``` + +Result: + +```text 66010819 ``` The size of dataset in ClickHouse is just 2.64 GiB: -``` +Query: + +```sql SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' +``` + +Result: + +```text 2.64 GiB ``` @@ -89,26 +103,39 @@ SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' Total distance travelled is 68 billion kilometers: -``` +```sql SELECT formatReadableQuantity(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) / 1000) FROM opensky +``` +Result: + +```text ┌─formatReadableQuantity(divide(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)), 1000))─┐ │ 68.72 billion │ └──────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` Average flight distance is around 1000 km. -``` -SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM opensky +Query: + +```sql +SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM opensky +``` + +Result: + +```text ┌─avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2))─┐ │ 1041090.6465708319 │ └────────────────────────────────────────────────────────────────────┘ ``` -### Most busy origin airports and the average distance seen: +### Most busy origin airports and the average distance seen -``` +Query: + +```sql SELECT origin, count(), @@ -119,8 +146,11 @@ WHERE origin != '' GROUP BY origin ORDER BY count() DESC LIMIT 100 +``` -Query id: f9010ea5-97d0-45a3-a5bd-9657906cd105 +Result: + +```text ┌─origin─┬─count()─┬─distance─┬─bar────────────────────────────────────┐ 1. │ KORD │ 745007 │ 1546108 │ ███████████████▍ │ @@ -228,9 +258,9 @@ Query id: f9010ea5-97d0-45a3-a5bd-9657906cd105 100 rows in set. Elapsed: 0.186 sec. Processed 48.31 million rows, 2.17 GB (259.27 million rows/s., 11.67 GB/s.) ``` -### Number of flights from three major Moscow airports, weekly: +### Number of flights from three major Moscow airports, weekly -``` +```sql SELECT toMonday(day) AS k, count() AS c, @@ -239,9 +269,11 @@ FROM opensky WHERE origin IN ('UUEE', 'UUDD', 'UUWW') GROUP BY k ORDER BY k ASC +``` -Query id: 1b446157-9519-4cc4-a1cb-178dfcc15a8e +Result: +```text ┌──────────k─┬────c─┬─bar──────────────────────────────────────────────────────────────────────────┐ 1. │ 2018-12-31 │ 5248 │ ████████████████████████████████████████████████████▍ │ 2. │ 2019-01-07 │ 6302 │ ███████████████████████████████████████████████████████████████ │ diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 948ebd35b21..b7b1e5a7e5e 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -15,7 +15,7 @@ Contains HM Land Registry data © Crown copyright and database right 2021. This ## Download the Dataset -``` +```bash wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv ``` @@ -23,7 +23,7 @@ Download will take about 2 minutes with good internet connection. ## Create the Table -``` +```sql CREATE TABLE uk_price_paid ( price UInt32, @@ -59,7 +59,7 @@ The preprocessing is: Preprocessed data is piped directly to `clickhouse-client` to be inserted into ClickHouse table in streaming fashion. -``` +```bash clickhouse-local --input-format CSV --structure ' uuid String, price UInt32, @@ -102,15 +102,30 @@ It will take about 40 seconds. ## Validate the Data -``` +Query: + +```sql SELECT count() FROM uk_price_paid 26248711 ``` +Result: + +```text +26248711 +``` + The size of dataset in ClickHouse is just 226 MiB: -``` +Query: + +```sql SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price_paid' +``` + +Result: + +```text 226.40 MiB ``` @@ -118,9 +133,15 @@ SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price ### Average price per year: -``` -SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, 80) FROM uk_price_paid GROUP BY year ORDER BY year +Query: +```sql +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, 80) FROM uk_price_paid GROUP BY year ORDER BY year +``` + +Result: + +```text ┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ │ 1995 │ 67932 │ █████▍ │ │ 1996 │ 71505 │ █████▋ │ @@ -156,9 +177,13 @@ SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, ### Average price per year in London: -``` -SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, 100) FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year +Query: +```sql +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, 100) FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year +``` + +```text ┌─year─┬───price─┬─bar(round(avg(price)), 0, 2000000, 100)───────────────┐ │ 1995 │ 109112 │ █████▍ │ │ 1996 │ 118667 │ █████▊ │ @@ -196,7 +221,7 @@ Something happened in 2013. I don't have a clue. Maybe you have a clue what happ ### The most expensive neighborhoods: -``` +```sql SELECT town, district, @@ -211,7 +236,9 @@ GROUP BY HAVING c >= 100 ORDER BY price DESC LIMIT 100 +``` +```text ┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ │ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ │ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ @@ -328,9 +355,9 @@ The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhou ### Build a projection -``` --- create an aggregate projection by dimensions (toYear(date), district, town) +Create an aggregate projection by dimensions (toYear(date), district, town): +```sql ALTER TABLE uk_price_paid ADD PROJECTION projection_by_year_district_town ( @@ -346,10 +373,11 @@ ALTER TABLE uk_price_paid district, town ); +``` --- populate the projection for existing data (without it projection will be --- created for only newly inserted data) +Populate the projection for existing data (without it projection will be created for only newly inserted data) +```sql ALTER TABLE uk_price_paid MATERIALIZE PROJECTION projection_by_year_district_town SETTINGS mutations_sync = 1; @@ -357,14 +385,18 @@ SETTINGS mutations_sync = 1; ## Test performance -Let's run the same 3 queries. +Enable projections for selects and let's run the same 3 queries. -``` --- enable projections for selects +```sql set allow_experimental_projection_optimization=1; +``` --- Q1) Average price per year: +### Average price per year + +Query: + +```sql SELECT toYear(date) AS year, round(avg(price)) AS price, @@ -372,7 +404,11 @@ SELECT FROM uk_price_paid GROUP BY year ORDER BY year ASC; +``` +Result: + +```text ┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ │ 1995 │ 67932 │ █████▍ │ │ 1996 │ 71505 │ █████▋ │ @@ -404,9 +440,13 @@ ORDER BY year ASC; └──────┴────────┴────────────────────────────────────────┘ 27 rows in set. Elapsed: 0.003 sec. Processed 106.87 thousand rows, 3.21 MB (31.92 million rows/s., 959.03 MB/s.) +``` --- Q2) Average price per year in London: +### Average price per year in London +Query: + +```sql SELECT toYear(date) AS year, round(avg(price)) AS price, @@ -415,7 +455,11 @@ FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year ASC; +``` +Result: + +```text ┌─year─┬───price─┬─bar(round(avg(price)), 0, 2000000, 100)───────────────┐ │ 1995 │ 109112 │ █████▍ │ │ 1996 │ 118667 │ █████▊ │ @@ -447,10 +491,13 @@ ORDER BY year ASC; └──────┴─────────┴───────────────────────────────────────────────────────┘ 27 rows in set. Elapsed: 0.005 sec. Processed 106.87 thousand rows, 3.53 MB (23.49 million rows/s., 775.95 MB/s.) +``` --- Q3) The most expensive neighborhoods: --- the condition (date >= '2020-01-01') needs to be modified to match projection dimension (toYear(date) >= 2020) +### The most expensive neighborhoods: the condition (date >= '2020-01-01') needs to be modified to match projection dimension (toYear(date) >= 2020) +Query: + +```sql SELECT town, district, @@ -465,7 +512,11 @@ GROUP BY HAVING c >= 100 ORDER BY price DESC LIMIT 100 +``` +Result: + +```text ┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ │ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ │ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index c9f6729cdd7..f30d29e91df 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -24,7 +24,7 @@ Download will take about 2 minutes with good internet connection. There are 30 f ## Create the Table -``` +```sql CREATE TABLE opensky ( callsign String, @@ -50,7 +50,7 @@ CREATE TABLE opensky Upload data into ClickHouse in parallel: -``` +```bash ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c ' gzip -c -d "{}" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"' ``` @@ -73,15 +73,29 @@ for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --dat ## Validate the Data -``` +Query: + +```sql SELECT count() FROM opensky +``` + +Result: + +```text 66010819 ``` The size of dataset in ClickHouse is just 2.64 GiB: -``` +Query: + +```sql SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' +``` + +Result: + +```text 2.64 GiB ``` @@ -89,26 +103,39 @@ SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' Total distance travelled is 68 billion kilometers: -``` +```sql SELECT formatReadableQuantity(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) / 1000) FROM opensky +``` +Result: + +```text ┌─formatReadableQuantity(divide(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)), 1000))─┐ │ 68.72 billion │ └──────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` Average flight distance is around 1000 km. -``` -SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM opensky +Query: + +```sql +SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM opensky +``` + +Result: + +```text ┌─avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2))─┐ │ 1041090.6465708319 │ └────────────────────────────────────────────────────────────────────┘ ``` -### Most busy origin airports and the average distance seen: +### Most busy origin airports and the average distance seen -``` +Query: + +```sql SELECT origin, count(), @@ -119,8 +146,11 @@ WHERE origin != '' GROUP BY origin ORDER BY count() DESC LIMIT 100 +``` -Query id: f9010ea5-97d0-45a3-a5bd-9657906cd105 +Result: + +```text ┌─origin─┬─count()─┬─distance─┬─bar────────────────────────────────────┐ 1. │ KORD │ 745007 │ 1546108 │ ███████████████▍ │ @@ -228,9 +258,9 @@ Query id: f9010ea5-97d0-45a3-a5bd-9657906cd105 100 rows in set. Elapsed: 0.186 sec. Processed 48.31 million rows, 2.17 GB (259.27 million rows/s., 11.67 GB/s.) ``` -### Number of flights from three major Moscow airports, weekly: +### Number of flights from three major Moscow airports, weekly -``` +```sql SELECT toMonday(day) AS k, count() AS c, @@ -239,9 +269,11 @@ FROM opensky WHERE origin IN ('UUEE', 'UUDD', 'UUWW') GROUP BY k ORDER BY k ASC +``` -Query id: 1b446157-9519-4cc4-a1cb-178dfcc15a8e +Result: +```text ┌──────────k─┬────c─┬─bar──────────────────────────────────────────────────────────────────────────┐ 1. │ 2018-12-31 │ 5248 │ ████████████████████████████████████████████████████▍ │ 2. │ 2019-01-07 │ 6302 │ ███████████████████████████████████████████████████████████████ │ diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index 1f8013ed64a..b7b1e5a7e5e 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -1,6 +1,6 @@ --- toc_priority: 20 -toc_title: Данные о недвижимости в Великобритании +toc_title: UK Property Price Paid --- # UK Property Price Paid @@ -15,7 +15,7 @@ Contains HM Land Registry data © Crown copyright and database right 2021. This ## Download the Dataset -``` +```bash wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv ``` @@ -23,7 +23,7 @@ Download will take about 2 minutes with good internet connection. ## Create the Table -``` +```sql CREATE TABLE uk_price_paid ( price UInt32, @@ -59,7 +59,7 @@ The preprocessing is: Preprocessed data is piped directly to `clickhouse-client` to be inserted into ClickHouse table in streaming fashion. -``` +```bash clickhouse-local --input-format CSV --structure ' uuid String, price UInt32, @@ -102,15 +102,30 @@ It will take about 40 seconds. ## Validate the Data -``` +Query: + +```sql SELECT count() FROM uk_price_paid 26248711 ``` +Result: + +```text +26248711 +``` + The size of dataset in ClickHouse is just 226 MiB: -``` +Query: + +```sql SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price_paid' +``` + +Result: + +```text 226.40 MiB ``` @@ -118,9 +133,15 @@ SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price ### Average price per year: -``` -SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, 80) FROM uk_price_paid GROUP BY year ORDER BY year +Query: +```sql +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, 80) FROM uk_price_paid GROUP BY year ORDER BY year +``` + +Result: + +```text ┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ │ 1995 │ 67932 │ █████▍ │ │ 1996 │ 71505 │ █████▋ │ @@ -156,9 +177,13 @@ SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, ### Average price per year in London: -``` -SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, 100) FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year +Query: +```sql +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, 100) FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year +``` + +```text ┌─year─┬───price─┬─bar(round(avg(price)), 0, 2000000, 100)───────────────┐ │ 1995 │ 109112 │ █████▍ │ │ 1996 │ 118667 │ █████▊ │ @@ -196,7 +221,7 @@ Something happened in 2013. I don't have a clue. Maybe you have a clue what happ ### The most expensive neighborhoods: -``` +```sql SELECT town, district, @@ -211,7 +236,9 @@ GROUP BY HAVING c >= 100 ORDER BY price DESC LIMIT 100 +``` +```text ┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ │ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ │ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ @@ -328,9 +355,9 @@ The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhou ### Build a projection -``` --- create an aggregate projection by dimensions (toYear(date), district, town) +Create an aggregate projection by dimensions (toYear(date), district, town): +```sql ALTER TABLE uk_price_paid ADD PROJECTION projection_by_year_district_town ( @@ -346,10 +373,11 @@ ALTER TABLE uk_price_paid district, town ); +``` --- populate the projection for existing data (without it projection will be --- created for only newly inserted data) +Populate the projection for existing data (without it projection will be created for only newly inserted data) +```sql ALTER TABLE uk_price_paid MATERIALIZE PROJECTION projection_by_year_district_town SETTINGS mutations_sync = 1; @@ -357,14 +385,18 @@ SETTINGS mutations_sync = 1; ## Test performance -Let's run the same 3 queries. +Enable projections for selects and let's run the same 3 queries. -``` --- enable projections for selects +```sql set allow_experimental_projection_optimization=1; +``` --- Q1) Average price per year: +### Average price per year + +Query: + +```sql SELECT toYear(date) AS year, round(avg(price)) AS price, @@ -372,7 +404,11 @@ SELECT FROM uk_price_paid GROUP BY year ORDER BY year ASC; +``` +Result: + +```text ┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ │ 1995 │ 67932 │ █████▍ │ │ 1996 │ 71505 │ █████▋ │ @@ -404,9 +440,13 @@ ORDER BY year ASC; └──────┴────────┴────────────────────────────────────────┘ 27 rows in set. Elapsed: 0.003 sec. Processed 106.87 thousand rows, 3.21 MB (31.92 million rows/s., 959.03 MB/s.) +``` --- Q2) Average price per year in London: +### Average price per year in London +Query: + +```sql SELECT toYear(date) AS year, round(avg(price)) AS price, @@ -415,7 +455,11 @@ FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year ASC; +``` +Result: + +```text ┌─year─┬───price─┬─bar(round(avg(price)), 0, 2000000, 100)───────────────┐ │ 1995 │ 109112 │ █████▍ │ │ 1996 │ 118667 │ █████▊ │ @@ -447,10 +491,13 @@ ORDER BY year ASC; └──────┴─────────┴───────────────────────────────────────────────────────┘ 27 rows in set. Elapsed: 0.005 sec. Processed 106.87 thousand rows, 3.53 MB (23.49 million rows/s., 775.95 MB/s.) +``` --- Q3) The most expensive neighborhoods: --- the condition (date >= '2020-01-01') needs to be modified to match projection dimension (toYear(date) >= 2020) +### The most expensive neighborhoods: the condition (date >= '2020-01-01') needs to be modified to match projection dimension (toYear(date) >= 2020) +Query: + +```sql SELECT town, district, @@ -465,7 +512,11 @@ GROUP BY HAVING c >= 100 ORDER BY price DESC LIMIT 100 +``` +Result: + +```text ┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ │ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ │ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ From 1f8ffce8eba054e45930e9559de37cf523f45f16 Mon Sep 17 00:00:00 2001 From: Roman Zhukov Date: Mon, 6 Sep 2021 18:24:59 +0000 Subject: [PATCH 011/142] Add touch drafts. --- .../getting-started/example-datasets/opensky.md | 15 +++++++-------- .../example-datasets/uk-price-paid.md | 15 +++++++-------- .../getting-started/example-datasets/opensky.md | 15 +++++++-------- .../example-datasets/uk-price-paid.md | 15 +++++++-------- 4 files changed, 28 insertions(+), 32 deletions(-) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md index f30d29e91df..f7c708de93c 100644 --- a/docs/en/getting-started/example-datasets/opensky.md +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -67,7 +67,7 @@ Finally, `clickhouse-client` will do insertion. It will read input data in `CSVW Parallel upload takes 24 seconds. If you don't like parallel upload, here is sequential variant: -``` +```bash for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"; done ``` @@ -76,7 +76,7 @@ for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --dat Query: ```sql -SELECT count() FROM opensky +SELECT count() FROM opensky; ``` Result: @@ -90,7 +90,7 @@ The size of dataset in ClickHouse is just 2.64 GiB: Query: ```sql -SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' +SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky'; ``` Result: @@ -104,7 +104,7 @@ Result: Total distance travelled is 68 billion kilometers: ```sql -SELECT formatReadableQuantity(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) / 1000) FROM opensky +SELECT formatReadableQuantity(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) / 1000) FROM opensky; ``` Result: @@ -120,7 +120,7 @@ Average flight distance is around 1000 km. Query: ```sql -SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM opensky +SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM opensky; ``` Result: @@ -145,13 +145,12 @@ FROM opensky WHERE origin != '' GROUP BY origin ORDER BY count() DESC -LIMIT 100 +LIMIT 100; ``` Result: ```text - ┌─origin─┬─count()─┬─distance─┬─bar────────────────────────────────────┐ 1. │ KORD │ 745007 │ 1546108 │ ███████████████▍ │ 2. │ KDFW │ 696702 │ 1358721 │ █████████████▌ │ @@ -268,7 +267,7 @@ SELECT FROM opensky WHERE origin IN ('UUEE', 'UUDD', 'UUWW') GROUP BY k -ORDER BY k ASC +ORDER BY k ASC; ``` Result: diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index b7b1e5a7e5e..ad17fe87371 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -105,8 +105,7 @@ It will take about 40 seconds. Query: ```sql -SELECT count() FROM uk_price_paid -26248711 +SELECT count() FROM uk_price_paid; ``` Result: @@ -120,7 +119,7 @@ The size of dataset in ClickHouse is just 226 MiB: Query: ```sql -SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price_paid' +SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price_paid'; ``` Result: @@ -131,12 +130,12 @@ Result: ## Run Some Queries -### Average price per year: +### Average price per year Query: ```sql -SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, 80) FROM uk_price_paid GROUP BY year ORDER BY year +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, 80) FROM uk_price_paid GROUP BY year ORDER BY year; ``` Result: @@ -180,7 +179,7 @@ Result: Query: ```sql -SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, 100) FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, 100) FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year; ``` ```text @@ -235,7 +234,7 @@ GROUP BY district HAVING c >= 100 ORDER BY price DESC -LIMIT 100 +LIMIT 100; ``` ```text @@ -511,7 +510,7 @@ GROUP BY district HAVING c >= 100 ORDER BY price DESC -LIMIT 100 +LIMIT 100; ``` Result: diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index f30d29e91df..f7c708de93c 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -67,7 +67,7 @@ Finally, `clickhouse-client` will do insertion. It will read input data in `CSVW Parallel upload takes 24 seconds. If you don't like parallel upload, here is sequential variant: -``` +```bash for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"; done ``` @@ -76,7 +76,7 @@ for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --dat Query: ```sql -SELECT count() FROM opensky +SELECT count() FROM opensky; ``` Result: @@ -90,7 +90,7 @@ The size of dataset in ClickHouse is just 2.64 GiB: Query: ```sql -SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' +SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky'; ``` Result: @@ -104,7 +104,7 @@ Result: Total distance travelled is 68 billion kilometers: ```sql -SELECT formatReadableQuantity(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) / 1000) FROM opensky +SELECT formatReadableQuantity(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) / 1000) FROM opensky; ``` Result: @@ -120,7 +120,7 @@ Average flight distance is around 1000 km. Query: ```sql -SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM opensky +SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM opensky; ``` Result: @@ -145,13 +145,12 @@ FROM opensky WHERE origin != '' GROUP BY origin ORDER BY count() DESC -LIMIT 100 +LIMIT 100; ``` Result: ```text - ┌─origin─┬─count()─┬─distance─┬─bar────────────────────────────────────┐ 1. │ KORD │ 745007 │ 1546108 │ ███████████████▍ │ 2. │ KDFW │ 696702 │ 1358721 │ █████████████▌ │ @@ -268,7 +267,7 @@ SELECT FROM opensky WHERE origin IN ('UUEE', 'UUDD', 'UUWW') GROUP BY k -ORDER BY k ASC +ORDER BY k ASC; ``` Result: diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index b7b1e5a7e5e..ad17fe87371 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -105,8 +105,7 @@ It will take about 40 seconds. Query: ```sql -SELECT count() FROM uk_price_paid -26248711 +SELECT count() FROM uk_price_paid; ``` Result: @@ -120,7 +119,7 @@ The size of dataset in ClickHouse is just 226 MiB: Query: ```sql -SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price_paid' +SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price_paid'; ``` Result: @@ -131,12 +130,12 @@ Result: ## Run Some Queries -### Average price per year: +### Average price per year Query: ```sql -SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, 80) FROM uk_price_paid GROUP BY year ORDER BY year +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, 80) FROM uk_price_paid GROUP BY year ORDER BY year; ``` Result: @@ -180,7 +179,7 @@ Result: Query: ```sql -SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, 100) FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year +SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, 100) FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year; ``` ```text @@ -235,7 +234,7 @@ GROUP BY district HAVING c >= 100 ORDER BY price DESC -LIMIT 100 +LIMIT 100; ``` ```text @@ -511,7 +510,7 @@ GROUP BY district HAVING c >= 100 ORDER BY price DESC -LIMIT 100 +LIMIT 100; ``` Result: From f34cb097d8f340732cc78d1fd1bad7bf7bf888ea Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Sep 2021 23:13:06 +0300 Subject: [PATCH 012/142] Fix some tests. --- src/DataStreams/CountingBlockOutputStream.cpp | 12 +++- src/DataStreams/CountingBlockOutputStream.h | 4 +- .../PushingToViewsBlockOutputStream.cpp | 56 +++++++++++++++---- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- .../Executors/PushingPipelineExecutor.cpp | 7 ++- src/Processors/IAccumulatingTransform.cpp | 2 +- src/Processors/Sinks/SinkToStorage.cpp | 15 +++-- src/Processors/Sinks/SinkToStorage.h | 4 +- .../Transforms/SquashingChunksTransform.cpp | 1 + 9 files changed, 81 insertions(+), 22 deletions(-) diff --git a/src/DataStreams/CountingBlockOutputStream.cpp b/src/DataStreams/CountingBlockOutputStream.cpp index e17dc80ca13..823333866b3 100644 --- a/src/DataStreams/CountingBlockOutputStream.cpp +++ b/src/DataStreams/CountingBlockOutputStream.cpp @@ -17,8 +17,16 @@ void CountingTransform::transform(Chunk & chunk) Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); progress.incrementPiecewiseAtomically(local_progress); - ProfileEvents::increment(ProfileEvents::InsertedRows, local_progress.read_rows); - ProfileEvents::increment(ProfileEvents::InsertedBytes, local_progress.read_bytes); + if (thread_status) + { + thread_status->performance_counters.increment(ProfileEvents::InsertedRows, local_progress.read_rows); + thread_status->performance_counters.increment(ProfileEvents::InsertedBytes, local_progress.read_bytes); + } + else + { + ProfileEvents::increment(ProfileEvents::InsertedRows, local_progress.read_rows); + ProfileEvents::increment(ProfileEvents::InsertedBytes, local_progress.read_bytes); + } if (process_elem) process_elem->updateProgressOut(local_progress); diff --git a/src/DataStreams/CountingBlockOutputStream.h b/src/DataStreams/CountingBlockOutputStream.h index 702a9f190f8..9f7735b1589 100644 --- a/src/DataStreams/CountingBlockOutputStream.h +++ b/src/DataStreams/CountingBlockOutputStream.h @@ -12,7 +12,8 @@ namespace DB class CountingTransform final : public ISimpleTransform { public: - explicit CountingTransform(const Block & header) : ISimpleTransform(header, header, false) {} + explicit CountingTransform(const Block & header, ThreadStatus * thread_status_ = nullptr) + : ISimpleTransform(header, header, false), thread_status(thread_status_) {} String getName() const override { return "CountingTransform"; } @@ -37,6 +38,7 @@ protected: Progress progress; ProgressCallback progress_callback; QueryStatus * process_elem = nullptr; + ThreadStatus * thread_status = nullptr; }; } diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index fa91ba552fa..b447cb7d5f3 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -219,6 +219,8 @@ public: view.setException(std::move(status.exception)); } + else + view.runtime_stats.setStatus(QueryViewsLogElement::ViewStatus::QUERY_FINISH); } logQueryViews(views_data->views, views_data->context); @@ -233,6 +235,30 @@ private: std::exception_ptr any_exception; }; +class PushingToLiveViewSink final : public SinkToStorage +{ +public: + PushingToLiveViewSink(const Block & header, StorageLiveView & live_view_, StoragePtr storage_holder_, ContextPtr context_) + : SinkToStorage(header) + , live_view(live_view_) + , storage_holder(std::move(storage_holder_)) + , context(std::move(context_)) + { + } + + String getName() const override { return "PushingToLiveViewSink"; } + + void consume(Chunk chunk) override + { + StorageLiveView::writeIntoLiveView(live_view, getHeader().cloneWithColumns(chunk.detachColumns()), context); + } + +private: + StorageLiveView & live_view; + StoragePtr storage_holder; + ContextPtr context; +}; + Chain buildPushingToViewsDrain( const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot, @@ -246,8 +272,8 @@ Chain buildPushingToViewsDrain( /// If we don't write directly to the destination /// then expect that we're inserting with precalculated virtual columns - auto storage_header = no_destination ? metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals()) - : metadata_snapshot->getSampleBlock(); + auto storage_header = //no_destination ? metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals()) + metadata_snapshot->getSampleBlock(); /** TODO This is a very important line. At any insertion into the table one of streams should own lock. * Although now any insertion into the table is done via PushingToViewsBlockOutputStream, @@ -323,7 +349,7 @@ Chain buildPushingToViewsDrain( } auto view_runtime_data = std::make_shared( - std::move(thread_status), + thread_status.get(), database_table.getNameForLogs()); if (auto * materialized_view = dynamic_cast(dependent_table.get())) @@ -361,7 +387,7 @@ Chain buildPushingToViewsDrain( BlockIO io = interpreter.execute(); out = std::move(io.out); } - else if (const auto * live_view = dynamic_cast(dependent_table.get())) + else if (auto * live_view = dynamic_cast(dependent_table.get())) { type = QueryViewsLogElement::ViewType::LIVE; query = live_view->getInnerQuery(); // Used only to log in system.query_views_log @@ -390,11 +416,15 @@ Chain buildPushingToViewsDrain( nullptr, std::move(runtime_stats)}); - auto executing_inner_query = std::make_shared( - storage_header, views_data->views.back(), views_data->source_storage_id, views_data->source_metadata_snapshot, views_data->source_storage); - executing_inner_query->setRuntimeData(view_runtime_data); + //if (type == QueryViewsLogElement::ViewType::MATERIALIZED) + { + auto executing_inner_query = std::make_shared( + storage_header, views_data->views.back(), views_data->source_storage_id, views_data->source_metadata_snapshot, views_data->source_storage); + executing_inner_query->setRuntimeData(view_runtime_data); + + out.addSource(std::move(executing_inner_query)); + } - out.addSource(std::move(executing_inner_query)); chains.emplace_back(std::move(out)); /// Add the view to the query access info so it can appear in system.query_log @@ -436,12 +466,18 @@ Chain buildPushingToViewsDrain( result_chain = Chain(std::move(processors)); } + if (auto * live_view = dynamic_cast(storage.get())) + { + auto sink = std::make_shared(storage_header, *live_view, storage, context); + sink->setRuntimeData(runtime_data); + result_chain.addSource(std::move(sink)); + } /// Do not push to destination table if the flag is set - if (!no_destination) + else if (!no_destination) { auto sink = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), context); - sink->setRuntimeData(runtime_data); metadata_snapshot->check(sink->getHeader().getColumnsWithTypeAndName()); + sink->setRuntimeData(runtime_data); result_chain.addSource(std::move(sink)); } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index c873a0cde44..43eaf4c0234 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -324,7 +324,7 @@ BlockIO InterpreterInsertQuery::execute() table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0)); } - auto counting = std::make_shared(out.getInputHeader()); + auto counting = std::make_shared(out.getInputHeader(), runtime_data ? runtime_data->thread_status : nullptr); counting->setProcessListElement(getContext()->getProcessListElement()); out.addSource(std::move(counting)); diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index 61081a5dc05..c5a980ecdfa 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -18,7 +19,11 @@ public: String getName() const override { return "PullingOutputFormat"; } - void setData(Chunk chunk) { data = std::move(chunk); } + void setData(Chunk chunk) + { + need_data_flag = false; + data = std::move(chunk); + } protected: diff --git a/src/Processors/IAccumulatingTransform.cpp b/src/Processors/IAccumulatingTransform.cpp index 64bdbe2410f..35b45543151 100644 --- a/src/Processors/IAccumulatingTransform.cpp +++ b/src/Processors/IAccumulatingTransform.cpp @@ -1,5 +1,5 @@ #include - +#include namespace DB { diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index 61da1a38f21..ac54a848129 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -2,15 +2,16 @@ #include #include #include +#include #include namespace DB { ExceptionKeepingTransformRuntimeData::ExceptionKeepingTransformRuntimeData( - std::unique_ptr thread_status_, + ThreadStatus * thread_status_, std::string additional_exception_message_) - : thread_status(std::move(thread_status_)) + : thread_status(thread_status_) , additional_exception_message(std::move(additional_exception_message_)) { } @@ -85,7 +86,7 @@ static std::exception_ptr runStep(std::function step, ExceptionKeepingTr { /// Change thread context to store individual metrics. Once the work in done, go back to the original thread runtime_data->thread_status->resetPerformanceCountersLastUsage(); - current_thread = runtime_data->thread_status.get(); + current_thread = runtime_data->thread_status; } std::exception_ptr res; @@ -162,7 +163,13 @@ SinkToStorage::SinkToStorage(const Block & header) : ExceptionKeepingTransform(h void SinkToStorage::transform(Chunk & chunk) { - std::cerr << "--- sink to storage rows " << chunk.getNumRows() << std::endl; + /** Throw an exception if the sizes of arrays - elements of nested data structures doesn't match. + * We have to make this assertion before writing to table, because storage engine may assume that they have equal sizes. + * NOTE It'd better to do this check in serialization of nested structures (in place when this assumption is required), + * but currently we don't have methods for serialization of nested structures "as a whole". + */ + Nested::validateArraySizes(getHeader().cloneWithColumns(chunk.getColumns())); + consume(chunk.clone()); if (lastBlockIsDuplicate()) chunk.clear(); diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index 380696ed059..5a506daa63e 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -10,12 +10,12 @@ class ThreadStatus; struct ExceptionKeepingTransformRuntimeData { - std::unique_ptr thread_status = nullptr; + ThreadStatus * thread_status = nullptr; UInt64 elapsed_ms = 0; std::string additional_exception_message; ExceptionKeepingTransformRuntimeData( - std::unique_ptr thread_status_, + ThreadStatus * thread_status_, std::string additional_exception_message_); }; diff --git a/src/Processors/Transforms/SquashingChunksTransform.cpp b/src/Processors/Transforms/SquashingChunksTransform.cpp index 398ce9eb9fb..dfea6663c78 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.cpp +++ b/src/Processors/Transforms/SquashingChunksTransform.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { From 999a4fe83129427c2dab6808321848b5203247e9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 8 Sep 2021 21:29:38 +0300 Subject: [PATCH 013/142] Fix other tests. --- src/DataStreams/CountingBlockOutputStream.cpp | 3 ++ .../PushingToViewsBlockOutputStream.cpp | 22 +++++++----- .../PushingToViewsBlockOutputStream.h | 1 - src/Interpreters/ActionsVisitor.cpp | 2 ++ src/Interpreters/ExpressionAnalyzer.cpp | 1 + src/Interpreters/InterpreterInsertQuery.cpp | 7 ++-- src/Processors/Chain.h | 6 ++++ src/Processors/Pipe.cpp | 15 +++----- src/Processors/Pipe.h | 29 ++-------------- src/Processors/PipelineResourcesHolder.cpp | 25 ++++++++++++++ src/Processors/PipelineResourcesHolder.h | 34 +++++++++++++++++++ src/Processors/QueryPipeline.cpp | 6 ++++ src/Processors/QueryPipeline.h | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 1 + src/Processors/QueryPlan/ReadFromRemote.cpp | 1 + src/Processors/Sinks/SinkToStorage.cpp | 7 +++- src/Processors/Sinks/SinkToStorage.h | 4 +-- .../Transforms/SquashingChunksTransform.cpp | 20 ++++++++--- .../Transforms/SquashingChunksTransform.h | 12 ++++--- src/Storages/IStorage.cpp | 1 + src/Storages/StorageBuffer.cpp | 1 + src/Storages/StorageDistributed.cpp | 1 + src/Storages/StorageMaterializedView.cpp | 1 + src/Storages/StorageMergeTree.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 1 + src/Storages/StorageView.cpp | 1 + 26 files changed, 141 insertions(+), 64 deletions(-) create mode 100644 src/Processors/PipelineResourcesHolder.cpp create mode 100644 src/Processors/PipelineResourcesHolder.h diff --git a/src/DataStreams/CountingBlockOutputStream.cpp b/src/DataStreams/CountingBlockOutputStream.cpp index 823333866b3..edf03dc478d 100644 --- a/src/DataStreams/CountingBlockOutputStream.cpp +++ b/src/DataStreams/CountingBlockOutputStream.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace ProfileEvents @@ -17,6 +18,8 @@ void CountingTransform::transform(Chunk & chunk) Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); progress.incrementPiecewiseAtomically(local_progress); + std::cerr << "============ counting adding progress for " << static_cast(thread_status) << ' ' << chunk.getNumRows() << " rows\n"; + if (thread_status) { thread_status->performance_counters.increment(ProfileEvents::InsertedRows, local_progress.read_rows); diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index b447cb7d5f3..6ec6b2d1751 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -173,6 +173,7 @@ public: if (input.hasData()) { auto data = input.pullData(); + //std::cerr << "********** FinalizingViewsTransform got input " << i << " has exc " << bool(data.exception) << std::endl; if (data.exception) { if (views_data->has_exception && views_data->first_exception == data.exception) @@ -197,6 +198,9 @@ public: if (!statuses.empty()) return Status::Ready; + if (any_exception) + output.pushException(std::move(any_exception)); + output.finish(); return Status::Finished; } @@ -217,6 +221,7 @@ public: if (!any_exception) any_exception = status.exception; + //std::cerr << "=== Setting exception for " << view.table_id.getFullNameNotQuoted() << std::endl; view.setException(std::move(status.exception)); } else @@ -265,10 +270,10 @@ Chain buildPushingToViewsDrain( ContextPtr context, const ASTPtr & query_ptr, bool no_destination, - std::vector & locks, ExceptionKeepingTransformRuntimeDataPtr runtime_data) { checkStackSize(); + Chain result_chain; /// If we don't write directly to the destination /// then expect that we're inserting with precalculated virtual columns @@ -279,7 +284,7 @@ Chain buildPushingToViewsDrain( * Although now any insertion into the table is done via PushingToViewsBlockOutputStream, * but it's clear that here is not the best place for this functionality. */ - locks.emplace_back(storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout)); + result_chain.addTableLock(storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout)); /// If the "root" table deduplicates blocks, there are no need to make deduplication for children /// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks @@ -355,7 +360,7 @@ Chain buildPushingToViewsDrain( if (auto * materialized_view = dynamic_cast(dependent_table.get())) { type = QueryViewsLogElement::ViewType::MATERIALIZED; - locks.emplace_back(materialized_view->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout)); + result_chain.addTableLock(materialized_view->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout)); StoragePtr inner_table = materialized_view->getTargetTable(); auto inner_table_id = inner_table->getStorageID(); @@ -383,7 +388,7 @@ Chain buildPushingToViewsDrain( insert->columns = std::move(list); ASTPtr insert_query_ptr(insert.release()); - InterpreterInsertQuery interpreter(insert_query_ptr, insert_context, false, false, false, runtime_data); + InterpreterInsertQuery interpreter(insert_query_ptr, insert_context, false, false, false, view_runtime_data); BlockIO io = interpreter.execute(); out = std::move(io.out); } @@ -392,11 +397,11 @@ Chain buildPushingToViewsDrain( type = QueryViewsLogElement::ViewType::LIVE; query = live_view->getInnerQuery(); // Used only to log in system.query_views_log out = buildPushingToViewsDrain( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true, locks, view_runtime_data); + dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true, view_runtime_data); } else out = buildPushingToViewsDrain( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), false, locks, view_runtime_data); + dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), false, view_runtime_data); QueryViewsLogElement::ViewRuntimeStats runtime_stats{ target_name, @@ -435,8 +440,6 @@ Chain buildPushingToViewsDrain( } } - Chain result_chain; - size_t num_views = views_data->views.size(); if (num_views != 0) { @@ -454,6 +457,7 @@ Chain buildPushingToViewsDrain( for (auto & chain : chains) { + result_chain.attachResourcesFrom(chain); connect(*out, chain.getInputPort()); connect(chain.getOutputPort(), *in); ++in; @@ -581,6 +585,8 @@ static void logQueryViews(std::list & views, ContextPtr context try { + //std::cerr << "============ Logging for " << static_cast(view.runtime_stats.thread_status.get()) << ' ' << view.table_id.getNameForLogs() << "\n"; + if (view.runtime_stats.thread_status) view.runtime_stats.thread_status->logToQueryViewsLog(view); } diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index 94c87b63c5a..2bf1f79b556 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -51,7 +51,6 @@ Chain buildPushingToViewsDrain( ContextPtr context, const ASTPtr & query_ptr, bool no_destination, - std::vector & locks, ExceptionKeepingTransformRuntimeDataPtr runtime_data); diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 61e484ff6f1..347ec2ee968 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -26,6 +26,8 @@ #include #include +#include + #include #include #include diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 77598e69c00..056b7bfcc5c 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -56,6 +56,7 @@ #include #include +#include #include namespace DB diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 43eaf4c0234..530e9a9a8f6 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -51,7 +51,7 @@ InterpreterInsertQuery::InterpreterInsertQuery( , allow_materialized(allow_materialized_) , no_squash(no_squash_) , no_destination(no_destination_) - , runtime_data(runtime_data_) + , runtime_data(std::move(runtime_data_)) { checkStackSize(); } @@ -281,10 +281,7 @@ BlockIO InterpreterInsertQuery::execute() } else { - std::vector locks; - out = buildPushingToViewsDrain(table, metadata_snapshot, getContext(), query_ptr, no_destination, locks, runtime_data); - for (auto & lock : locks) - res.pipeline.addTableLock(std::move(lock)); + out = buildPushingToViewsDrain(table, metadata_snapshot, getContext(), query_ptr, no_destination, runtime_data); } /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. diff --git a/src/Processors/Chain.h b/src/Processors/Chain.h index fd3c72221b9..dd5cdf0fde9 100644 --- a/src/Processors/Chain.h +++ b/src/Processors/Chain.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -35,11 +36,16 @@ public: const std::list & getProcessors() const { return processors; } static std::list getProcessors(Chain chain) { return std::move(chain.processors); } + void addTableLock(TableLockHolder lock) { holder.table_locks.emplace_back(std::move(lock)); } + void attachResourcesFrom(Chain & other) { holder = std::move(other.holder); } + PipelineResourcesHolder detachResources() { return std::move(holder); } + private: /// -> source -> transform -> ... -> transform -> sink -> /// ^ -> -> -> -> ^ /// input port output port std::list processors; + PipelineResourcesHolder holder; }; } diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index d2aecd10f3f..55f000d5d7e 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include namespace DB @@ -98,18 +99,9 @@ static OutputPort * uniteTotals(const OutputPortRawPtrs & ports, const Block & h return totals_port; } -Pipe::Holder & Pipe::Holder::operator=(Holder && rhs) +void Pipe::addQueryPlan(std::unique_ptr plan) { - table_locks.insert(table_locks.end(), rhs.table_locks.begin(), rhs.table_locks.end()); - storage_holders.insert(storage_holders.end(), rhs.storage_holders.begin(), rhs.storage_holders.end()); - interpreter_context.insert(interpreter_context.end(), - rhs.interpreter_context.begin(), rhs.interpreter_context.end()); - for (auto & plan : rhs.query_plans) - query_plans.emplace_back(std::move(plan)); - - query_id_holder = std::move(rhs.query_id_holder); - - return *this; + holder.query_plans.emplace_back(std::move(plan)); } Pipe::Pipe(ProcessorPtr source, OutputPort * output, OutputPort * totals, OutputPort * extremes) @@ -689,6 +681,7 @@ void Pipe::addChains(std::vector chains) connect(*output_ports[i], chains[i].getInputPort()); output_ports[i] = &chains[i].getOutputPort(); + holder = chains[i].detachResources(); auto added_processors = Chain::getProcessors(std::move(chains[i])); for (auto & transform : added_processors) { diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 0b430631d71..bbcaa910872 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -1,12 +1,10 @@ #pragma once #include -#include -#include +#include #include #include #include -#include namespace DB { @@ -18,9 +16,6 @@ using Pipes = std::vector; class QueryPipeline; -class IStorage; -using StoragePtr = std::shared_ptr; - using OutputPortRawPtrs = std::vector; /// Pipe is a set of processors which represents the part of pipeline. @@ -118,29 +113,11 @@ public: void addStorageHolder(StoragePtr storage) { holder.storage_holders.emplace_back(std::move(storage)); } void addQueryIdHolder(std::shared_ptr query_id_holder) { holder.query_id_holder = std::move(query_id_holder); } /// For queries with nested interpreters (i.e. StorageDistributed) - void addQueryPlan(std::unique_ptr plan) { holder.query_plans.emplace_back(std::move(plan)); } + void addQueryPlan(std::unique_ptr plan); private: /// Destruction order: processors, header, locks, temporary storages, local contexts - - struct Holder - { - Holder() = default; - Holder(Holder &&) = default; - /// Custom mode assignment does not destroy data from lhs. It appends data from rhs to lhs. - Holder& operator=(Holder &&); - - /// Some processors may implicitly use Context or temporary Storage created by Interpreter. - /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, - /// because QueryPipeline is alive until query is finished. - std::vector> interpreter_context; - std::vector storage_holders; - std::vector table_locks; - std::vector> query_plans; - std::shared_ptr query_id_holder; - }; - - Holder holder; + PipelineResourcesHolder holder; /// Header is common for all output below. Block header; diff --git a/src/Processors/PipelineResourcesHolder.cpp b/src/Processors/PipelineResourcesHolder.cpp new file mode 100644 index 00000000000..9cb2ea301ad --- /dev/null +++ b/src/Processors/PipelineResourcesHolder.cpp @@ -0,0 +1,25 @@ +#include +#include + +namespace DB +{ + +PipelineResourcesHolder::PipelineResourcesHolder() = default; +PipelineResourcesHolder::PipelineResourcesHolder(PipelineResourcesHolder &&) = default; +PipelineResourcesHolder::~PipelineResourcesHolder() = default; + +PipelineResourcesHolder & PipelineResourcesHolder::operator=(PipelineResourcesHolder && rhs) +{ + table_locks.insert(table_locks.end(), rhs.table_locks.begin(), rhs.table_locks.end()); + storage_holders.insert(storage_holders.end(), rhs.storage_holders.begin(), rhs.storage_holders.end()); + interpreter_context.insert(interpreter_context.end(), + rhs.interpreter_context.begin(), rhs.interpreter_context.end()); + for (auto & plan : rhs.query_plans) + query_plans.emplace_back(std::move(plan)); + + query_id_holder = std::move(rhs.query_id_holder); + + return *this; +} + +} diff --git a/src/Processors/PipelineResourcesHolder.h b/src/Processors/PipelineResourcesHolder.h new file mode 100644 index 00000000000..fe8e65996d0 --- /dev/null +++ b/src/Processors/PipelineResourcesHolder.h @@ -0,0 +1,34 @@ +#pragma once +#include +#include + +namespace DB +{ + +class QueryPipeline; + +class IStorage; +using StoragePtr = std::shared_ptr; + +class QueryPlan; +class Context; + +struct PipelineResourcesHolder +{ + PipelineResourcesHolder(); + PipelineResourcesHolder(PipelineResourcesHolder &&); + ~PipelineResourcesHolder(); + /// Custom mode assignment does not destroy data from lhs. It appends data from rhs to lhs. + PipelineResourcesHolder& operator=(PipelineResourcesHolder &&); + + /// Some processors may implicitly use Context or temporary Storage created by Interpreter. + /// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here, + /// because QueryPipeline is alive until query is finished. + std::vector> interpreter_context; + std::vector storage_holders; + std::vector table_locks; + std::vector> query_plans; + std::shared_ptr query_id_holder; +}; + +} diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index b97193ff4d6..afba81bfee9 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -21,6 +21,7 @@ #include #include #include +#include namespace DB { @@ -29,6 +30,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +void QueryPipeline::addQueryPlan(std::unique_ptr plan) +{ + pipe.addQueryPlan(std::move(plan)); +} + void QueryPipeline::checkInitialized() { if (!initialized()) diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index e2c66a677c8..ebd9141e370 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -123,7 +123,7 @@ public: void addTableLock(TableLockHolder lock) { pipe.addTableLock(std::move(lock)); } void addInterpreterContext(std::shared_ptr context) { pipe.addInterpreterContext(std::move(context)); } void addStorageHolder(StoragePtr storage) { pipe.addStorageHolder(std::move(storage)); } - void addQueryPlan(std::unique_ptr plan) { pipe.addQueryPlan(std::move(plan)); } + void addQueryPlan(std::unique_ptr plan); void setLimits(const StreamLocalLimits & limits) { pipe.setLimits(limits); } void setLeafLimits(const SizeLimits & limits) { pipe.setLeafLimits(limits); } void setQuota(const std::shared_ptr & quota) { pipe.setQuota(quota); } diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 811e5885219..2f3287d178b 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 506ef795473..fce952eceb8 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index ac54a848129..c27f7e3488a 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -98,6 +98,7 @@ static std::exception_ptr runStep(std::function step, ExceptionKeepingTr } catch (Exception & exception) { + // std::cerr << "===== got exception " << getExceptionMessage(exception, false); if (runtime_data && !runtime_data->additional_exception_message.empty()) exception.addMessage(runtime_data->additional_exception_message); @@ -105,6 +106,7 @@ static std::exception_ptr runStep(std::function step, ExceptionKeepingTr } catch (...) { + // std::cerr << "===== got exception " << getExceptionMessage(std::current_exception(), false); res = std::current_exception(); } @@ -118,9 +120,10 @@ static std::exception_ptr runStep(std::function step, ExceptionKeepingTr return res; } - +\ void ExceptionKeepingTransform::work() { + // std::cerr << "============ Executing " << getName() << std::endl; if (!was_on_start_called) { was_on_start_called = true; @@ -138,6 +141,8 @@ void ExceptionKeepingTransform::work() if (auto exception = runStep([this] { transform(data.chunk); }, runtime_data.get())) { + // std::cerr << "===== got exception in " << getName() << std::endl; + // std::cerr << getExceptionMessage(exception, true) << std::endl; has_exception = true; data.chunk.clear(); data.exception = std::move(exception); diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index 5a506daa63e..25301c8465a 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -37,7 +37,7 @@ using ExceptionKeepingTransformRuntimeDataPtr = std::shared_ptr +#include #include namespace DB { -class SquashingChunksTransform : public IAccumulatingTransform +class SquashingChunksTransform : public ExceptionKeepingTransform { public: explicit SquashingChunksTransform( @@ -13,12 +13,16 @@ public: String getName() const override { return "SquashingTransform"; } + void work() override; + protected: - void consume(Chunk chunk) override; - Chunk generate() override; + void transform(Chunk & chunk) override; + void onFinish() override; + private: SquashingTransform squashing; + Chunk finish_chunk; }; } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 0e9f82a9802..21b2fb9af3d 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 57f16ac9b7d..aa8e4bab370 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index fcd0e255e5c..ca7f65446f9 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -53,6 +53,7 @@ #include #include +#include #include #include #include diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index f72f6fee180..ee399e661f8 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 32c2c76dd10..1e3219768f4 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 150a71a09e5..4d7f862d099 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -42,6 +42,7 @@ #include #include +#include #include #include diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 790b925f891..d30cb65640c 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -16,6 +16,7 @@ #include #include +#include #include #include #include From 7a5846c5f7c44c61df792c777013ddcc90836f0b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 8 Sep 2021 21:30:54 +0300 Subject: [PATCH 014/142] Comment debug output. --- src/DataStreams/CountingBlockOutputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataStreams/CountingBlockOutputStream.cpp b/src/DataStreams/CountingBlockOutputStream.cpp index edf03dc478d..398a9a31761 100644 --- a/src/DataStreams/CountingBlockOutputStream.cpp +++ b/src/DataStreams/CountingBlockOutputStream.cpp @@ -18,7 +18,7 @@ void CountingTransform::transform(Chunk & chunk) Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); progress.incrementPiecewiseAtomically(local_progress); - std::cerr << "============ counting adding progress for " << static_cast(thread_status) << ' ' << chunk.getNumRows() << " rows\n"; + //std::cerr << "============ counting adding progress for " << static_cast(thread_status) << ' ' << chunk.getNumRows() << " rows\n"; if (thread_status) { From aea1871755fcd3bfa31951e22fe00516cd427091 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Sep 2021 14:16:03 +0300 Subject: [PATCH 015/142] Fix some more tests. --- src/DataStreams/BlockIO.cpp | 2 +- src/DataStreams/SquashingTransform.cpp | 2 +- src/Processors/Chain.cpp | 6 ++++++ src/Processors/Chain.h | 2 ++ src/Processors/Sinks/SinkToStorage.cpp | 2 +- src/Processors/Transforms/SquashingChunksTransform.cpp | 7 +++++++ 6 files changed, 18 insertions(+), 3 deletions(-) diff --git a/src/DataStreams/BlockIO.cpp b/src/DataStreams/BlockIO.cpp index 79e1bd209a2..496bcf84877 100644 --- a/src/DataStreams/BlockIO.cpp +++ b/src/DataStreams/BlockIO.cpp @@ -38,7 +38,7 @@ void BlockIO::reset() */ /// TODO simplify it all - out = {}; + out.reset(); in.reset(); if (process_list_entry) process_list_entry->get().releaseQueryStreams(); diff --git a/src/DataStreams/SquashingTransform.cpp b/src/DataStreams/SquashingTransform.cpp index 1f6ca8a7306..ea99dc49780 100644 --- a/src/DataStreams/SquashingTransform.cpp +++ b/src/DataStreams/SquashingTransform.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB @@ -67,7 +68,6 @@ Block SquashingTransform::addImpl(ReferenceType input_block) } append(std::move(input_block)); - if (isEnoughSize(accumulated_block)) { Block to_return; diff --git a/src/Processors/Chain.cpp b/src/Processors/Chain.cpp index 2c863a9fc23..5e3b2e6a678 100644 --- a/src/Processors/Chain.cpp +++ b/src/Processors/Chain.cpp @@ -117,4 +117,10 @@ OutputPort & Chain::getOutputPort() const return processors.back()->getOutputs().front(); } +void Chain::reset() +{ + Chain to_remove = std::move(*this); + *this = Chain(); +} + } diff --git a/src/Processors/Chain.h b/src/Processors/Chain.h index dd5cdf0fde9..51825cb94a2 100644 --- a/src/Processors/Chain.h +++ b/src/Processors/Chain.h @@ -40,6 +40,8 @@ public: void attachResourcesFrom(Chain & other) { holder = std::move(other.holder); } PipelineResourcesHolder detachResources() { return std::move(holder); } + void reset(); + private: /// -> source -> transform -> ... -> transform -> sink -> /// ^ -> -> -> -> ^ diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index c27f7e3488a..b26c905c458 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -120,7 +120,7 @@ static std::exception_ptr runStep(std::function step, ExceptionKeepingTr return res; } -\ + void ExceptionKeepingTransform::work() { // std::cerr << "============ Executing " << getName() << std::endl; diff --git a/src/Processors/Transforms/SquashingChunksTransform.cpp b/src/Processors/Transforms/SquashingChunksTransform.cpp index f0670743f49..07f0772d9d7 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.cpp +++ b/src/Processors/Transforms/SquashingChunksTransform.cpp @@ -27,6 +27,13 @@ void SquashingChunksTransform::onFinish() void SquashingChunksTransform::work() { + if (has_exception) + { + data.chunk.clear(); + ready_input = false; + return; + } + ExceptionKeepingTransform::work(); if (finish_chunk) { From 5c220d62b32d7c0fe73ea60727b3cda51753f0d4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Sep 2021 15:46:06 +0300 Subject: [PATCH 016/142] Update SinkToStorage.cpp --- src/Processors/Sinks/SinkToStorage.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index b26c905c458..586b41eb918 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -71,7 +71,12 @@ IProcessor::Status ExceptionKeepingTransform::prepare() return Status::PortFull; } - ready_input = true; + if (has_exception) + /// In case of exception, just drop all other data. + /// If transform is stateful, it's state may be broken after exception from transform() + data.chunk.clear(); + else + ready_input = true; } return Status::Ready; From 47b96add9a821d0080402baf3c256e7dba3e935d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Sep 2021 16:59:15 +0300 Subject: [PATCH 017/142] Refactor some code. --- .../CheckConstraintsBlockOutputStream.cpp | 2 +- .../CheckConstraintsBlockOutputStream.h | 4 +- src/DataStreams/CountingBlockOutputStream.h | 6 +- src/Processors/Sinks/SinkToStorage.cpp | 167 +---------------- src/Processors/Sinks/SinkToStorage.h | 66 +------ .../Transforms/ExceptionKeepingTransform.cpp | 172 ++++++++++++++++++ .../Transforms/ExceptionKeepingTransform.h | 73 ++++++++ .../Transforms/ExpressionTransform.cpp | 2 +- .../Transforms/ExpressionTransform.h | 4 +- .../Transforms/SquashingChunksTransform.cpp | 2 +- 10 files changed, 257 insertions(+), 241 deletions(-) create mode 100644 src/Processors/Transforms/ExceptionKeepingTransform.cpp create mode 100644 src/Processors/Transforms/ExceptionKeepingTransform.h diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 9dace69d38f..875f26c03f4 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -27,7 +27,7 @@ CheckConstraintsTransform::CheckConstraintsTransform( const Block & header, const ConstraintsDescription & constraints_, ContextPtr context_) - : ISimpleTransform(header, header, false) + : ExceptionKeepingTransform(header, header) , table_id(table_id_) , constraints(constraints_) , expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())) diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.h b/src/DataStreams/CheckConstraintsBlockOutputStream.h index aa6d96aa5d3..13569bac0de 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.h +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -12,7 +12,7 @@ namespace DB * Otherwise just pass block to output unchanged. */ -class CheckConstraintsTransform final : public ISimpleTransform +class CheckConstraintsTransform final : public ExceptionKeepingTransform { public: CheckConstraintsTransform( diff --git a/src/DataStreams/CountingBlockOutputStream.h b/src/DataStreams/CountingBlockOutputStream.h index 9f7735b1589..9d0ccf11ace 100644 --- a/src/DataStreams/CountingBlockOutputStream.h +++ b/src/DataStreams/CountingBlockOutputStream.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include @@ -9,11 +9,11 @@ namespace DB /// Proxy class which counts number of written block, rows, bytes -class CountingTransform final : public ISimpleTransform +class CountingTransform final : public ExceptionKeepingTransform { public: explicit CountingTransform(const Block & header, ThreadStatus * thread_status_ = nullptr) - : ISimpleTransform(header, header, false), thread_status(thread_status_) {} + : ExceptionKeepingTransform(header, header), thread_status(thread_status_) {} String getName() const override { return "CountingTransform"; } diff --git a/src/Processors/Sinks/SinkToStorage.cpp b/src/Processors/Sinks/SinkToStorage.cpp index 586b41eb918..9ec0939f3a8 100644 --- a/src/Processors/Sinks/SinkToStorage.cpp +++ b/src/Processors/Sinks/SinkToStorage.cpp @@ -1,175 +1,10 @@ #include -#include -#include -#include #include -#include namespace DB { -ExceptionKeepingTransformRuntimeData::ExceptionKeepingTransformRuntimeData( - ThreadStatus * thread_status_, - std::string additional_exception_message_) - : thread_status(thread_status_) - , additional_exception_message(std::move(additional_exception_message_)) -{ -} - -ExceptionKeepingTransform::ExceptionKeepingTransform(const Block & in_header, const Block & out_header) - : IProcessor({in_header}, {out_header}) - , input(inputs.front()), output(outputs.front()) -{ -} - -IProcessor::Status ExceptionKeepingTransform::prepare() -{ - if (!was_on_start_called) - return Status::Ready; - - /// Check can output. - - if (output.isFinished()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Output port is finished for {}", getName()); - - if (!output.canPush()) - { - input.setNotNeeded(); - return Status::PortFull; - } - - /// Output if has data. - if (ready_output) - { - output.pushData(std::move(data)); - ready_output = false; - return Status::PortFull; - } - - if (!ready_input) - { - if (input.isFinished()) - { - if (!was_on_finish_called && !has_exception) - return Status::Ready; - - output.finish(); - return Status::Finished; - } - - input.setNeeded(); - - if (!input.hasData()) - return Status::NeedData; - - data = input.pullData(true); - - if (data.exception) - { - has_exception = true; - output.pushData(std::move(data)); - return Status::PortFull; - } - - if (has_exception) - /// In case of exception, just drop all other data. - /// If transform is stateful, it's state may be broken after exception from transform() - data.chunk.clear(); - else - ready_input = true; - } - - return Status::Ready; -} - -static std::exception_ptr runStep(std::function step, ExceptionKeepingTransformRuntimeData * runtime_data) -{ - auto * original_thread = current_thread; - SCOPE_EXIT({ current_thread = original_thread; }); - - if (runtime_data && runtime_data->thread_status) - { - /// Change thread context to store individual metrics. Once the work in done, go back to the original thread - runtime_data->thread_status->resetPerformanceCountersLastUsage(); - current_thread = runtime_data->thread_status; - } - - std::exception_ptr res; - Stopwatch watch; - - try - { - step(); - } - catch (Exception & exception) - { - // std::cerr << "===== got exception " << getExceptionMessage(exception, false); - if (runtime_data && !runtime_data->additional_exception_message.empty()) - exception.addMessage(runtime_data->additional_exception_message); - - res = std::current_exception(); - } - catch (...) - { - // std::cerr << "===== got exception " << getExceptionMessage(std::current_exception(), false); - res = std::current_exception(); - } - - if (runtime_data) - { - if (runtime_data->thread_status) - runtime_data->thread_status->updatePerformanceCounters(); - - runtime_data->elapsed_ms += watch.elapsedMilliseconds(); - } - - return res; -} - -void ExceptionKeepingTransform::work() -{ - // std::cerr << "============ Executing " << getName() << std::endl; - if (!was_on_start_called) - { - was_on_start_called = true; - - if (auto exception = runStep([this] { onStart(); }, runtime_data.get())) - { - has_exception = true; - ready_output = true; - data.exception = std::move(exception); - } - } - else if (ready_input) - { - ready_input = false; - - if (auto exception = runStep([this] { transform(data.chunk); }, runtime_data.get())) - { - // std::cerr << "===== got exception in " << getName() << std::endl; - // std::cerr << getExceptionMessage(exception, true) << std::endl; - has_exception = true; - data.chunk.clear(); - data.exception = std::move(exception); - } - - if (data.chunk || data.exception) - ready_output = true; - } - else if (!was_on_finish_called) - { - was_on_finish_called = true; - - if (auto exception = runStep([this] { onFinish(); }, runtime_data.get())) - { - has_exception = true; - ready_output = true; - data.exception = std::move(exception); - } - } -} - -SinkToStorage::SinkToStorage(const Block & header) : ExceptionKeepingTransform(header, header) {} +SinkToStorage::SinkToStorage(const Block & header) : ExceptionKeepingTransform(header, header, false) {} void SinkToStorage::transform(Chunk & chunk) { diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index 25301c8465a..90f16e55197 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -1,74 +1,10 @@ #pragma once -#include #include +#include namespace DB { -class ThreadStatus; - - -struct ExceptionKeepingTransformRuntimeData -{ - ThreadStatus * thread_status = nullptr; - UInt64 elapsed_ms = 0; - std::string additional_exception_message; - - ExceptionKeepingTransformRuntimeData( - ThreadStatus * thread_status_, - std::string additional_exception_message_); -}; - -using ExceptionKeepingTransformRuntimeDataPtr = std::shared_ptr; - -/// Has one input and one output. -/// Works similarly to ISimpleTransform, but with much care about exceptions. -/// -/// If input contain exception, this exception is pushed directly to output port. -/// If input contain data chunk, transform() is called for it. -/// When transform throws exception itself, data chunk is replaced by caught exception. -/// Transformed chunk or newly caught exception is pushed to output. -/// -/// There may be any number of exceptions read from input, transform keeps the order. -/// It is expected that output port won't be closed from the other side before all data is processed. -/// -/// Method onStart() is called before reading any data. -/// Method onFinish() is called after all data from input is processed, if no exception happened. -/// In case of exception, it is additionally pushed into pipeline. -class ExceptionKeepingTransform : public IProcessor -{ -protected: - InputPort & input; - OutputPort & output; - Port::Data data; - - bool ready_input = false; - bool ready_output = false; - bool has_exception = false; - bool was_on_start_called = false; - bool was_on_finish_called = false; - -//protected: - virtual void transform(Chunk & chunk) = 0; - virtual void onStart() {} - virtual void onFinish() {} - -public: - ExceptionKeepingTransform(const Block & in_header, const Block & out_header); - - Status prepare() override; - void work() override; - - InputPort & getInputPort() { return input; } - OutputPort & getOutputPort() { return output; } - - void setRuntimeData(ExceptionKeepingTransformRuntimeDataPtr runtime_data_) { runtime_data = std::move(runtime_data_); } - -private: - ExceptionKeepingTransformRuntimeDataPtr runtime_data; -}; - - /// Sink which is returned from Storage::read. class SinkToStorage : public ExceptionKeepingTransform { diff --git a/src/Processors/Transforms/ExceptionKeepingTransform.cpp b/src/Processors/Transforms/ExceptionKeepingTransform.cpp new file mode 100644 index 00000000000..16bc187d13d --- /dev/null +++ b/src/Processors/Transforms/ExceptionKeepingTransform.cpp @@ -0,0 +1,172 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +ExceptionKeepingTransformRuntimeData::ExceptionKeepingTransformRuntimeData( + ThreadStatus * thread_status_, + std::string additional_exception_message_) + : thread_status(thread_status_) + , additional_exception_message(std::move(additional_exception_message_)) +{ +} + +ExceptionKeepingTransform::ExceptionKeepingTransform(const Block & in_header, const Block & out_header, bool ignore_on_start_and_finish_) + : IProcessor({in_header}, {out_header}) + , input(inputs.front()), output(outputs.front()) + , ignore_on_start_and_finish(ignore_on_start_and_finish_) +{ +} + +IProcessor::Status ExceptionKeepingTransform::prepare() +{ + if (!ignore_on_start_and_finish && !was_on_start_called) + return Status::Ready; + + /// Check can output. + + if (output.isFinished()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Output port is finished for {}", getName()); + + if (!output.canPush()) + { + input.setNotNeeded(); + return Status::PortFull; + } + + /// Output if has data. + if (ready_output) + { + output.pushData(std::move(data)); + ready_output = false; + return Status::PortFull; + } + + if (!ready_input) + { + if (input.isFinished()) + { + if (!ignore_on_start_and_finish && !was_on_finish_called && !has_exception) + return Status::Ready; + + output.finish(); + return Status::Finished; + } + + input.setNeeded(); + + if (!input.hasData()) + return Status::NeedData; + + data = input.pullData(true); + + if (data.exception) + { + has_exception = true; + output.pushData(std::move(data)); + return Status::PortFull; + } + + if (has_exception) + /// In case of exception, just drop all other data. + /// If transform is stateful, it's state may be broken after exception from transform() + data.chunk.clear(); + else + ready_input = true; + } + + return Status::Ready; +} + +static std::exception_ptr runStep(std::function step, ExceptionKeepingTransformRuntimeData * runtime_data) +{ + auto * original_thread = current_thread; + SCOPE_EXIT({ current_thread = original_thread; }); + + if (runtime_data && runtime_data->thread_status) + { + /// Change thread context to store individual metrics. Once the work in done, go back to the original thread + runtime_data->thread_status->resetPerformanceCountersLastUsage(); + current_thread = runtime_data->thread_status; + } + + std::exception_ptr res; + Stopwatch watch; + + try + { + step(); + } + catch (Exception & exception) + { + // std::cerr << "===== got exception " << getExceptionMessage(exception, false); + if (runtime_data && !runtime_data->additional_exception_message.empty()) + exception.addMessage(runtime_data->additional_exception_message); + + res = std::current_exception(); + } + catch (...) + { + // std::cerr << "===== got exception " << getExceptionMessage(std::current_exception(), false); + res = std::current_exception(); + } + + if (runtime_data) + { + if (runtime_data->thread_status) + runtime_data->thread_status->updatePerformanceCounters(); + + runtime_data->elapsed_ms += watch.elapsedMilliseconds(); + } + + return res; +} + +void ExceptionKeepingTransform::work() +{ + // std::cerr << "============ Executing " << getName() << std::endl; + if (!ignore_on_start_and_finish && !was_on_start_called) + { + was_on_start_called = true; + + if (auto exception = runStep([this] { onStart(); }, runtime_data.get())) + { + has_exception = true; + ready_output = true; + data.exception = std::move(exception); + } + } + else if (ready_input) + { + ready_input = false; + + if (auto exception = runStep([this] { transform(data.chunk); }, runtime_data.get())) + { + // std::cerr << "===== got exception in " << getName() << std::endl; + // std::cerr << getExceptionMessage(exception, true) << std::endl; + has_exception = true; + data.chunk.clear(); + data.exception = std::move(exception); + } + + if (data.chunk || data.exception) + ready_output = true; + } + else if (!ignore_on_start_and_finish && !was_on_finish_called) + { + was_on_finish_called = true; + + if (auto exception = runStep([this] { onFinish(); }, runtime_data.get())) + { + has_exception = true; + ready_output = true; + data.exception = std::move(exception); + } + } +} + +} diff --git a/src/Processors/Transforms/ExceptionKeepingTransform.h b/src/Processors/Transforms/ExceptionKeepingTransform.h new file mode 100644 index 00000000000..943470b3f59 --- /dev/null +++ b/src/Processors/Transforms/ExceptionKeepingTransform.h @@ -0,0 +1,73 @@ +#pragma once +#include + +namespace DB +{ + + +class ThreadStatus; + + +struct ExceptionKeepingTransformRuntimeData +{ + ThreadStatus * thread_status = nullptr; + UInt64 elapsed_ms = 0; + std::string additional_exception_message; + + ExceptionKeepingTransformRuntimeData( + ThreadStatus * thread_status_, + std::string additional_exception_message_); +}; + +using ExceptionKeepingTransformRuntimeDataPtr = std::shared_ptr; + +/// Has one input and one output. +/// Works similarly to ISimpleTransform, but with much care about exceptions. +/// +/// If input contain exception, this exception is pushed directly to output port. +/// If input contain data chunk, transform() is called for it. +/// When transform throws exception itself, data chunk is replaced by caught exception. +/// Transformed chunk or newly caught exception is pushed to output. +/// +/// There may be any number of exceptions read from input, transform keeps the order. +/// It is expected that output port won't be closed from the other side before all data is processed. +/// +/// Method onStart() is called before reading any data. +/// Method onFinish() is called after all data from input is processed, if no exception happened. +/// In case of exception, it is additionally pushed into pipeline. +class ExceptionKeepingTransform : public IProcessor +{ +protected: + InputPort & input; + OutputPort & output; + Port::Data data; + + bool ready_input = false; + bool ready_output = false; + bool has_exception = false; + + const bool ignore_on_start_and_finish = true; + bool was_on_start_called = false; + bool was_on_finish_called = false; + +//protected: + virtual void transform(Chunk & chunk) = 0; + virtual void onStart() {} + virtual void onFinish() {} + +public: + ExceptionKeepingTransform(const Block & in_header, const Block & out_header, bool ignore_on_start_and_finish_ = true); + + Status prepare() override; + void work() override; + + InputPort & getInputPort() { return input; } + OutputPort & getOutputPort() { return output; } + + void setRuntimeData(ExceptionKeepingTransformRuntimeDataPtr runtime_data_) { runtime_data = std::move(runtime_data_); } + +private: + ExceptionKeepingTransformRuntimeDataPtr runtime_data; +}; + +} diff --git a/src/Processors/Transforms/ExpressionTransform.cpp b/src/Processors/Transforms/ExpressionTransform.cpp index efdff086efa..02b3c766f71 100644 --- a/src/Processors/Transforms/ExpressionTransform.cpp +++ b/src/Processors/Transforms/ExpressionTransform.cpp @@ -10,7 +10,7 @@ Block ExpressionTransform::transformHeader(Block header, const ActionsDAG & expr ExpressionTransform::ExpressionTransform(const Block & header_, ExpressionActionsPtr expression_) - : ISimpleTransform(header_, transformHeader(header_, expression_->getActionsDAG()), false) + : ExceptionKeepingTransform(header_, transformHeader(header_, expression_->getActionsDAG())) , expression(std::move(expression_)) { } diff --git a/src/Processors/Transforms/ExpressionTransform.h b/src/Processors/Transforms/ExpressionTransform.h index d4cc1c8f78a..0b3fde5147f 100644 --- a/src/Processors/Transforms/ExpressionTransform.h +++ b/src/Processors/Transforms/ExpressionTransform.h @@ -1,5 +1,5 @@ #pragma once -#include +#include namespace DB { @@ -14,7 +14,7 @@ class ActionsDAG; * For example: hits * 2 + 3, url LIKE '%yandex%' * The expression processes each row independently of the others. */ -class ExpressionTransform : public ISimpleTransform +class ExpressionTransform final : public ExceptionKeepingTransform { public: ExpressionTransform( diff --git a/src/Processors/Transforms/SquashingChunksTransform.cpp b/src/Processors/Transforms/SquashingChunksTransform.cpp index 07f0772d9d7..f5aef01463a 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.cpp +++ b/src/Processors/Transforms/SquashingChunksTransform.cpp @@ -6,7 +6,7 @@ namespace DB SquashingChunksTransform::SquashingChunksTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, bool reserve_memory) - : ExceptionKeepingTransform(header, header) + : ExceptionKeepingTransform(header, header, false) , squashing(min_block_size_rows, min_block_size_bytes, reserve_memory) { } From 50562da6df7a66b46c32cee9b123b8b1807d59a9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 9 Sep 2021 20:08:49 +0300 Subject: [PATCH 018/142] More refactoring. --- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- .../Transforms/ExpressionTransform.cpp | 18 +++++++++++++++++- .../Transforms/ExpressionTransform.h | 19 ++++++++++++++++++- 3 files changed, 36 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 530e9a9a8f6..1480e67e1fd 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -304,7 +304,7 @@ BlockIO InterpreterInsertQuery::execute() /// Actually we don't know structure of input blocks from query/table, /// because some clients break insertion protocol (columns != header) - out.addSource(std::make_shared(query_sample_block, adding_missing_defaults_actions)); + out.addSource(std::make_shared(query_sample_block, adding_missing_defaults_actions)); /// It's important to squash blocks as early as possible (before other transforms), /// because other transforms may work inefficient if block size is small. diff --git a/src/Processors/Transforms/ExpressionTransform.cpp b/src/Processors/Transforms/ExpressionTransform.cpp index 02b3c766f71..ca788f1dd9f 100644 --- a/src/Processors/Transforms/ExpressionTransform.cpp +++ b/src/Processors/Transforms/ExpressionTransform.cpp @@ -10,7 +10,7 @@ Block ExpressionTransform::transformHeader(Block header, const ActionsDAG & expr ExpressionTransform::ExpressionTransform(const Block & header_, ExpressionActionsPtr expression_) - : ExceptionKeepingTransform(header_, transformHeader(header_, expression_->getActionsDAG())) + : ISimpleTransform(header_, transformHeader(header_, expression_->getActionsDAG()), false) , expression(std::move(expression_)) { } @@ -25,4 +25,20 @@ void ExpressionTransform::transform(Chunk & chunk) chunk.setColumns(block.getColumns(), num_rows); } +ConvertingTransform::ConvertingTransform(const Block & header_, ExpressionActionsPtr expression_) + : ExceptionKeepingTransform(header_, ExpressionTransform::transformHeader(header_, expression_->getActionsDAG())) + , expression(std::move(expression_)) +{ +} + +void ConvertingTransform::transform(Chunk & chunk) +{ + size_t num_rows = chunk.getNumRows(); + auto block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); + + expression->execute(block, num_rows); + + chunk.setColumns(block.getColumns(), num_rows); +} + } diff --git a/src/Processors/Transforms/ExpressionTransform.h b/src/Processors/Transforms/ExpressionTransform.h index 0b3fde5147f..a76dc733e14 100644 --- a/src/Processors/Transforms/ExpressionTransform.h +++ b/src/Processors/Transforms/ExpressionTransform.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace DB { @@ -14,7 +15,7 @@ class ActionsDAG; * For example: hits * 2 + 3, url LIKE '%yandex%' * The expression processes each row independently of the others. */ -class ExpressionTransform final : public ExceptionKeepingTransform +class ExpressionTransform final : public ISimpleTransform { public: ExpressionTransform( @@ -32,4 +33,20 @@ private: ExpressionActionsPtr expression; }; +class ConvertingTransform final : public ExceptionKeepingTransform +{ +public: + ConvertingTransform( + const Block & header_, + ExpressionActionsPtr expression_); + + String getName() const override { return "ConvertingTransform"; } + +protected: + void transform(Chunk & chunk) override; + +private: + ExpressionActionsPtr expression; +}; + } From bc7c70f93fb05f16b9a41a1c65ba40a892e2b8dd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Sep 2021 12:03:45 +0300 Subject: [PATCH 019/142] Try to fix build. --- src/Interpreters/InterpreterInsertQuery.cpp | 1 - src/Interpreters/executeQuery.cpp | 1 - src/Processors/Sources/SinkToOutputStream.cpp | 25 ----------------- src/Processors/Sources/SinkToOutputStream.h | 27 ------------------- 4 files changed, 54 deletions(-) delete mode 100644 src/Processors/Sources/SinkToOutputStream.cpp delete mode 100644 src/Processors/Sources/SinkToOutputStream.h diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index d3896d373a3..aebf74f82b2 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 54cae085207..a16c6163188 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -54,7 +54,6 @@ #include #include #include -#include #include #include diff --git a/src/Processors/Sources/SinkToOutputStream.cpp b/src/Processors/Sources/SinkToOutputStream.cpp deleted file mode 100644 index f8a890a0d9d..00000000000 --- a/src/Processors/Sources/SinkToOutputStream.cpp +++ /dev/null @@ -1,25 +0,0 @@ -#include -#include - - -namespace DB -{ - -SinkToOutputStream::SinkToOutputStream(BlockOutputStreamPtr stream_) - : SinkToStorage(stream_->getHeader()) - , stream(std::move(stream_)) -{ - stream->writePrefix(); -} - -void SinkToOutputStream::consume(Chunk chunk) -{ - stream->write(getPort().getHeader().cloneWithColumns(chunk.detachColumns())); -} - -void SinkToOutputStream::onFinish() -{ - stream->writeSuffix(); -} - -} diff --git a/src/Processors/Sources/SinkToOutputStream.h b/src/Processors/Sources/SinkToOutputStream.h deleted file mode 100644 index 946a53b685e..00000000000 --- a/src/Processors/Sources/SinkToOutputStream.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -class IBlockOutputStream; -using BlockOutputStreamPtr = std::shared_ptr; - -/// Sink which writes data to IBlockOutputStream. -/// It's a temporary wrapper. -class SinkToOutputStream : public SinkToStorage -{ -public: - explicit SinkToOutputStream(BlockOutputStreamPtr stream); - - String getName() const override { return "SinkToOutputStream"; } - -protected: - void consume(Chunk chunk) override; - void onFinish() override; - -private: - BlockOutputStreamPtr stream; -}; - -} From 6aaf2d463647853dacb1be25b1f04971d83231aa Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 10 Sep 2021 12:33:43 +0300 Subject: [PATCH 020/142] Upload from local PC. --- .../example-datasets/opensky.md | 26 ++++---- .../example-datasets/opensky.md | 62 ++++++++++--------- 2 files changed, 46 insertions(+), 42 deletions(-) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md index f7c708de93c..ddf745dd986 100644 --- a/docs/en/getting-started/example-datasets/opensky.md +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -3,7 +3,7 @@ toc_priority: 20 toc_title: OpenSky --- -# Crowdsourced air traffic data from The OpenSky Network 2020 +# Crowdsourced air traffic data from The OpenSky Network 2020 "The data in this dataset is derived and cleaned from the full OpenSky dataset to illustrate the development of air traffic during the COVID-19 pandemic. It spans all flights seen by the network's more than 2500 members since 1 January 2019. More data will be periodically included in the dataset until the end of the COVID-19 pandemic". @@ -16,7 +16,9 @@ https://doi.org/10.5194/essd-13-357-2021 ## Download the Dataset -``` +Run the command: + +```bash wget -O- https://zenodo.org/record/5092942 | grep -oP 'https://zenodo.org/record/5092942/files/flightlist_\d+_\d+\.csv\.gz' | xargs wget ``` @@ -51,8 +53,7 @@ CREATE TABLE opensky Upload data into ClickHouse in parallel: ```bash -ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c ' - gzip -c -d "{}" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"' +ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c 'gzip -c -d "{}" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"' ``` Here we pass the list of files (`ls -1 flightlist_*.csv.gz`) to `xargs` for parallel processing. @@ -67,6 +68,7 @@ Finally, `clickhouse-client` will do insertion. It will read input data in `CSVW Parallel upload takes 24 seconds. If you don't like parallel upload, here is sequential variant: + ```bash for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"; done ``` @@ -85,7 +87,7 @@ Result: 66010819 ``` -The size of dataset in ClickHouse is just 2.64 GiB: +The size of dataset in ClickHouse is just 2.66 GiB, check it. Query: @@ -96,12 +98,14 @@ SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' Result: ```text -2.64 GiB +2.66 GiB ``` ## Run Some Queries -Total distance travelled is 68 billion kilometers: +Total distance travelled is 68 billion kilometers. + +Query: ```sql SELECT formatReadableQuantity(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) / 1000) FROM opensky; @@ -253,12 +257,12 @@ Result: 99. │ EDDT │ 115122 │ 941740 │ █████████▍ │ 100. │ EFHK │ 114860 │ 1629143 │ ████████████████▎ │ └────────┴─────────┴──────────┴────────────────────────────────────────┘ - -100 rows in set. Elapsed: 0.186 sec. Processed 48.31 million rows, 2.17 GB (259.27 million rows/s., 11.67 GB/s.) ``` ### Number of flights from three major Moscow airports, weekly +Query: + ```sql SELECT toMonday(day) AS k, @@ -406,10 +410,8 @@ Result: 130. │ 2021-06-21 │ 6061 │ ████████████████████████████████████████████████████████████▌ │ 131. │ 2021-06-28 │ 2554 │ █████████████████████████▌ │ └────────────┴──────┴──────────────────────────────────────────────────────────────────────────────┘ - -131 rows in set. Elapsed: 0.014 sec. Processed 655.36 thousand rows, 11.14 MB (47.56 million rows/s., 808.48 MB/s.) ``` -### Test it in Playground +### Online Playground The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICBvcmlnaW4sCiAgICBjb3VudCgpLAogICAgcm91bmQoYXZnKGdlb0Rpc3RhbmNlKGxvbmdpdHVkZV8xLCBsYXRpdHVkZV8xLCBsb25naXR1ZGVfMiwgbGF0aXR1ZGVfMikpKSBBUyBkaXN0YW5jZSwKICAgIGJhcihkaXN0YW5jZSwgMCwgMTAwMDAwMDAsIDEwMCkgQVMgYmFyCkZST00gb3BlbnNreQpXSEVSRSBvcmlnaW4gIT0gJycKR1JPVVAgQlkgb3JpZ2luCk9SREVSIEJZIGNvdW50KCkgREVTQwpMSU1JVCAxMDA=). diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index f7c708de93c..66cad531c80 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -3,26 +3,28 @@ toc_priority: 20 toc_title: OpenSky --- -# Crowdsourced air traffic data from The OpenSky Network 2020 +# Набор данных о воздушном движении из сети OpenSky Network 2020 -"The data in this dataset is derived and cleaned from the full OpenSky dataset to illustrate the development of air traffic during the COVID-19 pandemic. It spans all flights seen by the network's more than 2500 members since 1 January 2019. More data will be periodically included in the dataset until the end of the COVID-19 pandemic". +"Данные в этом наборе получены и отфильтрованы из полного набора данных OpenSky, чтобы проиллюстрировать развитие воздушного движения во время пандемии COVID-19. Набор включает в себя все рейсы, которые видели более 2500 участников сети с 1 января 2019 года. Дополнительные данные будут периодически включаться в набор данных до окончания пандемии COVID-19". -Source: https://zenodo.org/record/5092942#.YRBCyTpRXYd +Источник: https://zenodo.org/record/5092942#.YRBCyTpRXYd Martin Strohmeier, Xavier Olive, Jannis Lübbe, Matthias Schäfer, and Vincent Lenders "Crowdsourced air traffic data from the OpenSky Network 2019–2020" Earth System Science Data 13(2), 2021 https://doi.org/10.5194/essd-13-357-2021 -## Download the Dataset +## Загрузите набор данных -``` +Выполните команду: + +```bash wget -O- https://zenodo.org/record/5092942 | grep -oP 'https://zenodo.org/record/5092942/files/flightlist_\d+_\d+\.csv\.gz' | xargs wget ``` -Download will take about 2 minutes with good internet connection. There are 30 files with total size of 4.3 GB. +Загрузка займет около 2 минут при хорошем подключении к Интернету. Будут загружено 30 файлов, общим размером 4,3 ГБ. -## Create the Table +## Создайте таблицу ```sql CREATE TABLE opensky @@ -46,9 +48,9 @@ CREATE TABLE opensky ) ENGINE = MergeTree ORDER BY (origin, destination, callsign); ``` -## Import Data +## Импортируйте данные в ClickHouse -Upload data into ClickHouse in parallel: +Параллельно загружайте данные в ClickHouse: ```bash ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c ' @@ -71,43 +73,45 @@ If you don't like parallel upload, here is sequential variant: for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"; done ``` -## Validate the Data +## Проверьте загруженную информацию -Query: +Запрос: ```sql SELECT count() FROM opensky; ``` -Result: +Результат: ```text 66010819 ``` -The size of dataset in ClickHouse is just 2.64 GiB: +The size of dataset in ClickHouse is just 2.66 GiB, check it. -Query: +Запрос: ```sql SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky'; ``` -Result: +Результат: ```text -2.64 GiB +2.66 GiB ``` -## Run Some Queries +## Примеры -Total distance travelled is 68 billion kilometers: +Total distance travelled is 68 billion kilometers. + +Запрос: ```sql SELECT formatReadableQuantity(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) / 1000) FROM opensky; ``` -Result: +Результат: ```text ┌─formatReadableQuantity(divide(sum(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)), 1000))─┐ @@ -117,13 +121,13 @@ Result: Average flight distance is around 1000 km. -Query: +Запрос: ```sql SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM opensky; ``` -Result: +Результат: ```text ┌─avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2))─┐ @@ -133,7 +137,7 @@ Result: ### Most busy origin airports and the average distance seen -Query: +Запрос: ```sql SELECT @@ -148,7 +152,7 @@ ORDER BY count() DESC LIMIT 100; ``` -Result: +Результат: ```text ┌─origin─┬─count()─┬─distance─┬─bar────────────────────────────────────┐ @@ -253,12 +257,12 @@ Result: 99. │ EDDT │ 115122 │ 941740 │ █████████▍ │ 100. │ EFHK │ 114860 │ 1629143 │ ████████████████▎ │ └────────┴─────────┴──────────┴────────────────────────────────────────┘ - -100 rows in set. Elapsed: 0.186 sec. Processed 48.31 million rows, 2.17 GB (259.27 million rows/s., 11.67 GB/s.) ``` ### Number of flights from three major Moscow airports, weekly +Запрос: + ```sql SELECT toMonday(day) AS k, @@ -270,7 +274,7 @@ GROUP BY k ORDER BY k ASC; ``` -Result: +Результат: ```text ┌──────────k─┬────c─┬─bar──────────────────────────────────────────────────────────────────────────┐ @@ -406,10 +410,8 @@ Result: 130. │ 2021-06-21 │ 6061 │ ████████████████████████████████████████████████████████████▌ │ 131. │ 2021-06-28 │ 2554 │ █████████████████████████▌ │ └────────────┴──────┴──────────────────────────────────────────────────────────────────────────────┘ - -131 rows in set. Elapsed: 0.014 sec. Processed 655.36 thousand rows, 11.14 MB (47.56 million rows/s., 808.48 MB/s.) ``` -### Test it in Playground +### Online Playground -The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICBvcmlnaW4sCiAgICBjb3VudCgpLAogICAgcm91bmQoYXZnKGdlb0Rpc3RhbmNlKGxvbmdpdHVkZV8xLCBsYXRpdHVkZV8xLCBsb25naXR1ZGVfMiwgbGF0aXR1ZGVfMikpKSBBUyBkaXN0YW5jZSwKICAgIGJhcihkaXN0YW5jZSwgMCwgMTAwMDAwMDAsIDEwMCkgQVMgYmFyCkZST00gb3BlbnNreQpXSEVSRSBvcmlnaW4gIT0gJycKR1JPVVAgQlkgb3JpZ2luCk9SREVSIEJZIGNvdW50KCkgREVTQwpMSU1JVCAxMDA=). +Вы можете протестировать другие запросы к этому набору данным с помощью интерактивного ресурса [Online Playground](https://gh-api.clickhouse.tech/play?user=play). Например, [вот так](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICBvcmlnaW4sCiAgICBjb3VudCgpLAogICAgcm91bmQoYXZnKGdlb0Rpc3RhbmNlKGxvbmdpdHVkZV8xLCBsYXRpdHVkZV8xLCBsb25naXR1ZGVfMiwgbGF0aXR1ZGVfMikpKSBBUyBkaXN0YW5jZSwKICAgIGJhcihkaXN0YW5jZSwgMCwgMTAwMDAwMDAsIDEwMCkgQVMgYmFyCkZST00gb3BlbnNreQpXSEVSRSBvcmlnaW4gIT0gJycKR1JPVVAgQlkgb3JpZ2luCk9SREVSIEJZIGNvdW50KCkgREVTQwpMSU1JVCAxMDA=). Однако, обратите внимание, что здесь нельзя создавать временные таблицы. From 083b422e87936dca67f070495f3275555d86f8a1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Sep 2021 13:18:04 +0300 Subject: [PATCH 021/142] Try to fix build. --- src/Storages/tests/gtest_storage_log.cpp | 23 ++++++++++++++++------- 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index b3ceef7e697..27c97aa7eb5 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include #include @@ -16,7 +15,10 @@ #include #include -#include +#include +#include +#include +#include #include #if !defined(__clang__) @@ -100,9 +102,12 @@ std::string writeData(int rows, DB::StoragePtr & table, const DB::ContextPtr con block.insert(column); } - auto out = std::make_shared(table->write({}, metadata_snapshot, context)); - out->write(block); - out->writeSuffix(); + Chain chain; + chain.addSource(table->write({}, metadata_snapshot, context)); + + PushingPipelineExecutor executor(chain); + executor.push(block); + executor.finish(); return data; } @@ -122,7 +127,7 @@ std::string readData(DB::StoragePtr & table, const DB::ContextPtr context) QueryPipeline pipeline; pipeline.init(table->read(column_names, metadata_snapshot, query_info, context, stage, 8192, 1)); - BlockInputStreamPtr in = std::make_shared(std::move(pipeline)); + PullingPipelineExecutor executor(pipeline); Block sample; { @@ -137,8 +142,12 @@ std::string readData(DB::StoragePtr & table, const DB::ContextPtr context) WriteBufferFromOwnString out_buf; BlockOutputStreamPtr output = FormatFactory::instance().getOutputStream("Values", out_buf, sample, context); - copyData(*in, *output); + Block data; + output->writePrefix(); + while (executor.pull(data)) + output->write(data); + output->writeSuffix(); output->flush(); return out_buf.str(); From 7dc345fefa100658f301cbb42a9f9c6a5820ecf6 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 10 Sep 2021 14:06:21 +0300 Subject: [PATCH 022/142] Update opensky eng/ru Update opensky eng/ru --- .../example-datasets/opensky.md | 4 +-- .../example-datasets/opensky.md | 34 ++++++++++--------- 2 files changed, 20 insertions(+), 18 deletions(-) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md index ddf745dd986..b74395e120e 100644 --- a/docs/en/getting-started/example-datasets/opensky.md +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -5,7 +5,7 @@ toc_title: OpenSky # Crowdsourced air traffic data from The OpenSky Network 2020 -"The data in this dataset is derived and cleaned from the full OpenSky dataset to illustrate the development of air traffic during the COVID-19 pandemic. It spans all flights seen by the network's more than 2500 members since 1 January 2019. More data will be periodically included in the dataset until the end of the COVID-19 pandemic". +> The data in this dataset is derived and cleaned from the full OpenSky dataset to illustrate the development of air traffic during the COVID-19 pandemic. It spans all flights seen by the network's more than 2500 members since 1 January 2019. More data will be periodically included in the dataset until the end of the COVID-19 pandemic. Source: https://zenodo.org/record/5092942#.YRBCyTpRXYd @@ -63,7 +63,7 @@ For every file, `xargs` will run a script with `bash -c`. The script has substit The script will decompress the file (`gzip -c -d "{}"`) to standard output (`-c` parameter) and the output is redirected to `clickhouse-client`. -Finally, `clickhouse-client` will do insertion. It will read input data in `CSVWithNames` format. We also asked to parse DateTime fields with extended parser (`--date_time_input_format best_effort`) to recognize ISO-8601 format with timezone offsets. +Finally, `clickhouse-client` will do insertion. It will read input data in [CSVWithNames](../../interfaces/formats.md#csvwithnames) format. We also asked to parse DateTime fields with extended parser (`--date_time_input_format best_effort`) to recognize ISO-8601 format with timezone offsets. Parallel upload takes 24 seconds. diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index 66cad531c80..443337a6640 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -5,7 +5,7 @@ toc_title: OpenSky # Набор данных о воздушном движении из сети OpenSky Network 2020 -"Данные в этом наборе получены и отфильтрованы из полного набора данных OpenSky, чтобы проиллюстрировать развитие воздушного движения во время пандемии COVID-19. Набор включает в себя все рейсы, которые видели более 2500 участников сети с 1 января 2019 года. Дополнительные данные будут периодически включаться в набор данных до окончания пандемии COVID-19". +> Данные в этом наборе получены и отфильтрованы из полного набора данных OpenSky, чтобы проиллюстрировать развитие воздушного движения во время пандемии COVID-19. Набор включает в себя все рейсы, которые видели более 2500 участников сети с 1 января 2019 года. Дополнительные данные будут периодически включаться в набор данных до окончания пандемии COVID-19. Источник: https://zenodo.org/record/5092942#.YRBCyTpRXYd @@ -50,25 +50,27 @@ CREATE TABLE opensky ## Импортируйте данные в ClickHouse -Параллельно загружайте данные в ClickHouse: +Загрузите параллельными потоками данные в ClickHouse: ```bash -ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c ' - gzip -c -d "{}" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"' +ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c 'gzip -c -d "{}" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"' ``` -Here we pass the list of files (`ls -1 flightlist_*.csv.gz`) to `xargs` for parallel processing. -`xargs -P100` specifies to use up to 100 parallel workers but as we only have 30 files, the number of workers will be only 30. +Здесь мы передаем список файлов (`ls -1 flightlist_*.csv.gz`) в `xargs` для параллельной обработки. -For every file, `xargs` will run a script with `bash -c`. The script has substitution in form of `{}` and the `xargs` command will substitute the filename to it (we have asked it for xargs with `-I{}`). +`xargs -P100` указывает на использование до 100 параллельных обработчиков, но поскольку у нас всего 30 файлов, то количество обработчиков будет всего 30. -The script will decompress the file (`gzip -c -d "{}"`) to standard output (`-c` parameter) and the output is redirected to `clickhouse-client`. +Для каждого файла `xargs` будет запускать скрипт с `bash -c`. Сценарий имеет подстановку в виде ` {}`, а команда `xargs` заменяет имя файла (мы указали это для xargs с помощью `-I{}`). -Finally, `clickhouse-client` will do insertion. It will read input data in `CSVWithNames` format. We also asked to parse DateTime fields with extended parser (`--date_time_input_format best_effort`) to recognize ISO-8601 format with timezone offsets. +Скрипт распакует файл (`gzip -c -d "{}"`) в стандартный вывод (параметр`-c`) и перенаправит его в `clickhouse-client`. -Parallel upload takes 24 seconds. +В итоге: клиент clickhouse добавит данные в таблицу `opensky`. Входные данные импортируются в формате [CSVWithNames](../../interfaces/formats.md#csvwithnames). -If you don't like parallel upload, here is sequential variant: +Чтобы распознать формат ISO-8601 со смещениями часовых поясов в полях типа `DateTime`, указывается параметр парсера `--date_time_input_format best_effort`. + +Загрузка параллельными потоками займёт около 24 секунд. + +Также вы можете последовательный вариант загрузки: ```bash for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"; done ``` @@ -87,7 +89,7 @@ SELECT count() FROM opensky; 66010819 ``` -The size of dataset in ClickHouse is just 2.66 GiB, check it. +Размер набора данных в ClickHouse составляет всего 2,66 гигабайта, проверьте это. Запрос: @@ -103,7 +105,7 @@ SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' ## Примеры -Total distance travelled is 68 billion kilometers. +Общее пройденное расстояние составляет 68 миллиардов километров. Запрос: @@ -119,7 +121,7 @@ SELECT formatReadableQuantity(sum(geoDistance(longitude_1, latitude_1, longitude └──────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` -Average flight distance is around 1000 km. +Средняя дальность полета составляет около 1000 км. Запрос: @@ -135,7 +137,7 @@ SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM o └────────────────────────────────────────────────────────────────────┘ ``` -### Most busy origin airports and the average distance seen +### Наиболее загруженные аэропорты в указанных координатах и среднее пройденное расстояние Запрос: @@ -259,7 +261,7 @@ LIMIT 100; └────────┴─────────┴──────────┴────────────────────────────────────────┘ ``` -### Number of flights from three major Moscow airports, weekly +### Номера рейсов из трех крупных аэропортов Москвы, еженедельно Запрос: From 285cfb7fd5c8d6d4d4d797ba8355e883870e24c2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Sep 2021 14:42:52 +0300 Subject: [PATCH 023/142] Fix build and style. --- .../PushingToViewsBlockOutputStream.cpp | 5 + .../PushingAsyncPipelineExecutor.cpp | 228 ++++++++++++++++++ .../Executors/PushingAsyncPipelineExecutor.h | 62 +++++ .../Transforms/ExceptionKeepingTransform.cpp | 5 + 4 files changed, 300 insertions(+) create mode 100644 src/Processors/Executors/PushingAsyncPipelineExecutor.cpp create mode 100644 src/Processors/Executors/PushingAsyncPipelineExecutor.h diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index d2dfaccb86c..d07c50ed15b 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -25,6 +25,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + struct ViewsData { std::list views; diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp new file mode 100644 index 00000000000..677999879b6 --- /dev/null +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -0,0 +1,228 @@ +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +class PushingAsyncSource : public ISource +{ +public: + explicit PushingAsyncSource(const Block & header) + : ISource(header) + {} + + String getName() const override { return "PushingAsyncSource"; } + + bool setData(Chunk chunk) + { + std::unique_lock lock(mutex); + condvar.wait(lock, [this] { return !has_data || is_finished; }); + + if (is_finished) + return false; + + data.swap(chunk); + has_data = true; + condvar.notify_one(); + + return true; + } + + void finish() + { + is_finished = true; + condvar.notify_all(); + } + +protected: + + Chunk generate() override + { + std::unique_lock lock(mutex); + condvar.wait(lock, [this] { return has_data || is_finished; }); + + Chunk res; + + res.swap(data); + has_data = false; + condvar.notify_one(); + + return res; + } + +private: + Chunk data; + bool has_data = false; + std::atomic_bool is_finished = false; + std::mutex mutex; + std::condition_variable condvar; +}; + +struct PushingAsyncPipelineExecutor::Data +{ + PipelineExecutorPtr executor; + std::exception_ptr exception; + PushingAsyncSource * source = nullptr; + std::atomic_bool is_finished = false; + std::atomic_bool has_exception = false; + ThreadFromGlobalPool thread; + Poco::Event finish_event; + + ~Data() + { + if (thread.joinable()) + thread.join(); + } + + void rethrowExceptionIfHas() + { + if (has_exception) + { + has_exception = false; + std::rethrow_exception(std::move(exception)); + } + } +}; + +static void threadFunction(PushingAsyncPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads) +{ + setThreadName("QueryPipelineEx"); + + try + { + if (thread_group) + CurrentThread::attachTo(thread_group); + + SCOPE_EXIT_SAFE( + if (thread_group) + CurrentThread::detachQueryIfNotDetached(); + ); + + data.executor->execute(num_threads); + } + catch (...) + { + data.exception = std::current_exception(); + data.has_exception = true; + + /// Finish source in case of exception. Otherwise thread.join() may hung. + if (data.source) + data.source->finish(); + } + + data.is_finished = true; + data.finish_event.set(); +} + + +PushingAsyncPipelineExecutor::PushingAsyncPipelineExecutor(Chain & chain_, size_t num_threads_) + : num_threads(num_threads_) + , chain(chain_) +{ + pushing_source = std::make_shared(chain.getInputHeader()); + auto sink = std::make_shared(chain.getOutputHeader()); + connect(pushing_source->getPort(), chain.getInputPort()); + connect(chain.getOutputPort(), sink->getPort()); + + processors = std::make_unique(); + processors->reserve(chain.getProcessors().size() + 2); + for (const auto & processor : chain.getProcessors()) + processors->push_back(processor); + processors->push_back(pushing_source); + processors->push_back(std::move(sink)); +} + +PushingAsyncPipelineExecutor::~PushingAsyncPipelineExecutor() +{ + try + { + finish(); + } + catch (...) + { + tryLogCurrentException("PushingAsyncPipelineExecutor"); + } +} + +const Block & PushingAsyncPipelineExecutor::getHeader() const +{ + return pushing_source->getPort().getHeader(); +} + + +void PushingAsyncPipelineExecutor::start() +{ + if (started) + return; + + started = true; + + data = std::make_unique(); + data->executor = std::make_shared(*processors); + data->source = pushing_source.get(); + + auto func = [&, thread_group = CurrentThread::getGroup()]() + { + threadFunction(*data, thread_group, num_threads); + }; + + data->thread = ThreadFromGlobalPool(std::move(func)); +} + +void PushingAsyncPipelineExecutor::push(Chunk chunk) +{ + if (!started) + start(); + + bool is_pushed = pushing_source->setData(std::move(chunk)); + data->rethrowExceptionIfHas(); + + if (!is_pushed) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Pipeline for PushingPipelineExecutor was finished before all data was inserted"); +} + +void PushingAsyncPipelineExecutor::push(Block block) +{ + push(Chunk(block.getColumns(), block.rows())); +} + +void PushingAsyncPipelineExecutor::finish() +{ + if (finished) + return; + finished = true; + + pushing_source->finish(); + + /// Join thread here to wait for possible exception. + if (data && data->thread.joinable()) + data->thread.join(); + + /// Rethrow exception to not swallow it in destructor. + if (data) + data->rethrowExceptionIfHas(); +} + +void PushingAsyncPipelineExecutor::cancel() +{ + /// Cancel execution if it wasn't finished. + if (data && !data->is_finished && data->executor) + data->executor->cancel(); + + finish(); +} + +} diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.h b/src/Processors/Executors/PushingAsyncPipelineExecutor.h new file mode 100644 index 00000000000..69271afe389 --- /dev/null +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.h @@ -0,0 +1,62 @@ +#pragma once +#include +#include + +namespace DB +{ + +class Block; +class Chunk; +class Chain; +class PushingAsyncSource; + +class PipelineExecutor; +using PipelineExecutorPtr = std::shared_ptr; + +class IProcessor; +using ProcessorPtr = std::shared_ptr; +using Processors = std::vector; + +/// Pushing executor for Chain of processors using several background threads. +/// Typical usage is: +/// +/// PushingAsyncPipelineExecutor executor(chain); +/// executor.start(); +/// while (auto chunk = ...) +/// executor.push(std::move(chunk)); +/// executor.finish(); +class PushingAsyncPipelineExecutor +{ +public: + explicit PushingAsyncPipelineExecutor(Chain & chain, size_t num_threads_); + ~PushingAsyncPipelineExecutor(); + + /// Get structure of returned block or chunk. + const Block & getHeader() const; + + void start(); + + void push(Chunk chunk); + void push(Block block); + + void finish(); + + /// Stop execution. It is not necessary, but helps to stop execution before executor is destroyed. + void cancel(); + + struct Data; + +private: + size_t num_threads; + + Chain & chain; + std::shared_ptr pushing_source; + + std::unique_ptr processors; + bool started = false; + bool finished = false; + + std::unique_ptr data; +}; + +} diff --git a/src/Processors/Transforms/ExceptionKeepingTransform.cpp b/src/Processors/Transforms/ExceptionKeepingTransform.cpp index 16bc187d13d..205f11de574 100644 --- a/src/Processors/Transforms/ExceptionKeepingTransform.cpp +++ b/src/Processors/Transforms/ExceptionKeepingTransform.cpp @@ -7,6 +7,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + ExceptionKeepingTransformRuntimeData::ExceptionKeepingTransformRuntimeData( ThreadStatus * thread_status_, std::string additional_exception_message_) From e6938127a642eb22a509543f9081955a73127e18 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Sep 2021 15:38:30 +0300 Subject: [PATCH 024/142] Fix some resouces. --- src/DataStreams/CheckConstraintsBlockOutputStream.cpp | 4 ++-- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 7 ++++++- src/Interpreters/InterpreterInsertQuery.cpp | 6 ++++++ src/Processors/Chain.h | 6 +++++- src/Processors/Executors/PullingPipelineExecutor.cpp | 5 +++++ .../Executors/PushingAsyncPipelineExecutor.cpp | 6 ++---- .../Executors/PushingAsyncPipelineExecutor.h | 4 +--- src/Processors/Executors/PushingPipelineExecutor.cpp | 4 ++-- src/Processors/Pipe.cpp | 10 ++++++++++ src/Processors/Pipe.h | 2 ++ 10 files changed, 41 insertions(+), 13 deletions(-) diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 875f26c03f4..f133c33e302 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -63,8 +63,8 @@ void CheckConstraintsTransform::transform(Chunk & chunk) /// Check if constraint value is nullable const auto & null_map = column_nullable->getNullMapColumn(); - const PaddedPODArray & data = null_map.getData(); - bool null_map_contains_null = !memoryIsZero(data.raw_data(), data.size() * sizeof(UInt8)); + const PaddedPODArray & null_map_data = null_map.getData(); + bool null_map_contains_null = !memoryIsZero(null_map_data.raw_data(), null_map_data.size() * sizeof(UInt8)); if (null_map_contains_null) throw Exception( diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index d07c50ed15b..6201a06c74d 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -406,6 +406,7 @@ Chain buildPushingToViewsDrain( ASTPtr insert_query_ptr(insert.release()); InterpreterInsertQuery interpreter(insert_query_ptr, insert_context, false, false, false, view_runtime_data); BlockIO io = interpreter.execute(); + io.out.attachResources(QueryPipeline::getPipe(std::move(io.pipeline)).detachResources()); out = std::move(io.out); } else if (auto * live_view = dynamic_cast(dependent_table.get())) @@ -473,11 +474,14 @@ Chain buildPushingToViewsDrain( auto out = copying_data->getOutputs().begin(); auto in = finalizing_views->getInputs().begin(); + size_t max_parallel_streams = 0; + std::list processors; for (auto & chain : chains) { - result_chain.attachResourcesFrom(chain); + max_parallel_streams += std::max(chain.getNumThreads(), 1); + result_chain.attachResources(chain.detachResources()); connect(*out, chain.getInputPort()); connect(chain.getOutputPort(), *in); ++in; @@ -488,6 +492,7 @@ Chain buildPushingToViewsDrain( processors.emplace_front(std::move(copying_data)); processors.emplace_back(std::move(finalizing_views)); result_chain = Chain(std::move(processors)); + result_chain.setNumThreads(max_parallel_streams); } if (auto * live_view = dynamic_cast(storage.get())) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index aebf74f82b2..5ba2d95a037 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -377,7 +377,10 @@ BlockIO InterpreterInsertQuery::execute() }); } else + { res.out = std::move(out_chains.at(0)); + res.out.setNumThreads(std::min(res.out.getNumThreads(), settings.max_threads)); + } res.pipeline.addStorageHolder(table); if (const auto * mv = dynamic_cast(table.get())) @@ -386,6 +389,9 @@ BlockIO InterpreterInsertQuery::execute() res.pipeline.addStorageHolder(inner_table); } + if (!res.out.empty()) + res.out.attachResources(QueryPipeline::getPipe(std::move(res.pipeline)).detachResources()); + return res; } diff --git a/src/Processors/Chain.h b/src/Processors/Chain.h index 51825cb94a2..08a812a8227 100644 --- a/src/Processors/Chain.h +++ b/src/Processors/Chain.h @@ -21,6 +21,9 @@ public: bool empty() const { return processors.empty(); } + size_t getNumThreads() const { return num_threads; } + void setNumThreads(size_t num_threads_) { num_threads = num_threads_; } + void addSource(ProcessorPtr processor); void addSink(ProcessorPtr processor); @@ -37,7 +40,7 @@ public: static std::list getProcessors(Chain chain) { return std::move(chain.processors); } void addTableLock(TableLockHolder lock) { holder.table_locks.emplace_back(std::move(lock)); } - void attachResourcesFrom(Chain & other) { holder = std::move(other.holder); } + void attachResources(PipelineResourcesHolder holder_) { holder = std::move(holder_); } PipelineResourcesHolder detachResources() { return std::move(holder); } void reset(); @@ -48,6 +51,7 @@ private: /// input port output port std::list processors; PipelineResourcesHolder holder; + size_t num_threads = 0; }; } diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index dc59e0a2f5a..870787a3806 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + PullingPipelineExecutor::PullingPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) { pulling_format = std::make_shared(pipeline.getHeader(), has_data_flag); diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 677999879b6..938ce6f9b49 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -127,9 +127,7 @@ static void threadFunction(PushingAsyncPipelineExecutor::Data & data, ThreadGrou } -PushingAsyncPipelineExecutor::PushingAsyncPipelineExecutor(Chain & chain_, size_t num_threads_) - : num_threads(num_threads_) - , chain(chain_) +PushingAsyncPipelineExecutor::PushingAsyncPipelineExecutor(Chain & chain_) : chain(chain_) { pushing_source = std::make_shared(chain.getInputHeader()); auto sink = std::make_shared(chain.getOutputHeader()); @@ -175,7 +173,7 @@ void PushingAsyncPipelineExecutor::start() auto func = [&, thread_group = CurrentThread::getGroup()]() { - threadFunction(*data, thread_group, num_threads); + threadFunction(*data, thread_group, chain.getNumThreads()); }; data->thread = ThreadFromGlobalPool(std::move(func)); diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.h b/src/Processors/Executors/PushingAsyncPipelineExecutor.h index 69271afe389..2dac7cc64f0 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.h +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.h @@ -28,7 +28,7 @@ using Processors = std::vector; class PushingAsyncPipelineExecutor { public: - explicit PushingAsyncPipelineExecutor(Chain & chain, size_t num_threads_); + explicit PushingAsyncPipelineExecutor(Chain & chain); ~PushingAsyncPipelineExecutor(); /// Get structure of returned block or chunk. @@ -47,8 +47,6 @@ public: struct Data; private: - size_t num_threads; - Chain & chain; std::shared_ptr pushing_source; diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index c5a980ecdfa..64da0c33492 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -17,7 +17,7 @@ public: , need_data_flag(need_data_flag_) {} - String getName() const override { return "PullingOutputFormat"; } + String getName() const override { return "PushingSource"; } void setData(Chunk chunk) { @@ -70,7 +70,7 @@ PushingPipelineExecutor::~PushingPipelineExecutor() } catch (...) { - tryLogCurrentException("PullingPipelineExecutor"); + tryLogCurrentException("PushingPipelineExecutor"); } } diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index 55f000d5d7e..5038ee9f100 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -104,6 +104,11 @@ void Pipe::addQueryPlan(std::unique_ptr plan) holder.query_plans.emplace_back(std::move(plan)); } +PipelineResourcesHolder Pipe::detachResources() +{ + return std::move(holder); +} + Pipe::Pipe(ProcessorPtr source, OutputPort * output, OutputPort * totals, OutputPort * extremes) { if (!source->getInputs().empty()) @@ -670,9 +675,13 @@ void Pipe::addChains(std::vector chains) dropTotals(); dropExtremes(); + size_t max_parallel_streams_for_chains = 0; + Block new_header; for (size_t i = 0; i < output_ports.size(); ++i) { + max_parallel_streams_for_chains += std::max(chains[i].getNumThreads(), 1); + if (i == 0) new_header = chains[i].getOutputHeader(); else @@ -693,6 +702,7 @@ void Pipe::addChains(std::vector chains) } header = std::move(new_header); + max_parallel_streams = std::max(max_parallel_streams, max_parallel_streams_for_chains); } void Pipe::resize(size_t num_streams, bool force, bool strict) diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index bbcaa910872..1568622fbd5 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -115,6 +115,8 @@ public: /// For queries with nested interpreters (i.e. StorageDistributed) void addQueryPlan(std::unique_ptr plan); + PipelineResourcesHolder detachResources(); + private: /// Destruction order: processors, header, locks, temporary storages, local contexts PipelineResourcesHolder holder; From 3a9d88fbc97defac6514a4f6c9f3e92dd3b59d10 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Sep 2021 17:52:24 +0300 Subject: [PATCH 025/142] Fix more tests. --- src/Interpreters/InterpreterInsertQuery.cpp | 6 ++ src/Server/TCPHandler.cpp | 63 +++++++++++++------ src/Storages/StorageValues.cpp | 2 - ...iew_and_too_many_parts_zookeeper.reference | 2 + ...lized_view_and_too_many_parts_zookeeper.sh | 2 +- 5 files changed, 54 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 5ba2d95a037..4f3a7d3e4f3 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -351,6 +351,8 @@ BlockIO InterpreterInsertQuery::execute() return std::make_shared(in_header, actions); }); + auto num_select_threads = res.pipeline.getNumThreads(); + res.pipeline.addChains(std::move(out_chains)); res.pipeline.setSinks([&](const Block & cur_header, QueryPipeline::StreamType) -> ProcessorPtr @@ -358,6 +360,10 @@ BlockIO InterpreterInsertQuery::execute() return std::make_shared(cur_header); }); + /// Don't use more threads for insert then for select to reduce memory consumption. + if (!settings.parallel_view_processing && res.pipeline.getNumThreads() > num_select_threads) + res.pipeline.setMaxThreads(num_select_threads); + if (!allow_materialized) { for (const auto & column : metadata_snapshot->getColumns()) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 3658d2ddcba..b3dece3192e 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -37,6 +37,7 @@ #include #include +#include #include #include "Core/Protocol.h" @@ -543,35 +544,61 @@ void TCPHandler::skipData() void TCPHandler::processInsertQuery() { - PushingPipelineExecutor executor(state.io.out); - /** Made above the rest of the lines, so that in case of `writePrefix` function throws an exception, - * client receive exception before sending data. - */ - executor.start(); + size_t num_threads = state.io.out.getNumThreads(); - /// Send ColumnsDescription for insertion table - if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA) + auto send_table_columns = [&]() { - const auto & table_id = query_context->getInsertionTable(); - if (query_context->getSettingsRef().input_format_defaults_for_omitted_fields) + /// Send ColumnsDescription for insertion table + if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA) { - if (!table_id.empty()) + const auto & table_id = query_context->getInsertionTable(); + if (query_context->getSettingsRef().input_format_defaults_for_omitted_fields) { - auto storage_ptr = DatabaseCatalog::instance().getTable(table_id, query_context); - sendTableColumns(storage_ptr->getInMemoryMetadataPtr()->getColumns()); + if (!table_id.empty()) + { + auto storage_ptr = DatabaseCatalog::instance().getTable(table_id, query_context); + sendTableColumns(storage_ptr->getInMemoryMetadataPtr()->getColumns()); + } } } + }; + + if (num_threads > 1) + { + PushingAsyncPipelineExecutor executor(state.io.out); + /** Made above the rest of the lines, so that in case of `writePrefix` function throws an exception, + * client receive exception before sending data. + */ + executor.start(); + + send_table_columns(); + + /// Send block to the client - table structure. + sendData(executor.getHeader()); + + sendLogs(); + + while (readDataNext()) + executor.push(std::move(state.block_for_insert)); + + executor.finish(); } + else + { + PushingPipelineExecutor executor(state.io.out); + executor.start(); - /// Send block to the client - table structure. - sendData(executor.getHeader()); + send_table_columns(); - sendLogs(); + sendData(executor.getHeader()); - while (readDataNext()) - executor.push(std::move(state.block_for_insert)); + sendLogs(); - executor.finish(); + while (readDataNext()) + executor.push(std::move(state.block_for_insert)); + + executor.finish(); + } } diff --git a/src/Storages/StorageValues.cpp b/src/Storages/StorageValues.cpp index fd1652b2b04..ace5ca3667c 100644 --- a/src/Storages/StorageValues.cpp +++ b/src/Storages/StorageValues.cpp @@ -36,8 +36,6 @@ Pipe StorageValues::read( for (const auto & name : column_names) block.insert(res_block.getByName(name)); - std::cerr << "=== Reading from Values " << block.rows() << " rows\n"; - Chunk chunk(block.getColumns(), block.rows()); return Pipe(std::make_shared(block.cloneEmpty(), std::move(chunk))); } diff --git a/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.reference b/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.reference index 6e0517557ad..34c1c258e4b 100644 --- a/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.reference +++ b/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.reference @@ -3,7 +3,9 @@ b 1 c 1 a 1 +a 2 b 1 c 1 +c 2 1 diff --git a/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh b/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh index def8e8f4cfe..7b48c96f068 100755 --- a/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh +++ b/tests/queries/0_stateless/00633_materialized_view_and_too_many_parts_zookeeper.sh @@ -23,7 +23,7 @@ if ${CLICKHOUSE_CLIENT} --query "INSERT INTO root VALUES (2)" 2>/dev/null; then fi echo -${CLICKHOUSE_CLIENT} --query "SELECT _table, d FROM merge('${CLICKHOUSE_DATABASE}', '^[abc]\$') ORDER BY _table" +${CLICKHOUSE_CLIENT} --query "SELECT _table, d FROM merge('${CLICKHOUSE_DATABASE}', '^[abc]\$') ORDER BY _table, d" ${CLICKHOUSE_CLIENT} --query "DROP TABLE root" ${CLICKHOUSE_CLIENT} --query "DROP TABLE a" From 05f533d4623f97a465ee1a9a84db1df70473858c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 10 Sep 2021 17:54:42 +0300 Subject: [PATCH 026/142] Fix style. --- src/DataStreams/CheckConstraintsBlockOutputStream.cpp | 6 +++--- src/Processors/Executors/PushingPipelineExecutor.cpp | 5 +++++ 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index f133c33e302..26275f7d3c7 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -80,15 +80,15 @@ void CheckConstraintsTransform::transform(Chunk & chunk) const ColumnUInt8 & res_column_uint8 = assert_cast(*result_column); - const UInt8 * data = res_column_uint8.getData().data(); + const UInt8 * res_data = res_column_uint8.getData().data(); size_t size = res_column_uint8.size(); /// Is violated. - if (!memoryIsByte(data, size, 1)) + if (!memoryIsByte(res_data, size, 1)) { size_t row_idx = 0; for (; row_idx < size; ++row_idx) - if (data[row_idx] != 1) + if (res_data[row_idx] != 1) break; Names related_columns = constraint_expr->getRequiredColumns(); diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index 64da0c33492..f5857b4fa8a 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -9,6 +9,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + class PushingSource : public ISource { public: From 057312a6f479906cac4343c111c49c1362b2f826 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sun, 12 Sep 2021 19:58:02 +0300 Subject: [PATCH 027/142] Update uk-price-paid.md --- .../example-datasets/uk-price-paid.md | 632 +++++++++--------- 1 file changed, 322 insertions(+), 310 deletions(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index ad17fe87371..bdb8e7ba2cf 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -110,11 +110,11 @@ SELECT count() FROM uk_price_paid; Result: -```text -26248711 -``` +┌──count()─┐ +│ 26321785 │ +└──────────┘ -The size of dataset in ClickHouse is just 226 MiB: +The size of dataset in ClickHouse is just 278 MiB: Query: @@ -125,12 +125,14 @@ SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price Result: ```text -226.40 MiB +┌─formatReadableSize(total_bytes)─┐ +│ 278.80 MiB │ +└─────────────────────────────────┘ ``` ## Run Some Queries -### Average price per year +### Average price per year: Query: @@ -145,33 +147,31 @@ Result: │ 1995 │ 67932 │ █████▍ │ │ 1996 │ 71505 │ █████▋ │ │ 1997 │ 78532 │ ██████▎ │ -│ 1998 │ 85435 │ ██████▋ │ -│ 1999 │ 96036 │ ███████▋ │ -│ 2000 │ 107478 │ ████████▌ │ -│ 2001 │ 118886 │ █████████▌ │ -│ 2002 │ 137940 │ ███████████ │ -│ 2003 │ 155888 │ ████████████▍ │ +│ 1998 │ 85436 │ ██████▋ │ +│ 1999 │ 96037 │ ███████▋ │ +│ 2000 │ 107479 │ ████████▌ │ +│ 2001 │ 118885 │ █████████▌ │ +│ 2002 │ 137941 │ ███████████ │ +│ 2003 │ 155889 │ ████████████▍ │ │ 2004 │ 178885 │ ██████████████▎ │ -│ 2005 │ 189350 │ ███████████████▏ │ +│ 2005 │ 189351 │ ███████████████▏ │ │ 2006 │ 203528 │ ████████████████▎ │ -│ 2007 │ 219377 │ █████████████████▌ │ +│ 2007 │ 219378 │ █████████████████▌ │ │ 2008 │ 217056 │ █████████████████▎ │ │ 2009 │ 213419 │ █████████████████ │ -│ 2010 │ 236110 │ ██████████████████▊ │ -│ 2011 │ 232804 │ ██████████████████▌ │ -│ 2012 │ 238366 │ ███████████████████ │ +│ 2010 │ 236109 │ ██████████████████▊ │ +│ 2011 │ 232805 │ ██████████████████▌ │ +│ 2012 │ 238367 │ ███████████████████ │ │ 2013 │ 256931 │ ████████████████████▌ │ -│ 2014 │ 279917 │ ██████████████████████▍ │ -│ 2015 │ 297264 │ ███████████████████████▋ │ -│ 2016 │ 313197 │ █████████████████████████ │ -│ 2017 │ 346070 │ ███████████████████████████▋ │ -│ 2018 │ 350117 │ ████████████████████████████ │ -│ 2019 │ 351010 │ ████████████████████████████ │ -│ 2020 │ 368974 │ █████████████████████████████▌ │ -│ 2021 │ 384351 │ ██████████████████████████████▋ │ +│ 2014 │ 279915 │ ██████████████████████▍ │ +│ 2015 │ 297266 │ ███████████████████████▋ │ +│ 2016 │ 313201 │ █████████████████████████ │ +│ 2017 │ 346097 │ ███████████████████████████▋ │ +│ 2018 │ 350116 │ ████████████████████████████ │ +│ 2019 │ 351013 │ ████████████████████████████ │ +│ 2020 │ 369420 │ █████████████████████████████▌ │ +│ 2021 │ 386903 │ ██████████████████████████████▊ │ └──────┴────────┴────────────────────────────────────────┘ - -27 rows in set. Elapsed: 0.027 sec. Processed 26.25 million rows, 157.49 MB (955.96 million rows/s., 5.74 GB/s.) ``` ### Average price per year in London: @@ -182,44 +182,46 @@ Query: SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, 100) FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year; ``` +Result: + ```text ┌─year─┬───price─┬─bar(round(avg(price)), 0, 2000000, 100)───────────────┐ -│ 1995 │ 109112 │ █████▍ │ +│ 1995 │ 109116 │ █████▍ │ │ 1996 │ 118667 │ █████▊ │ │ 1997 │ 136518 │ ██████▋ │ │ 1998 │ 152983 │ ███████▋ │ -│ 1999 │ 180633 │ █████████ │ -│ 2000 │ 215830 │ ██████████▋ │ -│ 2001 │ 232996 │ ███████████▋ │ -│ 2002 │ 263672 │ █████████████▏ │ +│ 1999 │ 180637 │ █████████ │ +│ 2000 │ 215838 │ ██████████▋ │ +│ 2001 │ 232994 │ ███████████▋ │ +│ 2002 │ 263670 │ █████████████▏ │ │ 2003 │ 278394 │ █████████████▊ │ -│ 2004 │ 304665 │ ███████████████▏ │ +│ 2004 │ 304666 │ ███████████████▏ │ │ 2005 │ 322875 │ ████████████████▏ │ -│ 2006 │ 356192 │ █████████████████▋ │ -│ 2007 │ 404055 │ ████████████████████▏ │ +│ 2006 │ 356191 │ █████████████████▋ │ +│ 2007 │ 404054 │ ████████████████████▏ │ │ 2008 │ 420741 │ █████████████████████ │ -│ 2009 │ 427754 │ █████████████████████▍ │ +│ 2009 │ 427753 │ █████████████████████▍ │ │ 2010 │ 480306 │ ████████████████████████ │ │ 2011 │ 496274 │ ████████████████████████▋ │ -│ 2012 │ 519441 │ █████████████████████████▊ │ -│ 2013 │ 616209 │ ██████████████████████████████▋ │ -│ 2014 │ 724144 │ ████████████████████████████████████▏ │ -│ 2015 │ 792112 │ ███████████████████████████████████████▌ │ -│ 2016 │ 843568 │ ██████████████████████████████████████████▏ │ -│ 2017 │ 982566 │ █████████████████████████████████████████████████▏ │ -│ 2018 │ 1016845 │ ██████████████████████████████████████████████████▋ │ -│ 2019 │ 1043277 │ ████████████████████████████████████████████████████▏ │ -│ 2020 │ 1003963 │ ██████████████████████████████████████████████████▏ │ -│ 2021 │ 940794 │ ███████████████████████████████████████████████ │ +│ 2012 │ 519442 │ █████████████████████████▊ │ +│ 2013 │ 616212 │ ██████████████████████████████▋ │ +│ 2014 │ 724154 │ ████████████████████████████████████▏ │ +│ 2015 │ 792129 │ ███████████████████████████████████████▌ │ +│ 2016 │ 843655 │ ██████████████████████████████████████████▏ │ +│ 2017 │ 982642 │ █████████████████████████████████████████████████▏ │ +│ 2018 │ 1016835 │ ██████████████████████████████████████████████████▋ │ +│ 2019 │ 1042849 │ ████████████████████████████████████████████████████▏ │ +│ 2020 │ 1011889 │ ██████████████████████████████████████████████████▌ │ +│ 2021 │ 960343 │ ████████████████████████████████████████████████ │ └──────┴─────────┴───────────────────────────────────────────────────────┘ - -27 rows in set. Elapsed: 0.024 sec. Processed 26.25 million rows, 76.88 MB (1.08 billion rows/s., 3.15 GB/s.) ``` Something happened in 2013. I don't have a clue. Maybe you have a clue what happened in 2020? ### The most expensive neighborhoods: +Query: + ```sql SELECT town, @@ -237,124 +239,121 @@ ORDER BY price DESC LIMIT 100; ``` +Result: + ```text + ┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ -│ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ -│ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ -│ LONDON │ KENSINGTON AND CHELSEA │ 2367 │ 2342422 │ ██████████████████████████████████████████████▋ │ -│ LEATHERHEAD │ ELMBRIDGE │ 108 │ 1927143 │ ██████████████████████████████████████▌ │ -│ VIRGINIA WATER │ RUNNYMEDE │ 142 │ 1868819 │ █████████████████████████████████████▍ │ -│ LONDON │ CAMDEN │ 2815 │ 1736788 │ ██████████████████████████████████▋ │ -│ THORNTON HEATH │ CROYDON │ 521 │ 1733051 │ ██████████████████████████████████▋ │ -│ WINDLESHAM │ SURREY HEATH │ 103 │ 1717255 │ ██████████████████████████████████▎ │ -│ BARNET │ ENFIELD │ 115 │ 1503458 │ ██████████████████████████████ │ -│ OXFORD │ SOUTH OXFORDSHIRE │ 298 │ 1275200 │ █████████████████████████▌ │ -│ LONDON │ ISLINGTON │ 2458 │ 1274308 │ █████████████████████████▍ │ -│ COBHAM │ ELMBRIDGE │ 364 │ 1260005 │ █████████████████████████▏ │ -│ LONDON │ HOUNSLOW │ 618 │ 1215682 │ ████████████████████████▎ │ -│ ASCOT │ WINDSOR AND MAIDENHEAD │ 379 │ 1215146 │ ████████████████████████▎ │ -│ LONDON │ RICHMOND UPON THAMES │ 654 │ 1207551 │ ████████████████████████▏ │ -│ BEACONSFIELD │ BUCKINGHAMSHIRE │ 307 │ 1186220 │ ███████████████████████▋ │ -│ RICHMOND │ RICHMOND UPON THAMES │ 805 │ 1100420 │ ██████████████████████ │ -│ LONDON │ HAMMERSMITH AND FULHAM │ 2888 │ 1062959 │ █████████████████████▎ │ -│ WEYBRIDGE │ ELMBRIDGE │ 607 │ 1027161 │ ████████████████████▌ │ -│ RADLETT │ HERTSMERE │ 265 │ 1015896 │ ████████████████████▎ │ -│ SALCOMBE │ SOUTH HAMS │ 124 │ 1014393 │ ████████████████████▎ │ -│ BURFORD │ WEST OXFORDSHIRE │ 102 │ 993100 │ ███████████████████▋ │ -│ ESHER │ ELMBRIDGE │ 454 │ 969770 │ ███████████████████▍ │ -│ HINDHEAD │ WAVERLEY │ 128 │ 967786 │ ███████████████████▎ │ -│ BROCKENHURST │ NEW FOREST │ 121 │ 967046 │ ███████████████████▎ │ -│ LEATHERHEAD │ GUILDFORD │ 191 │ 964489 │ ███████████████████▎ │ -│ GERRARDS CROSS │ BUCKINGHAMSHIRE │ 376 │ 958555 │ ███████████████████▏ │ -│ EAST MOLESEY │ ELMBRIDGE │ 181 │ 943457 │ ██████████████████▋ │ -│ OLNEY │ MILTON KEYNES │ 220 │ 942892 │ ██████████████████▋ │ -│ CHALFONT ST GILES │ BUCKINGHAMSHIRE │ 135 │ 926950 │ ██████████████████▌ │ -│ HENLEY-ON-THAMES │ SOUTH OXFORDSHIRE │ 509 │ 905732 │ ██████████████████ │ -│ KINGSTON UPON THAMES │ KINGSTON UPON THAMES │ 889 │ 899689 │ █████████████████▊ │ -│ BELVEDERE │ BEXLEY │ 313 │ 895336 │ █████████████████▊ │ -│ CRANBROOK │ TUNBRIDGE WELLS │ 404 │ 888190 │ █████████████████▋ │ -│ LONDON │ EALING │ 2460 │ 865893 │ █████████████████▎ │ -│ MAIDENHEAD │ BUCKINGHAMSHIRE │ 114 │ 863814 │ █████████████████▎ │ -│ LONDON │ MERTON │ 1958 │ 857192 │ █████████████████▏ │ -│ GUILDFORD │ WAVERLEY │ 131 │ 854447 │ █████████████████ │ -│ LONDON │ HACKNEY │ 3088 │ 846571 │ ████████████████▊ │ -│ LYMM │ WARRINGTON │ 285 │ 839920 │ ████████████████▋ │ -│ HARPENDEN │ ST ALBANS │ 606 │ 836994 │ ████████████████▋ │ -│ LONDON │ WANDSWORTH │ 6113 │ 832292 │ ████████████████▋ │ -│ LONDON │ SOUTHWARK │ 3612 │ 831319 │ ████████████████▋ │ -│ BERKHAMSTED │ DACORUM │ 502 │ 830356 │ ████████████████▌ │ -│ KINGS LANGLEY │ DACORUM │ 137 │ 821358 │ ████████████████▍ │ -│ TONBRIDGE │ TUNBRIDGE WELLS │ 339 │ 806736 │ ████████████████▏ │ -│ EPSOM │ REIGATE AND BANSTEAD │ 157 │ 805903 │ ████████████████ │ -│ WOKING │ GUILDFORD │ 161 │ 803283 │ ████████████████ │ -│ STOCKBRIDGE │ TEST VALLEY │ 168 │ 801973 │ ████████████████ │ -│ TEDDINGTON │ RICHMOND UPON THAMES │ 539 │ 798591 │ ███████████████▊ │ -│ OXFORD │ VALE OF WHITE HORSE │ 329 │ 792907 │ ███████████████▋ │ -│ LONDON │ BARNET │ 3624 │ 789583 │ ███████████████▋ │ -│ TWICKENHAM │ RICHMOND UPON THAMES │ 1090 │ 787760 │ ███████████████▋ │ -│ LUTON │ CENTRAL BEDFORDSHIRE │ 196 │ 786051 │ ███████████████▋ │ -│ TONBRIDGE │ MAIDSTONE │ 277 │ 785746 │ ███████████████▋ │ -│ TOWCESTER │ WEST NORTHAMPTONSHIRE │ 186 │ 783532 │ ███████████████▋ │ -│ LONDON │ LAMBETH │ 4832 │ 783422 │ ███████████████▋ │ -│ LUTTERWORTH │ HARBOROUGH │ 515 │ 781775 │ ███████████████▋ │ -│ WOODSTOCK │ WEST OXFORDSHIRE │ 135 │ 777499 │ ███████████████▌ │ -│ ALRESFORD │ WINCHESTER │ 196 │ 775577 │ ███████████████▌ │ -│ LONDON │ NEWHAM │ 2942 │ 768551 │ ███████████████▎ │ -│ ALDERLEY EDGE │ CHESHIRE EAST │ 168 │ 768280 │ ███████████████▎ │ -│ MARLOW │ BUCKINGHAMSHIRE │ 301 │ 762784 │ ███████████████▎ │ -│ BILLINGSHURST │ CHICHESTER │ 134 │ 760920 │ ███████████████▏ │ -│ LONDON │ TOWER HAMLETS │ 4183 │ 759635 │ ███████████████▏ │ -│ MIDHURST │ CHICHESTER │ 245 │ 759101 │ ███████████████▏ │ -│ THAMES DITTON │ ELMBRIDGE │ 227 │ 753347 │ ███████████████ │ -│ POTTERS BAR │ WELWYN HATFIELD │ 163 │ 752926 │ ███████████████ │ -│ REIGATE │ REIGATE AND BANSTEAD │ 555 │ 740961 │ ██████████████▋ │ -│ TADWORTH │ REIGATE AND BANSTEAD │ 477 │ 738997 │ ██████████████▋ │ -│ SEVENOAKS │ SEVENOAKS │ 1074 │ 734658 │ ██████████████▋ │ -│ PETWORTH │ CHICHESTER │ 138 │ 732432 │ ██████████████▋ │ -│ BOURNE END │ BUCKINGHAMSHIRE │ 127 │ 730742 │ ██████████████▌ │ -│ PURLEY │ CROYDON │ 540 │ 727721 │ ██████████████▌ │ -│ OXTED │ TANDRIDGE │ 320 │ 726078 │ ██████████████▌ │ -│ LONDON │ HARINGEY │ 2988 │ 724573 │ ██████████████▍ │ -│ BANSTEAD │ REIGATE AND BANSTEAD │ 373 │ 713834 │ ██████████████▎ │ -│ PINNER │ HARROW │ 480 │ 712166 │ ██████████████▏ │ -│ MALMESBURY │ WILTSHIRE │ 293 │ 707747 │ ██████████████▏ │ -│ RICKMANSWORTH │ THREE RIVERS │ 732 │ 705400 │ ██████████████ │ -│ SLOUGH │ BUCKINGHAMSHIRE │ 359 │ 705002 │ ██████████████ │ -│ GREAT MISSENDEN │ BUCKINGHAMSHIRE │ 214 │ 704904 │ ██████████████ │ -│ READING │ SOUTH OXFORDSHIRE │ 295 │ 701697 │ ██████████████ │ -│ HYTHE │ FOLKESTONE AND HYTHE │ 457 │ 700334 │ ██████████████ │ -│ WELWYN │ WELWYN HATFIELD │ 217 │ 699649 │ █████████████▊ │ -│ CHIGWELL │ EPPING FOREST │ 242 │ 697869 │ █████████████▊ │ -│ BARNET │ BARNET │ 906 │ 695680 │ █████████████▊ │ -│ HASLEMERE │ CHICHESTER │ 120 │ 694028 │ █████████████▊ │ -│ LEATHERHEAD │ MOLE VALLEY │ 748 │ 692026 │ █████████████▋ │ -│ LONDON │ BRENT │ 1945 │ 690799 │ █████████████▋ │ -│ HASLEMERE │ WAVERLEY │ 258 │ 690765 │ █████████████▋ │ -│ NORTHWOOD │ HILLINGDON │ 252 │ 690753 │ █████████████▋ │ -│ WALTON-ON-THAMES │ ELMBRIDGE │ 871 │ 689431 │ █████████████▋ │ -│ INGATESTONE │ BRENTWOOD │ 150 │ 688345 │ █████████████▋ │ -│ OXFORD │ OXFORD │ 1761 │ 686114 │ █████████████▋ │ -│ CHISLEHURST │ BROMLEY │ 410 │ 682892 │ █████████████▋ │ -│ KINGS LANGLEY │ THREE RIVERS │ 109 │ 682320 │ █████████████▋ │ -│ ASHTEAD │ MOLE VALLEY │ 280 │ 680483 │ █████████████▌ │ -│ WOKING │ SURREY HEATH │ 269 │ 679035 │ █████████████▌ │ -│ ASCOT │ BRACKNELL FOREST │ 160 │ 678632 │ █████████████▌ │ +│ LONDON │ CITY OF WESTMINSTER │ 3606 │ 3280239 │ █████████████████████████████████████████████████████████████████▌ │ +│ LONDON │ CITY OF LONDON │ 274 │ 3160502 │ ███████████████████████████████████████████████████████████████▏ │ +│ LONDON │ KENSINGTON AND CHELSEA │ 2550 │ 2308478 │ ██████████████████████████████████████████████▏ │ +│ LEATHERHEAD │ ELMBRIDGE │ 114 │ 1897407 │ █████████████████████████████████████▊ │ +│ LONDON │ CAMDEN │ 3033 │ 1805404 │ ████████████████████████████████████ │ +│ VIRGINIA WATER │ RUNNYMEDE │ 156 │ 1753247 │ ███████████████████████████████████ │ +│ WINDLESHAM │ SURREY HEATH │ 108 │ 1677613 │ █████████████████████████████████▌ │ +│ THORNTON HEATH │ CROYDON │ 546 │ 1671721 │ █████████████████████████████████▍ │ +│ BARNET │ ENFIELD │ 124 │ 1505840 │ ██████████████████████████████ │ +│ COBHAM │ ELMBRIDGE │ 387 │ 1237250 │ ████████████████████████▋ │ +│ LONDON │ ISLINGTON │ 2668 │ 1236980 │ ████████████████████████▋ │ +│ OXFORD │ SOUTH OXFORDSHIRE │ 321 │ 1220907 │ ████████████████████████▍ │ +│ LONDON │ RICHMOND UPON THAMES │ 704 │ 1215551 │ ████████████████████████▎ │ +│ LONDON │ HOUNSLOW │ 671 │ 1207493 │ ████████████████████████▏ │ +│ ASCOT │ WINDSOR AND MAIDENHEAD │ 407 │ 1183299 │ ███████████████████████▋ │ +│ BEACONSFIELD │ BUCKINGHAMSHIRE │ 330 │ 1175615 │ ███████████████████████▌ │ +│ RICHMOND │ RICHMOND UPON THAMES │ 874 │ 1110444 │ ██████████████████████▏ │ +│ LONDON │ HAMMERSMITH AND FULHAM │ 3086 │ 1053983 │ █████████████████████ │ +│ SURBITON │ ELMBRIDGE │ 100 │ 1011800 │ ████████████████████▏ │ +│ RADLETT │ HERTSMERE │ 283 │ 1011712 │ ████████████████████▏ │ +│ SALCOMBE │ SOUTH HAMS │ 127 │ 1011624 │ ████████████████████▏ │ +│ WEYBRIDGE │ ELMBRIDGE │ 655 │ 1007265 │ ████████████████████▏ │ +│ ESHER │ ELMBRIDGE │ 485 │ 986581 │ ███████████████████▋ │ +│ LEATHERHEAD │ GUILDFORD │ 202 │ 977320 │ ███████████████████▌ │ +│ BURFORD │ WEST OXFORDSHIRE │ 111 │ 966893 │ ███████████████████▎ │ +│ BROCKENHURST │ NEW FOREST │ 129 │ 956675 │ ███████████████████▏ │ +│ HINDHEAD │ WAVERLEY │ 137 │ 953753 │ ███████████████████ │ +│ GERRARDS CROSS │ BUCKINGHAMSHIRE │ 419 │ 951121 │ ███████████████████ │ +│ EAST MOLESEY │ ELMBRIDGE │ 192 │ 936769 │ ██████████████████▋ │ +│ CHALFONT ST GILES │ BUCKINGHAMSHIRE │ 146 │ 925515 │ ██████████████████▌ │ +│ LONDON │ TOWER HAMLETS │ 4388 │ 918304 │ ██████████████████▎ │ +│ OLNEY │ MILTON KEYNES │ 235 │ 910646 │ ██████████████████▏ │ +│ HENLEY-ON-THAMES │ SOUTH OXFORDSHIRE │ 540 │ 902418 │ ██████████████████ │ +│ LONDON │ SOUTHWARK │ 3885 │ 892997 │ █████████████████▋ │ +│ KINGSTON UPON THAMES │ KINGSTON UPON THAMES │ 960 │ 885969 │ █████████████████▋ │ +│ LONDON │ EALING │ 2658 │ 871755 │ █████████████████▍ │ +│ CRANBROOK │ TUNBRIDGE WELLS │ 431 │ 862348 │ █████████████████▏ │ +│ LONDON │ MERTON │ 2099 │ 859118 │ █████████████████▏ │ +│ BELVEDERE │ BEXLEY │ 346 │ 842423 │ ████████████████▋ │ +│ GUILDFORD │ WAVERLEY │ 143 │ 841277 │ ████████████████▋ │ +│ HARPENDEN │ ST ALBANS │ 657 │ 841216 │ ████████████████▋ │ +│ LONDON │ HACKNEY │ 3307 │ 837090 │ ████████████████▋ │ +│ LONDON │ WANDSWORTH │ 6566 │ 832663 │ ████████████████▋ │ +│ MAIDENHEAD │ BUCKINGHAMSHIRE │ 123 │ 824299 │ ████████████████▍ │ +│ KINGS LANGLEY │ DACORUM │ 145 │ 821331 │ ████████████████▍ │ +│ BERKHAMSTED │ DACORUM │ 543 │ 818415 │ ████████████████▎ │ +│ GREAT MISSENDEN │ BUCKINGHAMSHIRE │ 226 │ 802807 │ ████████████████ │ +│ BILLINGSHURST │ CHICHESTER │ 144 │ 797829 │ ███████████████▊ │ +│ WOKING │ GUILDFORD │ 176 │ 793494 │ ███████████████▋ │ +│ STOCKBRIDGE │ TEST VALLEY │ 178 │ 793269 │ ███████████████▋ │ +│ EPSOM │ REIGATE AND BANSTEAD │ 172 │ 791862 │ ███████████████▋ │ +│ TONBRIDGE │ TUNBRIDGE WELLS │ 360 │ 787876 │ ███████████████▋ │ +│ TEDDINGTON │ RICHMOND UPON THAMES │ 595 │ 786492 │ ███████████████▋ │ +│ TWICKENHAM │ RICHMOND UPON THAMES │ 1155 │ 786193 │ ███████████████▋ │ +│ LYNDHURST │ NEW FOREST │ 102 │ 785593 │ ███████████████▋ │ +│ LONDON │ LAMBETH │ 5228 │ 774574 │ ███████████████▍ │ +│ LONDON │ BARNET │ 3955 │ 773259 │ ███████████████▍ │ +│ OXFORD │ VALE OF WHITE HORSE │ 353 │ 772088 │ ███████████████▍ │ +│ TONBRIDGE │ MAIDSTONE │ 305 │ 770740 │ ███████████████▍ │ +│ LUTTERWORTH │ HARBOROUGH │ 538 │ 768634 │ ███████████████▎ │ +│ WOODSTOCK │ WEST OXFORDSHIRE │ 140 │ 766037 │ ███████████████▎ │ +│ MIDHURST │ CHICHESTER │ 257 │ 764815 │ ███████████████▎ │ +│ MARLOW │ BUCKINGHAMSHIRE │ 327 │ 761876 │ ███████████████▏ │ +│ LONDON │ NEWHAM │ 3237 │ 761784 │ ███████████████▏ │ +│ ALDERLEY EDGE │ CHESHIRE EAST │ 178 │ 757318 │ ███████████████▏ │ +│ LUTON │ CENTRAL BEDFORDSHIRE │ 212 │ 754283 │ ███████████████ │ +│ PETWORTH │ CHICHESTER │ 154 │ 754220 │ ███████████████ │ +│ ALRESFORD │ WINCHESTER │ 219 │ 752718 │ ███████████████ │ +│ POTTERS BAR │ WELWYN HATFIELD │ 174 │ 748465 │ ██████████████▊ │ +│ HASLEMERE │ CHICHESTER │ 128 │ 746907 │ ██████████████▊ │ +│ TADWORTH │ REIGATE AND BANSTEAD │ 502 │ 743252 │ ██████████████▋ │ +│ THAMES DITTON │ ELMBRIDGE │ 244 │ 741913 │ ██████████████▋ │ +│ REIGATE │ REIGATE AND BANSTEAD │ 581 │ 738198 │ ██████████████▋ │ +│ BOURNE END │ BUCKINGHAMSHIRE │ 138 │ 735190 │ ██████████████▋ │ +│ SEVENOAKS │ SEVENOAKS │ 1156 │ 730018 │ ██████████████▌ │ +│ OXTED │ TANDRIDGE │ 336 │ 729123 │ ██████████████▌ │ +│ INGATESTONE │ BRENTWOOD │ 166 │ 728103 │ ██████████████▌ │ +│ LONDON │ BRENT │ 2079 │ 720605 │ ██████████████▍ │ +│ LONDON │ HARINGEY │ 3216 │ 717780 │ ██████████████▎ │ +│ PURLEY │ CROYDON │ 575 │ 716108 │ ██████████████▎ │ +│ WELWYN │ WELWYN HATFIELD │ 222 │ 710603 │ ██████████████▏ │ +│ RICKMANSWORTH │ THREE RIVERS │ 798 │ 704571 │ ██████████████ │ +│ BANSTEAD │ REIGATE AND BANSTEAD │ 401 │ 701293 │ ██████████████ │ +│ CHIGWELL │ EPPING FOREST │ 261 │ 701203 │ ██████████████ │ +│ PINNER │ HARROW │ 528 │ 698885 │ █████████████▊ │ +│ HASLEMERE │ WAVERLEY │ 280 │ 696659 │ █████████████▊ │ +│ SLOUGH │ BUCKINGHAMSHIRE │ 396 │ 694917 │ █████████████▊ │ +│ WALTON-ON-THAMES │ ELMBRIDGE │ 946 │ 692395 │ █████████████▋ │ +│ READING │ SOUTH OXFORDSHIRE │ 318 │ 691988 │ █████████████▋ │ +│ NORTHWOOD │ HILLINGDON │ 271 │ 690643 │ █████████████▋ │ +│ FELTHAM │ HOUNSLOW │ 763 │ 688595 │ █████████████▋ │ +│ ASHTEAD │ MOLE VALLEY │ 303 │ 687923 │ █████████████▋ │ +│ BARNET │ BARNET │ 975 │ 686980 │ █████████████▋ │ +│ WOKING │ SURREY HEATH │ 283 │ 686669 │ █████████████▋ │ +│ MALMESBURY │ WILTSHIRE │ 323 │ 683324 │ █████████████▋ │ +│ AMERSHAM │ BUCKINGHAMSHIRE │ 496 │ 680962 │ █████████████▌ │ +│ CHISLEHURST │ BROMLEY │ 430 │ 680209 │ █████████████▌ │ +│ HYTHE │ FOLKESTONE AND HYTHE │ 490 │ 676908 │ █████████████▌ │ +│ MAYFIELD │ WEALDEN │ 101 │ 676210 │ █████████████▌ │ +│ ASCOT │ BRACKNELL FOREST │ 168 │ 676004 │ █████████████▌ │ └──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ - -100 rows in set. Elapsed: 0.039 sec. Processed 26.25 million rows, 278.03 MB (674.32 million rows/s., 7.14 GB/s.) ``` -### Test it in Playground - -The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). - ## Let's speed up queries using projections [Projections](https://../../sql-reference/statements/alter/projection/) allow to improve queries speed by storing pre-aggregated data. ### Build a projection -Create an aggregate projection by dimensions (toYear(date), district, town): +Create an aggregate projection by dimensions `toYear(date)`, `district`, `town`. ```sql ALTER TABLE uk_price_paid @@ -374,7 +373,7 @@ ALTER TABLE uk_price_paid ); ``` -Populate the projection for existing data (without it projection will be created for only newly inserted data) +Populate the projection for existing data (without it projection will be created for only newly inserted data). ```sql ALTER TABLE uk_price_paid @@ -384,14 +383,15 @@ SETTINGS mutations_sync = 1; ## Test performance -Enable projections for selects and let's run the same 3 queries. +Let's run the same 3 queries. + +Enable projections for selects ```sql -set allow_experimental_projection_optimization=1; +SET allow_experimental_projection_optimization = 1; ``` - -### Average price per year +### Query 1. Average price per year Query: @@ -412,36 +412,34 @@ Result: │ 1995 │ 67932 │ █████▍ │ │ 1996 │ 71505 │ █████▋ │ │ 1997 │ 78532 │ ██████▎ │ -│ 1998 │ 85435 │ ██████▋ │ -│ 1999 │ 96036 │ ███████▋ │ -│ 2000 │ 107478 │ ████████▌ │ -│ 2001 │ 118886 │ █████████▌ │ -│ 2002 │ 137940 │ ███████████ │ -│ 2003 │ 155888 │ ████████████▍ │ +│ 1998 │ 85436 │ ██████▋ │ +│ 1999 │ 96037 │ ███████▋ │ +│ 2000 │ 107479 │ ████████▌ │ +│ 2001 │ 118885 │ █████████▌ │ +│ 2002 │ 137941 │ ███████████ │ +│ 2003 │ 155889 │ ████████████▍ │ │ 2004 │ 178885 │ ██████████████▎ │ -│ 2005 │ 189350 │ ███████████████▏ │ +│ 2005 │ 189351 │ ███████████████▏ │ │ 2006 │ 203528 │ ████████████████▎ │ -│ 2007 │ 219377 │ █████████████████▌ │ +│ 2007 │ 219378 │ █████████████████▌ │ │ 2008 │ 217056 │ █████████████████▎ │ │ 2009 │ 213419 │ █████████████████ │ -│ 2010 │ 236110 │ ██████████████████▊ │ -│ 2011 │ 232804 │ ██████████████████▌ │ -│ 2012 │ 238366 │ ███████████████████ │ +│ 2010 │ 236109 │ ██████████████████▊ │ +│ 2011 │ 232805 │ ██████████████████▌ │ +│ 2012 │ 238367 │ ███████████████████ │ │ 2013 │ 256931 │ ████████████████████▌ │ -│ 2014 │ 279917 │ ██████████████████████▍ │ -│ 2015 │ 297264 │ ███████████████████████▋ │ -│ 2016 │ 313197 │ █████████████████████████ │ -│ 2017 │ 346070 │ ███████████████████████████▋ │ -│ 2018 │ 350117 │ ████████████████████████████ │ -│ 2019 │ 351010 │ ████████████████████████████ │ -│ 2020 │ 368974 │ █████████████████████████████▌ │ -│ 2021 │ 384351 │ ██████████████████████████████▋ │ +│ 2014 │ 279915 │ ██████████████████████▍ │ +│ 2015 │ 297266 │ ███████████████████████▋ │ +│ 2016 │ 313201 │ █████████████████████████ │ +│ 2017 │ 346097 │ ███████████████████████████▋ │ +│ 2018 │ 350116 │ ████████████████████████████ │ +│ 2019 │ 351013 │ ████████████████████████████ │ +│ 2020 │ 369420 │ █████████████████████████████▌ │ +│ 2021 │ 386903 │ ██████████████████████████████▊ │ └──────┴────────┴────────────────────────────────────────┘ - -27 rows in set. Elapsed: 0.003 sec. Processed 106.87 thousand rows, 3.21 MB (31.92 million rows/s., 959.03 MB/s.) ``` -### Average price per year in London +### Query 2. Average price per year in London Query: @@ -460,39 +458,39 @@ Result: ```text ┌─year─┬───price─┬─bar(round(avg(price)), 0, 2000000, 100)───────────────┐ -│ 1995 │ 109112 │ █████▍ │ +│ 1995 │ 109116 │ █████▍ │ │ 1996 │ 118667 │ █████▊ │ │ 1997 │ 136518 │ ██████▋ │ │ 1998 │ 152983 │ ███████▋ │ -│ 1999 │ 180633 │ █████████ │ -│ 2000 │ 215830 │ ██████████▋ │ -│ 2001 │ 232996 │ ███████████▋ │ -│ 2002 │ 263672 │ █████████████▏ │ +│ 1999 │ 180637 │ █████████ │ +│ 2000 │ 215838 │ ██████████▋ │ +│ 2001 │ 232994 │ ███████████▋ │ +│ 2002 │ 263670 │ █████████████▏ │ │ 2003 │ 278394 │ █████████████▊ │ -│ 2004 │ 304665 │ ███████████████▏ │ +│ 2004 │ 304666 │ ███████████████▏ │ │ 2005 │ 322875 │ ████████████████▏ │ -│ 2006 │ 356192 │ █████████████████▋ │ -│ 2007 │ 404055 │ ████████████████████▏ │ +│ 2006 │ 356191 │ █████████████████▋ │ +│ 2007 │ 404054 │ ████████████████████▏ │ │ 2008 │ 420741 │ █████████████████████ │ -│ 2009 │ 427754 │ █████████████████████▍ │ +│ 2009 │ 427753 │ █████████████████████▍ │ │ 2010 │ 480306 │ ████████████████████████ │ │ 2011 │ 496274 │ ████████████████████████▋ │ -│ 2012 │ 519441 │ █████████████████████████▊ │ -│ 2013 │ 616209 │ ██████████████████████████████▋ │ -│ 2014 │ 724144 │ ████████████████████████████████████▏ │ -│ 2015 │ 792112 │ ███████████████████████████████████████▌ │ -│ 2016 │ 843568 │ ██████████████████████████████████████████▏ │ -│ 2017 │ 982566 │ █████████████████████████████████████████████████▏ │ -│ 2018 │ 1016845 │ ██████████████████████████████████████████████████▋ │ -│ 2019 │ 1043277 │ ████████████████████████████████████████████████████▏ │ -│ 2020 │ 1003963 │ ██████████████████████████████████████████████████▏ │ -│ 2021 │ 940794 │ ███████████████████████████████████████████████ │ +│ 2012 │ 519442 │ █████████████████████████▊ │ +│ 2013 │ 616212 │ ██████████████████████████████▋ │ +│ 2014 │ 724154 │ ████████████████████████████████████▏ │ +│ 2015 │ 792129 │ ███████████████████████████████████████▌ │ +│ 2016 │ 843655 │ ██████████████████████████████████████████▏ │ +│ 2017 │ 982642 │ █████████████████████████████████████████████████▏ │ +│ 2018 │ 1016835 │ ██████████████████████████████████████████████████▋ │ +│ 2019 │ 1042849 │ ████████████████████████████████████████████████████▏ │ +│ 2020 │ 1011889 │ ██████████████████████████████████████████████████▌ │ +│ 2021 │ 960343 │ ████████████████████████████████████████████████ │ └──────┴─────────┴───────────────────────────────────────────────────────┘ - -27 rows in set. Elapsed: 0.005 sec. Processed 106.87 thousand rows, 3.53 MB (23.49 million rows/s., 775.95 MB/s.) ``` -### The most expensive neighborhoods: the condition (date >= '2020-01-01') needs to be modified to match projection dimension (toYear(date) >= 2020) +### Query 3. The most expensive neighborhoods + +The condition (date >= '2020-01-01') needs to be modified to match projection dimension (toYear(date) >= 2020). Query: @@ -517,115 +515,129 @@ Result: ```text ┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ -│ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ -│ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ -│ LONDON │ KENSINGTON AND CHELSEA │ 2367 │ 2342422 │ ██████████████████████████████████████████████▋ │ -│ LEATHERHEAD │ ELMBRIDGE │ 108 │ 1927143 │ ██████████████████████████████████████▌ │ -│ VIRGINIA WATER │ RUNNYMEDE │ 142 │ 1868819 │ █████████████████████████████████████▍ │ -│ LONDON │ CAMDEN │ 2815 │ 1736788 │ ██████████████████████████████████▋ │ -│ THORNTON HEATH │ CROYDON │ 521 │ 1733051 │ ██████████████████████████████████▋ │ -│ WINDLESHAM │ SURREY HEATH │ 103 │ 1717255 │ ██████████████████████████████████▎ │ -│ BARNET │ ENFIELD │ 115 │ 1503458 │ ██████████████████████████████ │ -│ OXFORD │ SOUTH OXFORDSHIRE │ 298 │ 1275200 │ █████████████████████████▌ │ -│ LONDON │ ISLINGTON │ 2458 │ 1274308 │ █████████████████████████▍ │ -│ COBHAM │ ELMBRIDGE │ 364 │ 1260005 │ █████████████████████████▏ │ -│ LONDON │ HOUNSLOW │ 618 │ 1215682 │ ████████████████████████▎ │ -│ ASCOT │ WINDSOR AND MAIDENHEAD │ 379 │ 1215146 │ ████████████████████████▎ │ -│ LONDON │ RICHMOND UPON THAMES │ 654 │ 1207551 │ ████████████████████████▏ │ -│ BEACONSFIELD │ BUCKINGHAMSHIRE │ 307 │ 1186220 │ ███████████████████████▋ │ -│ RICHMOND │ RICHMOND UPON THAMES │ 805 │ 1100420 │ ██████████████████████ │ -│ LONDON │ HAMMERSMITH AND FULHAM │ 2888 │ 1062959 │ █████████████████████▎ │ -│ WEYBRIDGE │ ELMBRIDGE │ 607 │ 1027161 │ ████████████████████▌ │ -│ RADLETT │ HERTSMERE │ 265 │ 1015896 │ ████████████████████▎ │ -│ SALCOMBE │ SOUTH HAMS │ 124 │ 1014393 │ ████████████████████▎ │ -│ BURFORD │ WEST OXFORDSHIRE │ 102 │ 993100 │ ███████████████████▋ │ -│ ESHER │ ELMBRIDGE │ 454 │ 969770 │ ███████████████████▍ │ -│ HINDHEAD │ WAVERLEY │ 128 │ 967786 │ ███████████████████▎ │ -│ BROCKENHURST │ NEW FOREST │ 121 │ 967046 │ ███████████████████▎ │ -│ LEATHERHEAD │ GUILDFORD │ 191 │ 964489 │ ███████████████████▎ │ -│ GERRARDS CROSS │ BUCKINGHAMSHIRE │ 376 │ 958555 │ ███████████████████▏ │ -│ EAST MOLESEY │ ELMBRIDGE │ 181 │ 943457 │ ██████████████████▋ │ -│ OLNEY │ MILTON KEYNES │ 220 │ 942892 │ ██████████████████▋ │ -│ CHALFONT ST GILES │ BUCKINGHAMSHIRE │ 135 │ 926950 │ ██████████████████▌ │ -│ HENLEY-ON-THAMES │ SOUTH OXFORDSHIRE │ 509 │ 905732 │ ██████████████████ │ -│ KINGSTON UPON THAMES │ KINGSTON UPON THAMES │ 889 │ 899689 │ █████████████████▊ │ -│ BELVEDERE │ BEXLEY │ 313 │ 895336 │ █████████████████▊ │ -│ CRANBROOK │ TUNBRIDGE WELLS │ 404 │ 888190 │ █████████████████▋ │ -│ LONDON │ EALING │ 2460 │ 865893 │ █████████████████▎ │ -│ MAIDENHEAD │ BUCKINGHAMSHIRE │ 114 │ 863814 │ █████████████████▎ │ -│ LONDON │ MERTON │ 1958 │ 857192 │ █████████████████▏ │ -│ GUILDFORD │ WAVERLEY │ 131 │ 854447 │ █████████████████ │ -│ LONDON │ HACKNEY │ 3088 │ 846571 │ ████████████████▊ │ -│ LYMM │ WARRINGTON │ 285 │ 839920 │ ████████████████▋ │ -│ HARPENDEN │ ST ALBANS │ 606 │ 836994 │ ████████████████▋ │ -│ LONDON │ WANDSWORTH │ 6113 │ 832292 │ ████████████████▋ │ -│ LONDON │ SOUTHWARK │ 3612 │ 831319 │ ████████████████▋ │ -│ BERKHAMSTED │ DACORUM │ 502 │ 830356 │ ████████████████▌ │ -│ KINGS LANGLEY │ DACORUM │ 137 │ 821358 │ ████████████████▍ │ -│ TONBRIDGE │ TUNBRIDGE WELLS │ 339 │ 806736 │ ████████████████▏ │ -│ EPSOM │ REIGATE AND BANSTEAD │ 157 │ 805903 │ ████████████████ │ -│ WOKING │ GUILDFORD │ 161 │ 803283 │ ████████████████ │ -│ STOCKBRIDGE │ TEST VALLEY │ 168 │ 801973 │ ████████████████ │ -│ TEDDINGTON │ RICHMOND UPON THAMES │ 539 │ 798591 │ ███████████████▊ │ -│ OXFORD │ VALE OF WHITE HORSE │ 329 │ 792907 │ ███████████████▋ │ -│ LONDON │ BARNET │ 3624 │ 789583 │ ███████████████▋ │ -│ TWICKENHAM │ RICHMOND UPON THAMES │ 1090 │ 787760 │ ███████████████▋ │ -│ LUTON │ CENTRAL BEDFORDSHIRE │ 196 │ 786051 │ ███████████████▋ │ -│ TONBRIDGE │ MAIDSTONE │ 277 │ 785746 │ ███████████████▋ │ -│ TOWCESTER │ WEST NORTHAMPTONSHIRE │ 186 │ 783532 │ ███████████████▋ │ -│ LONDON │ LAMBETH │ 4832 │ 783422 │ ███████████████▋ │ -│ LUTTERWORTH │ HARBOROUGH │ 515 │ 781775 │ ███████████████▋ │ -│ WOODSTOCK │ WEST OXFORDSHIRE │ 135 │ 777499 │ ███████████████▌ │ -│ ALRESFORD │ WINCHESTER │ 196 │ 775577 │ ███████████████▌ │ -│ LONDON │ NEWHAM │ 2942 │ 768551 │ ███████████████▎ │ -│ ALDERLEY EDGE │ CHESHIRE EAST │ 168 │ 768280 │ ███████████████▎ │ -│ MARLOW │ BUCKINGHAMSHIRE │ 301 │ 762784 │ ███████████████▎ │ -│ BILLINGSHURST │ CHICHESTER │ 134 │ 760920 │ ███████████████▏ │ -│ LONDON │ TOWER HAMLETS │ 4183 │ 759635 │ ███████████████▏ │ -│ MIDHURST │ CHICHESTER │ 245 │ 759101 │ ███████████████▏ │ -│ THAMES DITTON │ ELMBRIDGE │ 227 │ 753347 │ ███████████████ │ -│ POTTERS BAR │ WELWYN HATFIELD │ 163 │ 752926 │ ███████████████ │ -│ REIGATE │ REIGATE AND BANSTEAD │ 555 │ 740961 │ ██████████████▋ │ -│ TADWORTH │ REIGATE AND BANSTEAD │ 477 │ 738997 │ ██████████████▋ │ -│ SEVENOAKS │ SEVENOAKS │ 1074 │ 734658 │ ██████████████▋ │ -│ PETWORTH │ CHICHESTER │ 138 │ 732432 │ ██████████████▋ │ -│ BOURNE END │ BUCKINGHAMSHIRE │ 127 │ 730742 │ ██████████████▌ │ -│ PURLEY │ CROYDON │ 540 │ 727721 │ ██████████████▌ │ -│ OXTED │ TANDRIDGE │ 320 │ 726078 │ ██████████████▌ │ -│ LONDON │ HARINGEY │ 2988 │ 724573 │ ██████████████▍ │ -│ BANSTEAD │ REIGATE AND BANSTEAD │ 373 │ 713834 │ ██████████████▎ │ -│ PINNER │ HARROW │ 480 │ 712166 │ ██████████████▏ │ -│ MALMESBURY │ WILTSHIRE │ 293 │ 707747 │ ██████████████▏ │ -│ RICKMANSWORTH │ THREE RIVERS │ 732 │ 705400 │ ██████████████ │ -│ SLOUGH │ BUCKINGHAMSHIRE │ 359 │ 705002 │ ██████████████ │ -│ GREAT MISSENDEN │ BUCKINGHAMSHIRE │ 214 │ 704904 │ ██████████████ │ -│ READING │ SOUTH OXFORDSHIRE │ 295 │ 701697 │ ██████████████ │ -│ HYTHE │ FOLKESTONE AND HYTHE │ 457 │ 700334 │ ██████████████ │ -│ WELWYN │ WELWYN HATFIELD │ 217 │ 699649 │ █████████████▊ │ -│ CHIGWELL │ EPPING FOREST │ 242 │ 697869 │ █████████████▊ │ -│ BARNET │ BARNET │ 906 │ 695680 │ █████████████▊ │ -│ HASLEMERE │ CHICHESTER │ 120 │ 694028 │ █████████████▊ │ -│ LEATHERHEAD │ MOLE VALLEY │ 748 │ 692026 │ █████████████▋ │ -│ LONDON │ BRENT │ 1945 │ 690799 │ █████████████▋ │ -│ HASLEMERE │ WAVERLEY │ 258 │ 690765 │ █████████████▋ │ -│ NORTHWOOD │ HILLINGDON │ 252 │ 690753 │ █████████████▋ │ -│ WALTON-ON-THAMES │ ELMBRIDGE │ 871 │ 689431 │ █████████████▋ │ -│ INGATESTONE │ BRENTWOOD │ 150 │ 688345 │ █████████████▋ │ -│ OXFORD │ OXFORD │ 1761 │ 686114 │ █████████████▋ │ -│ CHISLEHURST │ BROMLEY │ 410 │ 682892 │ █████████████▋ │ -│ KINGS LANGLEY │ THREE RIVERS │ 109 │ 682320 │ █████████████▋ │ -│ ASHTEAD │ MOLE VALLEY │ 280 │ 680483 │ █████████████▌ │ -│ WOKING │ SURREY HEATH │ 269 │ 679035 │ █████████████▌ │ -│ ASCOT │ BRACKNELL FOREST │ 160 │ 678632 │ █████████████▌ │ +│ LONDON │ CITY OF WESTMINSTER │ 3606 │ 3280239 │ █████████████████████████████████████████████████████████████████▌ │ +│ LONDON │ CITY OF LONDON │ 274 │ 3160502 │ ███████████████████████████████████████████████████████████████▏ │ +│ LONDON │ KENSINGTON AND CHELSEA │ 2550 │ 2308478 │ ██████████████████████████████████████████████▏ │ +│ LEATHERHEAD │ ELMBRIDGE │ 114 │ 1897407 │ █████████████████████████████████████▊ │ +│ LONDON │ CAMDEN │ 3033 │ 1805404 │ ████████████████████████████████████ │ +│ VIRGINIA WATER │ RUNNYMEDE │ 156 │ 1753247 │ ███████████████████████████████████ │ +│ WINDLESHAM │ SURREY HEATH │ 108 │ 1677613 │ █████████████████████████████████▌ │ +│ THORNTON HEATH │ CROYDON │ 546 │ 1671721 │ █████████████████████████████████▍ │ +│ BARNET │ ENFIELD │ 124 │ 1505840 │ ██████████████████████████████ │ +│ COBHAM │ ELMBRIDGE │ 387 │ 1237250 │ ████████████████████████▋ │ +│ LONDON │ ISLINGTON │ 2668 │ 1236980 │ ████████████████████████▋ │ +│ OXFORD │ SOUTH OXFORDSHIRE │ 321 │ 1220907 │ ████████████████████████▍ │ +│ LONDON │ RICHMOND UPON THAMES │ 704 │ 1215551 │ ████████████████████████▎ │ +│ LONDON │ HOUNSLOW │ 671 │ 1207493 │ ████████████████████████▏ │ +│ ASCOT │ WINDSOR AND MAIDENHEAD │ 407 │ 1183299 │ ███████████████████████▋ │ +│ BEACONSFIELD │ BUCKINGHAMSHIRE │ 330 │ 1175615 │ ███████████████████████▌ │ +│ RICHMOND │ RICHMOND UPON THAMES │ 874 │ 1110444 │ ██████████████████████▏ │ +│ LONDON │ HAMMERSMITH AND FULHAM │ 3086 │ 1053983 │ █████████████████████ │ +│ SURBITON │ ELMBRIDGE │ 100 │ 1011800 │ ████████████████████▏ │ +│ RADLETT │ HERTSMERE │ 283 │ 1011712 │ ████████████████████▏ │ +│ SALCOMBE │ SOUTH HAMS │ 127 │ 1011624 │ ████████████████████▏ │ +│ WEYBRIDGE │ ELMBRIDGE │ 655 │ 1007265 │ ████████████████████▏ │ +│ ESHER │ ELMBRIDGE │ 485 │ 986581 │ ███████████████████▋ │ +│ LEATHERHEAD │ GUILDFORD │ 202 │ 977320 │ ███████████████████▌ │ +│ BURFORD │ WEST OXFORDSHIRE │ 111 │ 966893 │ ███████████████████▎ │ +│ BROCKENHURST │ NEW FOREST │ 129 │ 956675 │ ███████████████████▏ │ +│ HINDHEAD │ WAVERLEY │ 137 │ 953753 │ ███████████████████ │ +│ GERRARDS CROSS │ BUCKINGHAMSHIRE │ 419 │ 951121 │ ███████████████████ │ +│ EAST MOLESEY │ ELMBRIDGE │ 192 │ 936769 │ ██████████████████▋ │ +│ CHALFONT ST GILES │ BUCKINGHAMSHIRE │ 146 │ 925515 │ ██████████████████▌ │ +│ LONDON │ TOWER HAMLETS │ 4388 │ 918304 │ ██████████████████▎ │ +│ OLNEY │ MILTON KEYNES │ 235 │ 910646 │ ██████████████████▏ │ +│ HENLEY-ON-THAMES │ SOUTH OXFORDSHIRE │ 540 │ 902418 │ ██████████████████ │ +│ LONDON │ SOUTHWARK │ 3885 │ 892997 │ █████████████████▋ │ +│ KINGSTON UPON THAMES │ KINGSTON UPON THAMES │ 960 │ 885969 │ █████████████████▋ │ +│ LONDON │ EALING │ 2658 │ 871755 │ █████████████████▍ │ +│ CRANBROOK │ TUNBRIDGE WELLS │ 431 │ 862348 │ █████████████████▏ │ +│ LONDON │ MERTON │ 2099 │ 859118 │ █████████████████▏ │ +│ BELVEDERE │ BEXLEY │ 346 │ 842423 │ ████████████████▋ │ +│ GUILDFORD │ WAVERLEY │ 143 │ 841277 │ ████████████████▋ │ +│ HARPENDEN │ ST ALBANS │ 657 │ 841216 │ ████████████████▋ │ +│ LONDON │ HACKNEY │ 3307 │ 837090 │ ████████████████▋ │ +│ LONDON │ WANDSWORTH │ 6566 │ 832663 │ ████████████████▋ │ +│ MAIDENHEAD │ BUCKINGHAMSHIRE │ 123 │ 824299 │ ████████████████▍ │ +│ KINGS LANGLEY │ DACORUM │ 145 │ 821331 │ ████████████████▍ │ +│ BERKHAMSTED │ DACORUM │ 543 │ 818415 │ ████████████████▎ │ +│ GREAT MISSENDEN │ BUCKINGHAMSHIRE │ 226 │ 802807 │ ████████████████ │ +│ BILLINGSHURST │ CHICHESTER │ 144 │ 797829 │ ███████████████▊ │ +│ WOKING │ GUILDFORD │ 176 │ 793494 │ ███████████████▋ │ +│ STOCKBRIDGE │ TEST VALLEY │ 178 │ 793269 │ ███████████████▋ │ +│ EPSOM │ REIGATE AND BANSTEAD │ 172 │ 791862 │ ███████████████▋ │ +│ TONBRIDGE │ TUNBRIDGE WELLS │ 360 │ 787876 │ ███████████████▋ │ +│ TEDDINGTON │ RICHMOND UPON THAMES │ 595 │ 786492 │ ███████████████▋ │ +│ TWICKENHAM │ RICHMOND UPON THAMES │ 1155 │ 786193 │ ███████████████▋ │ +│ LYNDHURST │ NEW FOREST │ 102 │ 785593 │ ███████████████▋ │ +│ LONDON │ LAMBETH │ 5228 │ 774574 │ ███████████████▍ │ +│ LONDON │ BARNET │ 3955 │ 773259 │ ███████████████▍ │ +│ OXFORD │ VALE OF WHITE HORSE │ 353 │ 772088 │ ███████████████▍ │ +│ TONBRIDGE │ MAIDSTONE │ 305 │ 770740 │ ███████████████▍ │ +│ LUTTERWORTH │ HARBOROUGH │ 538 │ 768634 │ ███████████████▎ │ +│ WOODSTOCK │ WEST OXFORDSHIRE │ 140 │ 766037 │ ███████████████▎ │ +│ MIDHURST │ CHICHESTER │ 257 │ 764815 │ ███████████████▎ │ +│ MARLOW │ BUCKINGHAMSHIRE │ 327 │ 761876 │ ███████████████▏ │ +│ LONDON │ NEWHAM │ 3237 │ 761784 │ ███████████████▏ │ +│ ALDERLEY EDGE │ CHESHIRE EAST │ 178 │ 757318 │ ███████████████▏ │ +│ LUTON │ CENTRAL BEDFORDSHIRE │ 212 │ 754283 │ ███████████████ │ +│ PETWORTH │ CHICHESTER │ 154 │ 754220 │ ███████████████ │ +│ ALRESFORD │ WINCHESTER │ 219 │ 752718 │ ███████████████ │ +│ POTTERS BAR │ WELWYN HATFIELD │ 174 │ 748465 │ ██████████████▊ │ +│ HASLEMERE │ CHICHESTER │ 128 │ 746907 │ ██████████████▊ │ +│ TADWORTH │ REIGATE AND BANSTEAD │ 502 │ 743252 │ ██████████████▋ │ +│ THAMES DITTON │ ELMBRIDGE │ 244 │ 741913 │ ██████████████▋ │ +│ REIGATE │ REIGATE AND BANSTEAD │ 581 │ 738198 │ ██████████████▋ │ +│ BOURNE END │ BUCKINGHAMSHIRE │ 138 │ 735190 │ ██████████████▋ │ +│ SEVENOAKS │ SEVENOAKS │ 1156 │ 730018 │ ██████████████▌ │ +│ OXTED │ TANDRIDGE │ 336 │ 729123 │ ██████████████▌ │ +│ INGATESTONE │ BRENTWOOD │ 166 │ 728103 │ ██████████████▌ │ +│ LONDON │ BRENT │ 2079 │ 720605 │ ██████████████▍ │ +│ LONDON │ HARINGEY │ 3216 │ 717780 │ ██████████████▎ │ +│ PURLEY │ CROYDON │ 575 │ 716108 │ ██████████████▎ │ +│ WELWYN │ WELWYN HATFIELD │ 222 │ 710603 │ ██████████████▏ │ +│ RICKMANSWORTH │ THREE RIVERS │ 798 │ 704571 │ ██████████████ │ +│ BANSTEAD │ REIGATE AND BANSTEAD │ 401 │ 701293 │ ██████████████ │ +│ CHIGWELL │ EPPING FOREST │ 261 │ 701203 │ ██████████████ │ +│ PINNER │ HARROW │ 528 │ 698885 │ █████████████▊ │ +│ HASLEMERE │ WAVERLEY │ 280 │ 696659 │ █████████████▊ │ +│ SLOUGH │ BUCKINGHAMSHIRE │ 396 │ 694917 │ █████████████▊ │ +│ WALTON-ON-THAMES │ ELMBRIDGE │ 946 │ 692395 │ █████████████▋ │ +│ READING │ SOUTH OXFORDSHIRE │ 318 │ 691988 │ █████████████▋ │ +│ NORTHWOOD │ HILLINGDON │ 271 │ 690643 │ █████████████▋ │ +│ FELTHAM │ HOUNSLOW │ 763 │ 688595 │ █████████████▋ │ +│ ASHTEAD │ MOLE VALLEY │ 303 │ 687923 │ █████████████▋ │ +│ BARNET │ BARNET │ 975 │ 686980 │ █████████████▋ │ +│ WOKING │ SURREY HEATH │ 283 │ 686669 │ █████████████▋ │ +│ MALMESBURY │ WILTSHIRE │ 323 │ 683324 │ █████████████▋ │ +│ AMERSHAM │ BUCKINGHAMSHIRE │ 496 │ 680962 │ █████████████▌ │ +│ CHISLEHURST │ BROMLEY │ 430 │ 680209 │ █████████████▌ │ +│ HYTHE │ FOLKESTONE AND HYTHE │ 490 │ 676908 │ █████████████▌ │ +│ MAYFIELD │ WEALDEN │ 101 │ 676210 │ █████████████▌ │ +│ ASCOT │ BRACKNELL FOREST │ 168 │ 676004 │ █████████████▌ │ └──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ - -100 rows in set. Elapsed: 0.005 sec. Processed 12.85 thousand rows, 813.40 KB (2.73 million rows/s., 172.95 MB/s.) ``` All 3 queries work much faster and read fewer rows. +```text +Query 1 + +no projection: 27 rows in set. Elapsed: 0.158 sec. Processed 26.32 million rows, 157.93 MB (166.57 million rows/s., 999.39 MB/s.) + projection: 27 rows in set. Elapsed: 0.007 sec. Processed 105.96 thousand rows, 3.33 MB (14.58 million rows/s., 458.13 MB/s.) + + +Query 2 + +no projection: 27 rows in set. Elapsed: 0.163 sec. Processed 26.32 million rows, 80.01 MB (161.75 million rows/s., 491.64 MB/s.) + projection: 27 rows in set. Elapsed: 0.008 sec. Processed 105.96 thousand rows, 3.67 MB (13.29 million rows/s., 459.89 MB/s.) + +Query 3 + +no projection: 100 rows in set. Elapsed: 0.069 sec. Processed 26.32 million rows, 62.47 MB (382.13 million rows/s., 906.93 MB/s.) + projection: 100 rows in set. Elapsed: 0.029 sec. Processed 8.08 thousand rows, 511.08 KB (276.06 thousand rows/s., 17.47 MB/s.) ``` -Q1) -no projection: 27 rows in set. Elapsed: 0.027 sec. Processed 26.25 million rows, 157.49 MB (955.96 million rows/s., 5.74 GB/s.) - projection: 27 rows in set. Elapsed: 0.003 sec. Processed 106.87 thousand rows, 3.21 MB (31.92 million rows/s., 959.03 MB/s.) -``` + +### Test it in Playground + +The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). From b37168e86fb847afa77103c0a99555e49d10b742 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Sep 2021 14:27:15 +0300 Subject: [PATCH 028/142] Fix asan. --- src/Processors/Chain.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Chain.h b/src/Processors/Chain.h index 08a812a8227..9f27e37cc40 100644 --- a/src/Processors/Chain.h +++ b/src/Processors/Chain.h @@ -46,11 +46,12 @@ public: void reset(); private: + PipelineResourcesHolder holder; + /// -> source -> transform -> ... -> transform -> sink -> /// ^ -> -> -> -> ^ /// input port output port std::list processors; - PipelineResourcesHolder holder; size_t num_threads = 0; }; From 3ce302c76c5b6d4f86f8b3274b809c5c61d4e59f Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 13 Sep 2021 17:29:51 +0300 Subject: [PATCH 029/142] Update UK price paid. Update UK price paid. --- .../example-datasets/opensky.md | 4 +- .../example-datasets/uk-price-paid.md | 32 +- .../example-datasets/uk-price-paid.md | 732 +++++++++--------- 3 files changed, 395 insertions(+), 373 deletions(-) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md index b74395e120e..8d9cfd99ec9 100644 --- a/docs/en/getting-started/example-datasets/opensky.md +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -3,7 +3,7 @@ toc_priority: 20 toc_title: OpenSky --- -# Crowdsourced air traffic data from The OpenSky Network 2020 +# Crowdsourced air traffic data from The OpenSky Network 2020 > The data in this dataset is derived and cleaned from the full OpenSky dataset to illustrate the development of air traffic during the COVID-19 pandemic. It spans all flights seen by the network's more than 2500 members since 1 January 2019. More data will be periodically included in the dataset until the end of the COVID-19 pandemic. @@ -414,4 +414,4 @@ Result: ### Online Playground -The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICBvcmlnaW4sCiAgICBjb3VudCgpLAogICAgcm91bmQoYXZnKGdlb0Rpc3RhbmNlKGxvbmdpdHVkZV8xLCBsYXRpdHVkZV8xLCBsb25naXR1ZGVfMiwgbGF0aXR1ZGVfMikpKSBBUyBkaXN0YW5jZSwKICAgIGJhcihkaXN0YW5jZSwgMCwgMTAwMDAwMDAsIDEwMCkgQVMgYmFyCkZST00gb3BlbnNreQpXSEVSRSBvcmlnaW4gIT0gJycKR1JPVVAgQlkgb3JpZ2luCk9SREVSIEJZIGNvdW50KCkgREVTQwpMSU1JVCAxMDA=). +You can test other queries to this data set using the interactive resource [Online Playground](https://gh-api.clickhouse.tech/play?user=play). For example, [like this](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICBvcmlnaW4sCiAgICBjb3VudCgpLAogICAgcm91bmQoYXZnKGdlb0Rpc3RhbmNlKGxvbmdpdHVkZV8xLCBsYXRpdHVkZV8xLCBsb25naXR1ZGVfMiwgbGF0aXR1ZGVfMikpKSBBUyBkaXN0YW5jZSwKICAgIGJhcihkaXN0YW5jZSwgMCwgMTAwMDAwMDAsIDEwMCkgQVMgYmFyCkZST00gb3BlbnNreQpXSEVSRSBvcmlnaW4gIT0gJycKR1JPVVAgQlkgb3JpZ2luCk9SREVSIEJZIGNvdW50KCkgREVTQwpMSU1JVCAxMDA=). However, please note that you cannot create temporary tables here. diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index bdb8e7ba2cf..d1053f008ca 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -6,7 +6,7 @@ toc_title: UK Property Price Paid # UK Property Price Paid The dataset contains data about prices paid for real-estate property in England and Wales. The data is available since year 1995. -The size of the dataset in uncompressed form is about 4 GiB and it will take about 226 MiB in ClickHouse. +The size of the dataset in uncompressed form is about 4 GiB and it will take about 278 MiB in ClickHouse. Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data @@ -15,6 +15,8 @@ Contains HM Land Registry data © Crown copyright and database right 2021. This ## Download the Dataset +Run the command: + ```bash wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv ``` @@ -114,7 +116,7 @@ Result: │ 26321785 │ └──────────┘ -The size of dataset in ClickHouse is just 278 MiB: +The size of dataset in ClickHouse is just 278 MiB, check it. Query: @@ -132,7 +134,7 @@ Result: ## Run Some Queries -### Average price per year: +### Query 1. Average Price Per Year Query: @@ -174,7 +176,7 @@ Result: └──────┴────────┴────────────────────────────────────────┘ ``` -### Average price per year in London: +### Query 2. Average Price per Year in London Query: @@ -218,7 +220,7 @@ Result: Something happened in 2013. I don't have a clue. Maybe you have a clue what happened in 2020? -### The most expensive neighborhoods: +### Query 3. The Most Expensive Neighborhoods Query: @@ -347,13 +349,13 @@ Result: └──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ ``` -## Let's speed up queries using projections +## Let's Speed Up Queries Using Projections [Projections](https://../../sql-reference/statements/alter/projection/) allow to improve queries speed by storing pre-aggregated data. -### Build a projection +### Build a Projection -Create an aggregate projection by dimensions `toYear(date)`, `district`, `town`. +Create an aggregate projection by dimensions `toYear(date)`, `district`, `town`: ```sql ALTER TABLE uk_price_paid @@ -373,7 +375,7 @@ ALTER TABLE uk_price_paid ); ``` -Populate the projection for existing data (without it projection will be created for only newly inserted data). +Populate the projection for existing data (without it projection will be created for only newly inserted data): ```sql ALTER TABLE uk_price_paid @@ -381,17 +383,17 @@ ALTER TABLE uk_price_paid SETTINGS mutations_sync = 1; ``` -## Test performance +## Test Performance Let's run the same 3 queries. -Enable projections for selects +Enable projections for selects: ```sql SET allow_experimental_projection_optimization = 1; ``` -### Query 1. Average price per year +### Query 1. Average Price Per Year Query: @@ -439,7 +441,7 @@ Result: └──────┴────────┴────────────────────────────────────────┘ ``` -### Query 2. Average price per year in London +### Query 2. Average Price Per Year in London Query: @@ -488,7 +490,7 @@ Result: └──────┴─────────┴───────────────────────────────────────────────────────┘ ``` -### Query 3. The most expensive neighborhoods +### Query 3. The Most Expensive Neighborhoods The condition (date >= '2020-01-01') needs to be modified to match projection dimension (toYear(date) >= 2020). @@ -638,6 +640,6 @@ no projection: 100 rows in set. Elapsed: 0.069 sec. Processed 26.32 million rows projection: 100 rows in set. Elapsed: 0.029 sec. Processed 8.08 thousand rows, 511.08 KB (276.06 thousand rows/s., 17.47 MB/s.) ``` -### Test it in Playground +### Test It in Playground The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index ad17fe87371..7f5f383a844 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -1,27 +1,35 @@ --- toc_priority: 20 -toc_title: UK Property Price Paid +toc_title: Данные о недвижимости в Великобритании --- -# UK Property Price Paid +# Данные о недвижимости в Великобритании -The dataset contains data about prices paid for real-estate property in England and Wales. The data is available since year 1995. -The size of the dataset in uncompressed form is about 4 GiB and it will take about 226 MiB in ClickHouse. +Набор данных содержит данные о ценах, уплаченных за недвижимость в Англии и Уэльсе. +Данные доступны с 1995 года. +Размер набора данных в несжатом виде составляет около 4 гигабайт, а в ClickHouse он займет около 278 Мбайт. -Source: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads -Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data -Contains HM Land Registry data © Crown copyright and database right 2021. This data is licensed under the Open Government Licence v3.0. +Источник: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads +Описание полей таблицы: https://www.gov.uk/guidance/about-the-price-paid-data -## Download the Dataset +Содержит данные HM Land Registry data © Авторское право короны и право на базу данных 2021. + +Crown Авторское право Короны и право на базу данных 2021. Эти данные лицензированы в соответствии с лицензией Open Government Licence v3.0. + + + +## Загрузите набор данных + +Выполните команду: ```bash wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv ``` -Download will take about 2 minutes with good internet connection. +Загрузка займет около 2 минут при хорошем подключении к Интернету. -## Create the Table +## Создайте таблицу ```sql CREATE TABLE uk_price_paid @@ -44,20 +52,20 @@ CREATE TABLE uk_price_paid ) ENGINE = MergeTree ORDER BY (postcode1, postcode2, addr1, addr2); ``` -## Preprocess and Import Data +## Обработайте и импортируйте данные -We will use `clickhouse-local` tool for data preprocessing and `clickhouse-client` to upload it. +Мы будем использовать инструмент `clickhouse-local` для предварительной обработки данных и `clickhouse-client` для загрузки данных. -In this example, we define the structure of source data from the CSV file and specify a query to preprocess the data with `clickhouse-local`. +В этом примере мы определяем структуру исходных данных из CSV-файла и указываем запрос для предварительной обработки данных с помощью `clickhouse-local`. -The preprocessing is: -- splitting the postcode to two different columns `postcode1` and `postcode2` that is better for storage and queries; -- coverting the `time` field to date as it only contains 00:00 time; -- ignoring the `uuid` field because we don't need it for analysis; -- transforming `type` and `duration` to more readable Enum fields with function `transform`; -- transforming `is_new` and `category` fields from single-character string (`Y`/`N` and `A`/`B`) to UInt8 field with 0 and 1. +Предварительная обработка включает: +- разделение почтового индекса на два разных столбца `postcode1` и `postcode2`, что лучше подходит для хранения данных и выполнения запросов к ним; +- преобразование поля `time` в `Date`, поскольку оно содержит только время 00:00; +- поле `uuid` игнорируется, потому что оно не будет использовано нами для анализа; +- преобразование полей `type` and `duration` в более читаемые поля типа `Enum` с помощью функции `transform`; +- преобразование полей `is_new` и `category` из односимвольной строки (`Y`/`N` и `A`/`B`) в поле `UInt8` со значениями 0 и 1 соответственно. -Preprocessed data is piped directly to `clickhouse-client` to be inserted into ClickHouse table in streaming fashion. +Предварительно обработанные данные передаются непосредственно в `clickhouse-client` для вставки в таблицу ClickHouse потоковым способом. ```bash clickhouse-local --input-format CSV --structure ' @@ -98,127 +106,129 @@ clickhouse-local --input-format CSV --structure ' FROM table" --date_time_input_format best_effort < pp-complete.csv | clickhouse-client --query "INSERT INTO uk_price_paid FORMAT TSV" ``` -It will take about 40 seconds. +Выполнение запроса займет около 40 секунд. -## Validate the Data +## Проверьте импортированные данные -Query: +Запрос: ```sql SELECT count() FROM uk_price_paid; ``` -Result: +Результат: -```text -26248711 -``` +┌──count()─┐ +│ 26321785 │ +└──────────┘ -The size of dataset in ClickHouse is just 226 MiB: +Размер набора данных в ClickHouse составляет всего 278 МБ, проверьте это. -Query: +Запрос: ```sql SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price_paid'; ``` -Result: +Результат: ```text -226.40 MiB +┌─formatReadableSize(total_bytes)─┐ +│ 278.80 MiB │ +└─────────────────────────────────┘ ``` -## Run Some Queries +## Примеры запросов -### Average price per year +### Запрос 1. Средняя цена за год -Query: +Запрос: ```sql SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, 80) FROM uk_price_paid GROUP BY year ORDER BY year; ``` -Result: +Результат: ```text ┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ │ 1995 │ 67932 │ █████▍ │ │ 1996 │ 71505 │ █████▋ │ │ 1997 │ 78532 │ ██████▎ │ -│ 1998 │ 85435 │ ██████▋ │ -│ 1999 │ 96036 │ ███████▋ │ -│ 2000 │ 107478 │ ████████▌ │ -│ 2001 │ 118886 │ █████████▌ │ -│ 2002 │ 137940 │ ███████████ │ -│ 2003 │ 155888 │ ████████████▍ │ +│ 1998 │ 85436 │ ██████▋ │ +│ 1999 │ 96037 │ ███████▋ │ +│ 2000 │ 107479 │ ████████▌ │ +│ 2001 │ 118885 │ █████████▌ │ +│ 2002 │ 137941 │ ███████████ │ +│ 2003 │ 155889 │ ████████████▍ │ │ 2004 │ 178885 │ ██████████████▎ │ -│ 2005 │ 189350 │ ███████████████▏ │ +│ 2005 │ 189351 │ ███████████████▏ │ │ 2006 │ 203528 │ ████████████████▎ │ -│ 2007 │ 219377 │ █████████████████▌ │ +│ 2007 │ 219378 │ █████████████████▌ │ │ 2008 │ 217056 │ █████████████████▎ │ │ 2009 │ 213419 │ █████████████████ │ -│ 2010 │ 236110 │ ██████████████████▊ │ -│ 2011 │ 232804 │ ██████████████████▌ │ -│ 2012 │ 238366 │ ███████████████████ │ +│ 2010 │ 236109 │ ██████████████████▊ │ +│ 2011 │ 232805 │ ██████████████████▌ │ +│ 2012 │ 238367 │ ███████████████████ │ │ 2013 │ 256931 │ ████████████████████▌ │ -│ 2014 │ 279917 │ ██████████████████████▍ │ -│ 2015 │ 297264 │ ███████████████████████▋ │ -│ 2016 │ 313197 │ █████████████████████████ │ -│ 2017 │ 346070 │ ███████████████████████████▋ │ -│ 2018 │ 350117 │ ████████████████████████████ │ -│ 2019 │ 351010 │ ████████████████████████████ │ -│ 2020 │ 368974 │ █████████████████████████████▌ │ -│ 2021 │ 384351 │ ██████████████████████████████▋ │ +│ 2014 │ 279915 │ ██████████████████████▍ │ +│ 2015 │ 297266 │ ███████████████████████▋ │ +│ 2016 │ 313201 │ █████████████████████████ │ +│ 2017 │ 346097 │ ███████████████████████████▋ │ +│ 2018 │ 350116 │ ████████████████████████████ │ +│ 2019 │ 351013 │ ████████████████████████████ │ +│ 2020 │ 369420 │ █████████████████████████████▌ │ +│ 2021 │ 386903 │ ██████████████████████████████▊ │ └──────┴────────┴────────────────────────────────────────┘ - -27 rows in set. Elapsed: 0.027 sec. Processed 26.25 million rows, 157.49 MB (955.96 million rows/s., 5.74 GB/s.) ``` -### Average price per year in London: +### Запрос 2. Средняя цена за год в Лондоне -Query: +Запрос: ```sql SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, 100) FROM uk_price_paid WHERE town = 'LONDON' GROUP BY year ORDER BY year; ``` +Результат: + ```text ┌─year─┬───price─┬─bar(round(avg(price)), 0, 2000000, 100)───────────────┐ -│ 1995 │ 109112 │ █████▍ │ +│ 1995 │ 109116 │ █████▍ │ │ 1996 │ 118667 │ █████▊ │ │ 1997 │ 136518 │ ██████▋ │ │ 1998 │ 152983 │ ███████▋ │ -│ 1999 │ 180633 │ █████████ │ -│ 2000 │ 215830 │ ██████████▋ │ -│ 2001 │ 232996 │ ███████████▋ │ -│ 2002 │ 263672 │ █████████████▏ │ +│ 1999 │ 180637 │ █████████ │ +│ 2000 │ 215838 │ ██████████▋ │ +│ 2001 │ 232994 │ ███████████▋ │ +│ 2002 │ 263670 │ █████████████▏ │ │ 2003 │ 278394 │ █████████████▊ │ -│ 2004 │ 304665 │ ███████████████▏ │ +│ 2004 │ 304666 │ ███████████████▏ │ │ 2005 │ 322875 │ ████████████████▏ │ -│ 2006 │ 356192 │ █████████████████▋ │ -│ 2007 │ 404055 │ ████████████████████▏ │ +│ 2006 │ 356191 │ █████████████████▋ │ +│ 2007 │ 404054 │ ████████████████████▏ │ │ 2008 │ 420741 │ █████████████████████ │ -│ 2009 │ 427754 │ █████████████████████▍ │ +│ 2009 │ 427753 │ █████████████████████▍ │ │ 2010 │ 480306 │ ████████████████████████ │ │ 2011 │ 496274 │ ████████████████████████▋ │ -│ 2012 │ 519441 │ █████████████████████████▊ │ -│ 2013 │ 616209 │ ██████████████████████████████▋ │ -│ 2014 │ 724144 │ ████████████████████████████████████▏ │ -│ 2015 │ 792112 │ ███████████████████████████████████████▌ │ -│ 2016 │ 843568 │ ██████████████████████████████████████████▏ │ -│ 2017 │ 982566 │ █████████████████████████████████████████████████▏ │ -│ 2018 │ 1016845 │ ██████████████████████████████████████████████████▋ │ -│ 2019 │ 1043277 │ ████████████████████████████████████████████████████▏ │ -│ 2020 │ 1003963 │ ██████████████████████████████████████████████████▏ │ -│ 2021 │ 940794 │ ███████████████████████████████████████████████ │ +│ 2012 │ 519442 │ █████████████████████████▊ │ +│ 2013 │ 616212 │ ██████████████████████████████▋ │ +│ 2014 │ 724154 │ ████████████████████████████████████▏ │ +│ 2015 │ 792129 │ ███████████████████████████████████████▌ │ +│ 2016 │ 843655 │ ██████████████████████████████████████████▏ │ +│ 2017 │ 982642 │ █████████████████████████████████████████████████▏ │ +│ 2018 │ 1016835 │ ██████████████████████████████████████████████████▋ │ +│ 2019 │ 1042849 │ ████████████████████████████████████████████████████▏ │ +│ 2020 │ 1011889 │ ██████████████████████████████████████████████████▌ │ +│ 2021 │ 960343 │ ████████████████████████████████████████████████ │ └──────┴─────────┴───────────────────────────────────────────────────────┘ - -27 rows in set. Elapsed: 0.024 sec. Processed 26.25 million rows, 76.88 MB (1.08 billion rows/s., 3.15 GB/s.) ``` -Something happened in 2013. I don't have a clue. Maybe you have a clue what happened in 2020? +Что-то случилось в 2013 году. Я понятия не имею. Может быть, вы имеете представление о том, что произошло в 2020 году? -### The most expensive neighborhoods: +### Запрос 3. Самые дорогие районы + +Запрос: ```sql SELECT @@ -237,124 +247,121 @@ ORDER BY price DESC LIMIT 100; ``` -```text -┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ -│ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ -│ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ -│ LONDON │ KENSINGTON AND CHELSEA │ 2367 │ 2342422 │ ██████████████████████████████████████████████▋ │ -│ LEATHERHEAD │ ELMBRIDGE │ 108 │ 1927143 │ ██████████████████████████████████████▌ │ -│ VIRGINIA WATER │ RUNNYMEDE │ 142 │ 1868819 │ █████████████████████████████████████▍ │ -│ LONDON │ CAMDEN │ 2815 │ 1736788 │ ██████████████████████████████████▋ │ -│ THORNTON HEATH │ CROYDON │ 521 │ 1733051 │ ██████████████████████████████████▋ │ -│ WINDLESHAM │ SURREY HEATH │ 103 │ 1717255 │ ██████████████████████████████████▎ │ -│ BARNET │ ENFIELD │ 115 │ 1503458 │ ██████████████████████████████ │ -│ OXFORD │ SOUTH OXFORDSHIRE │ 298 │ 1275200 │ █████████████████████████▌ │ -│ LONDON │ ISLINGTON │ 2458 │ 1274308 │ █████████████████████████▍ │ -│ COBHAM │ ELMBRIDGE │ 364 │ 1260005 │ █████████████████████████▏ │ -│ LONDON │ HOUNSLOW │ 618 │ 1215682 │ ████████████████████████▎ │ -│ ASCOT │ WINDSOR AND MAIDENHEAD │ 379 │ 1215146 │ ████████████████████████▎ │ -│ LONDON │ RICHMOND UPON THAMES │ 654 │ 1207551 │ ████████████████████████▏ │ -│ BEACONSFIELD │ BUCKINGHAMSHIRE │ 307 │ 1186220 │ ███████████████████████▋ │ -│ RICHMOND │ RICHMOND UPON THAMES │ 805 │ 1100420 │ ██████████████████████ │ -│ LONDON │ HAMMERSMITH AND FULHAM │ 2888 │ 1062959 │ █████████████████████▎ │ -│ WEYBRIDGE │ ELMBRIDGE │ 607 │ 1027161 │ ████████████████████▌ │ -│ RADLETT │ HERTSMERE │ 265 │ 1015896 │ ████████████████████▎ │ -│ SALCOMBE │ SOUTH HAMS │ 124 │ 1014393 │ ████████████████████▎ │ -│ BURFORD │ WEST OXFORDSHIRE │ 102 │ 993100 │ ███████████████████▋ │ -│ ESHER │ ELMBRIDGE │ 454 │ 969770 │ ███████████████████▍ │ -│ HINDHEAD │ WAVERLEY │ 128 │ 967786 │ ███████████████████▎ │ -│ BROCKENHURST │ NEW FOREST │ 121 │ 967046 │ ███████████████████▎ │ -│ LEATHERHEAD │ GUILDFORD │ 191 │ 964489 │ ███████████████████▎ │ -│ GERRARDS CROSS │ BUCKINGHAMSHIRE │ 376 │ 958555 │ ███████████████████▏ │ -│ EAST MOLESEY │ ELMBRIDGE │ 181 │ 943457 │ ██████████████████▋ │ -│ OLNEY │ MILTON KEYNES │ 220 │ 942892 │ ██████████████████▋ │ -│ CHALFONT ST GILES │ BUCKINGHAMSHIRE │ 135 │ 926950 │ ██████████████████▌ │ -│ HENLEY-ON-THAMES │ SOUTH OXFORDSHIRE │ 509 │ 905732 │ ██████████████████ │ -│ KINGSTON UPON THAMES │ KINGSTON UPON THAMES │ 889 │ 899689 │ █████████████████▊ │ -│ BELVEDERE │ BEXLEY │ 313 │ 895336 │ █████████████████▊ │ -│ CRANBROOK │ TUNBRIDGE WELLS │ 404 │ 888190 │ █████████████████▋ │ -│ LONDON │ EALING │ 2460 │ 865893 │ █████████████████▎ │ -│ MAIDENHEAD │ BUCKINGHAMSHIRE │ 114 │ 863814 │ █████████████████▎ │ -│ LONDON │ MERTON │ 1958 │ 857192 │ █████████████████▏ │ -│ GUILDFORD │ WAVERLEY │ 131 │ 854447 │ █████████████████ │ -│ LONDON │ HACKNEY │ 3088 │ 846571 │ ████████████████▊ │ -│ LYMM │ WARRINGTON │ 285 │ 839920 │ ████████████████▋ │ -│ HARPENDEN │ ST ALBANS │ 606 │ 836994 │ ████████████████▋ │ -│ LONDON │ WANDSWORTH │ 6113 │ 832292 │ ████████████████▋ │ -│ LONDON │ SOUTHWARK │ 3612 │ 831319 │ ████████████████▋ │ -│ BERKHAMSTED │ DACORUM │ 502 │ 830356 │ ████████████████▌ │ -│ KINGS LANGLEY │ DACORUM │ 137 │ 821358 │ ████████████████▍ │ -│ TONBRIDGE │ TUNBRIDGE WELLS │ 339 │ 806736 │ ████████████████▏ │ -│ EPSOM │ REIGATE AND BANSTEAD │ 157 │ 805903 │ ████████████████ │ -│ WOKING │ GUILDFORD │ 161 │ 803283 │ ████████████████ │ -│ STOCKBRIDGE │ TEST VALLEY │ 168 │ 801973 │ ████████████████ │ -│ TEDDINGTON │ RICHMOND UPON THAMES │ 539 │ 798591 │ ███████████████▊ │ -│ OXFORD │ VALE OF WHITE HORSE │ 329 │ 792907 │ ███████████████▋ │ -│ LONDON │ BARNET │ 3624 │ 789583 │ ███████████████▋ │ -│ TWICKENHAM │ RICHMOND UPON THAMES │ 1090 │ 787760 │ ███████████████▋ │ -│ LUTON │ CENTRAL BEDFORDSHIRE │ 196 │ 786051 │ ███████████████▋ │ -│ TONBRIDGE │ MAIDSTONE │ 277 │ 785746 │ ███████████████▋ │ -│ TOWCESTER │ WEST NORTHAMPTONSHIRE │ 186 │ 783532 │ ███████████████▋ │ -│ LONDON │ LAMBETH │ 4832 │ 783422 │ ███████████████▋ │ -│ LUTTERWORTH │ HARBOROUGH │ 515 │ 781775 │ ███████████████▋ │ -│ WOODSTOCK │ WEST OXFORDSHIRE │ 135 │ 777499 │ ███████████████▌ │ -│ ALRESFORD │ WINCHESTER │ 196 │ 775577 │ ███████████████▌ │ -│ LONDON │ NEWHAM │ 2942 │ 768551 │ ███████████████▎ │ -│ ALDERLEY EDGE │ CHESHIRE EAST │ 168 │ 768280 │ ███████████████▎ │ -│ MARLOW │ BUCKINGHAMSHIRE │ 301 │ 762784 │ ███████████████▎ │ -│ BILLINGSHURST │ CHICHESTER │ 134 │ 760920 │ ███████████████▏ │ -│ LONDON │ TOWER HAMLETS │ 4183 │ 759635 │ ███████████████▏ │ -│ MIDHURST │ CHICHESTER │ 245 │ 759101 │ ███████████████▏ │ -│ THAMES DITTON │ ELMBRIDGE │ 227 │ 753347 │ ███████████████ │ -│ POTTERS BAR │ WELWYN HATFIELD │ 163 │ 752926 │ ███████████████ │ -│ REIGATE │ REIGATE AND BANSTEAD │ 555 │ 740961 │ ██████████████▋ │ -│ TADWORTH │ REIGATE AND BANSTEAD │ 477 │ 738997 │ ██████████████▋ │ -│ SEVENOAKS │ SEVENOAKS │ 1074 │ 734658 │ ██████████████▋ │ -│ PETWORTH │ CHICHESTER │ 138 │ 732432 │ ██████████████▋ │ -│ BOURNE END │ BUCKINGHAMSHIRE │ 127 │ 730742 │ ██████████████▌ │ -│ PURLEY │ CROYDON │ 540 │ 727721 │ ██████████████▌ │ -│ OXTED │ TANDRIDGE │ 320 │ 726078 │ ██████████████▌ │ -│ LONDON │ HARINGEY │ 2988 │ 724573 │ ██████████████▍ │ -│ BANSTEAD │ REIGATE AND BANSTEAD │ 373 │ 713834 │ ██████████████▎ │ -│ PINNER │ HARROW │ 480 │ 712166 │ ██████████████▏ │ -│ MALMESBURY │ WILTSHIRE │ 293 │ 707747 │ ██████████████▏ │ -│ RICKMANSWORTH │ THREE RIVERS │ 732 │ 705400 │ ██████████████ │ -│ SLOUGH │ BUCKINGHAMSHIRE │ 359 │ 705002 │ ██████████████ │ -│ GREAT MISSENDEN │ BUCKINGHAMSHIRE │ 214 │ 704904 │ ██████████████ │ -│ READING │ SOUTH OXFORDSHIRE │ 295 │ 701697 │ ██████████████ │ -│ HYTHE │ FOLKESTONE AND HYTHE │ 457 │ 700334 │ ██████████████ │ -│ WELWYN │ WELWYN HATFIELD │ 217 │ 699649 │ █████████████▊ │ -│ CHIGWELL │ EPPING FOREST │ 242 │ 697869 │ █████████████▊ │ -│ BARNET │ BARNET │ 906 │ 695680 │ █████████████▊ │ -│ HASLEMERE │ CHICHESTER │ 120 │ 694028 │ █████████████▊ │ -│ LEATHERHEAD │ MOLE VALLEY │ 748 │ 692026 │ █████████████▋ │ -│ LONDON │ BRENT │ 1945 │ 690799 │ █████████████▋ │ -│ HASLEMERE │ WAVERLEY │ 258 │ 690765 │ █████████████▋ │ -│ NORTHWOOD │ HILLINGDON │ 252 │ 690753 │ █████████████▋ │ -│ WALTON-ON-THAMES │ ELMBRIDGE │ 871 │ 689431 │ █████████████▋ │ -│ INGATESTONE │ BRENTWOOD │ 150 │ 688345 │ █████████████▋ │ -│ OXFORD │ OXFORD │ 1761 │ 686114 │ █████████████▋ │ -│ CHISLEHURST │ BROMLEY │ 410 │ 682892 │ █████████████▋ │ -│ KINGS LANGLEY │ THREE RIVERS │ 109 │ 682320 │ █████████████▋ │ -│ ASHTEAD │ MOLE VALLEY │ 280 │ 680483 │ █████████████▌ │ -│ WOKING │ SURREY HEATH │ 269 │ 679035 │ █████████████▌ │ -│ ASCOT │ BRACKNELL FOREST │ 160 │ 678632 │ █████████████▌ │ -└──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ +Результат: -100 rows in set. Elapsed: 0.039 sec. Processed 26.25 million rows, 278.03 MB (674.32 million rows/s., 7.14 GB/s.) +```text + +┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ +│ LONDON │ CITY OF WESTMINSTER │ 3606 │ 3280239 │ █████████████████████████████████████████████████████████████████▌ │ +│ LONDON │ CITY OF LONDON │ 274 │ 3160502 │ ███████████████████████████████████████████████████████████████▏ │ +│ LONDON │ KENSINGTON AND CHELSEA │ 2550 │ 2308478 │ ██████████████████████████████████████████████▏ │ +│ LEATHERHEAD │ ELMBRIDGE │ 114 │ 1897407 │ █████████████████████████████████████▊ │ +│ LONDON │ CAMDEN │ 3033 │ 1805404 │ ████████████████████████████████████ │ +│ VIRGINIA WATER │ RUNNYMEDE │ 156 │ 1753247 │ ███████████████████████████████████ │ +│ WINDLESHAM │ SURREY HEATH │ 108 │ 1677613 │ █████████████████████████████████▌ │ +│ THORNTON HEATH │ CROYDON │ 546 │ 1671721 │ █████████████████████████████████▍ │ +│ BARNET │ ENFIELD │ 124 │ 1505840 │ ██████████████████████████████ │ +│ COBHAM │ ELMBRIDGE │ 387 │ 1237250 │ ████████████████████████▋ │ +│ LONDON │ ISLINGTON │ 2668 │ 1236980 │ ████████████████████████▋ │ +│ OXFORD │ SOUTH OXFORDSHIRE │ 321 │ 1220907 │ ████████████████████████▍ │ +│ LONDON │ RICHMOND UPON THAMES │ 704 │ 1215551 │ ████████████████████████▎ │ +│ LONDON │ HOUNSLOW │ 671 │ 1207493 │ ████████████████████████▏ │ +│ ASCOT │ WINDSOR AND MAIDENHEAD │ 407 │ 1183299 │ ███████████████████████▋ │ +│ BEACONSFIELD │ BUCKINGHAMSHIRE │ 330 │ 1175615 │ ███████████████████████▌ │ +│ RICHMOND │ RICHMOND UPON THAMES │ 874 │ 1110444 │ ██████████████████████▏ │ +│ LONDON │ HAMMERSMITH AND FULHAM │ 3086 │ 1053983 │ █████████████████████ │ +│ SURBITON │ ELMBRIDGE │ 100 │ 1011800 │ ████████████████████▏ │ +│ RADLETT │ HERTSMERE │ 283 │ 1011712 │ ████████████████████▏ │ +│ SALCOMBE │ SOUTH HAMS │ 127 │ 1011624 │ ████████████████████▏ │ +│ WEYBRIDGE │ ELMBRIDGE │ 655 │ 1007265 │ ████████████████████▏ │ +│ ESHER │ ELMBRIDGE │ 485 │ 986581 │ ███████████████████▋ │ +│ LEATHERHEAD │ GUILDFORD │ 202 │ 977320 │ ███████████████████▌ │ +│ BURFORD │ WEST OXFORDSHIRE │ 111 │ 966893 │ ███████████████████▎ │ +│ BROCKENHURST │ NEW FOREST │ 129 │ 956675 │ ███████████████████▏ │ +│ HINDHEAD │ WAVERLEY │ 137 │ 953753 │ ███████████████████ │ +│ GERRARDS CROSS │ BUCKINGHAMSHIRE │ 419 │ 951121 │ ███████████████████ │ +│ EAST MOLESEY │ ELMBRIDGE │ 192 │ 936769 │ ██████████████████▋ │ +│ CHALFONT ST GILES │ BUCKINGHAMSHIRE │ 146 │ 925515 │ ██████████████████▌ │ +│ LONDON │ TOWER HAMLETS │ 4388 │ 918304 │ ██████████████████▎ │ +│ OLNEY │ MILTON KEYNES │ 235 │ 910646 │ ██████████████████▏ │ +│ HENLEY-ON-THAMES │ SOUTH OXFORDSHIRE │ 540 │ 902418 │ ██████████████████ │ +│ LONDON │ SOUTHWARK │ 3885 │ 892997 │ █████████████████▋ │ +│ KINGSTON UPON THAMES │ KINGSTON UPON THAMES │ 960 │ 885969 │ █████████████████▋ │ +│ LONDON │ EALING │ 2658 │ 871755 │ █████████████████▍ │ +│ CRANBROOK │ TUNBRIDGE WELLS │ 431 │ 862348 │ █████████████████▏ │ +│ LONDON │ MERTON │ 2099 │ 859118 │ █████████████████▏ │ +│ BELVEDERE │ BEXLEY │ 346 │ 842423 │ ████████████████▋ │ +│ GUILDFORD │ WAVERLEY │ 143 │ 841277 │ ████████████████▋ │ +│ HARPENDEN │ ST ALBANS │ 657 │ 841216 │ ████████████████▋ │ +│ LONDON │ HACKNEY │ 3307 │ 837090 │ ████████████████▋ │ +│ LONDON │ WANDSWORTH │ 6566 │ 832663 │ ████████████████▋ │ +│ MAIDENHEAD │ BUCKINGHAMSHIRE │ 123 │ 824299 │ ████████████████▍ │ +│ KINGS LANGLEY │ DACORUM │ 145 │ 821331 │ ████████████████▍ │ +│ BERKHAMSTED │ DACORUM │ 543 │ 818415 │ ████████████████▎ │ +│ GREAT MISSENDEN │ BUCKINGHAMSHIRE │ 226 │ 802807 │ ████████████████ │ +│ BILLINGSHURST │ CHICHESTER │ 144 │ 797829 │ ███████████████▊ │ +│ WOKING │ GUILDFORD │ 176 │ 793494 │ ███████████████▋ │ +│ STOCKBRIDGE │ TEST VALLEY │ 178 │ 793269 │ ███████████████▋ │ +│ EPSOM │ REIGATE AND BANSTEAD │ 172 │ 791862 │ ███████████████▋ │ +│ TONBRIDGE │ TUNBRIDGE WELLS │ 360 │ 787876 │ ███████████████▋ │ +│ TEDDINGTON │ RICHMOND UPON THAMES │ 595 │ 786492 │ ███████████████▋ │ +│ TWICKENHAM │ RICHMOND UPON THAMES │ 1155 │ 786193 │ ███████████████▋ │ +│ LYNDHURST │ NEW FOREST │ 102 │ 785593 │ ███████████████▋ │ +│ LONDON │ LAMBETH │ 5228 │ 774574 │ ███████████████▍ │ +│ LONDON │ BARNET │ 3955 │ 773259 │ ███████████████▍ │ +│ OXFORD │ VALE OF WHITE HORSE │ 353 │ 772088 │ ███████████████▍ │ +│ TONBRIDGE │ MAIDSTONE │ 305 │ 770740 │ ███████████████▍ │ +│ LUTTERWORTH │ HARBOROUGH │ 538 │ 768634 │ ███████████████▎ │ +│ WOODSTOCK │ WEST OXFORDSHIRE │ 140 │ 766037 │ ███████████████▎ │ +│ MIDHURST │ CHICHESTER │ 257 │ 764815 │ ███████████████▎ │ +│ MARLOW │ BUCKINGHAMSHIRE │ 327 │ 761876 │ ███████████████▏ │ +│ LONDON │ NEWHAM │ 3237 │ 761784 │ ███████████████▏ │ +│ ALDERLEY EDGE │ CHESHIRE EAST │ 178 │ 757318 │ ███████████████▏ │ +│ LUTON │ CENTRAL BEDFORDSHIRE │ 212 │ 754283 │ ███████████████ │ +│ PETWORTH │ CHICHESTER │ 154 │ 754220 │ ███████████████ │ +│ ALRESFORD │ WINCHESTER │ 219 │ 752718 │ ███████████████ │ +│ POTTERS BAR │ WELWYN HATFIELD │ 174 │ 748465 │ ██████████████▊ │ +│ HASLEMERE │ CHICHESTER │ 128 │ 746907 │ ██████████████▊ │ +│ TADWORTH │ REIGATE AND BANSTEAD │ 502 │ 743252 │ ██████████████▋ │ +│ THAMES DITTON │ ELMBRIDGE │ 244 │ 741913 │ ██████████████▋ │ +│ REIGATE │ REIGATE AND BANSTEAD │ 581 │ 738198 │ ██████████████▋ │ +│ BOURNE END │ BUCKINGHAMSHIRE │ 138 │ 735190 │ ██████████████▋ │ +│ SEVENOAKS │ SEVENOAKS │ 1156 │ 730018 │ ██████████████▌ │ +│ OXTED │ TANDRIDGE │ 336 │ 729123 │ ██████████████▌ │ +│ INGATESTONE │ BRENTWOOD │ 166 │ 728103 │ ██████████████▌ │ +│ LONDON │ BRENT │ 2079 │ 720605 │ ██████████████▍ │ +│ LONDON │ HARINGEY │ 3216 │ 717780 │ ██████████████▎ │ +│ PURLEY │ CROYDON │ 575 │ 716108 │ ██████████████▎ │ +│ WELWYN │ WELWYN HATFIELD │ 222 │ 710603 │ ██████████████▏ │ +│ RICKMANSWORTH │ THREE RIVERS │ 798 │ 704571 │ ██████████████ │ +│ BANSTEAD │ REIGATE AND BANSTEAD │ 401 │ 701293 │ ██████████████ │ +│ CHIGWELL │ EPPING FOREST │ 261 │ 701203 │ ██████████████ │ +│ PINNER │ HARROW │ 528 │ 698885 │ █████████████▊ │ +│ HASLEMERE │ WAVERLEY │ 280 │ 696659 │ █████████████▊ │ +│ SLOUGH │ BUCKINGHAMSHIRE │ 396 │ 694917 │ █████████████▊ │ +│ WALTON-ON-THAMES │ ELMBRIDGE │ 946 │ 692395 │ █████████████▋ │ +│ READING │ SOUTH OXFORDSHIRE │ 318 │ 691988 │ █████████████▋ │ +│ NORTHWOOD │ HILLINGDON │ 271 │ 690643 │ █████████████▋ │ +│ FELTHAM │ HOUNSLOW │ 763 │ 688595 │ █████████████▋ │ +│ ASHTEAD │ MOLE VALLEY │ 303 │ 687923 │ █████████████▋ │ +│ BARNET │ BARNET │ 975 │ 686980 │ █████████████▋ │ +│ WOKING │ SURREY HEATH │ 283 │ 686669 │ █████████████▋ │ +│ MALMESBURY │ WILTSHIRE │ 323 │ 683324 │ █████████████▋ │ +│ AMERSHAM │ BUCKINGHAMSHIRE │ 496 │ 680962 │ █████████████▌ │ +│ CHISLEHURST │ BROMLEY │ 430 │ 680209 │ █████████████▌ │ +│ HYTHE │ FOLKESTONE AND HYTHE │ 490 │ 676908 │ █████████████▌ │ +│ MAYFIELD │ WEALDEN │ 101 │ 676210 │ █████████████▌ │ +│ ASCOT │ BRACKNELL FOREST │ 168 │ 676004 │ █████████████▌ │ +└──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ ``` -### Test it in Playground +## Ускорьте запросы с помощью проекций -The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). +[Проекции](https://../../sql-reference/statements/alter/projection/) позволяют повысить скорость запросов за счет хранения предварительно агрегированных данных. -## Let's speed up queries using projections +### Создайте проекцию -[Projections](https://../../sql-reference/statements/alter/projection/) allow to improve queries speed by storing pre-aggregated data. - -### Build a projection - -Create an aggregate projection by dimensions (toYear(date), district, town): +Создайте агрегирующую проекцию по параметрам `toYear(date)`, `district`, `town`: ```sql ALTER TABLE uk_price_paid @@ -374,7 +381,7 @@ ALTER TABLE uk_price_paid ); ``` -Populate the projection for existing data (without it projection will be created for only newly inserted data) +Заполните проекцию для текущих данных (иначе проекция будет создана только для добавляемых данных): ```sql ALTER TABLE uk_price_paid @@ -382,18 +389,19 @@ ALTER TABLE uk_price_paid SETTINGS mutations_sync = 1; ``` -## Test performance +## Проверьте производительность -Enable projections for selects and let's run the same 3 queries. +Давайте выполним те же 3 запроса. + +Включите поддержку проекций: ```sql -set allow_experimental_projection_optimization=1; +SET allow_experimental_projection_optimization = 1; ``` +### Запрос 1. Средняя цена за год -### Average price per year - -Query: +Запрос: ```sql SELECT @@ -405,45 +413,43 @@ GROUP BY year ORDER BY year ASC; ``` -Result: +Результат: ```text ┌─year─┬──price─┬─bar(round(avg(price)), 0, 1000000, 80)─┐ │ 1995 │ 67932 │ █████▍ │ │ 1996 │ 71505 │ █████▋ │ │ 1997 │ 78532 │ ██████▎ │ -│ 1998 │ 85435 │ ██████▋ │ -│ 1999 │ 96036 │ ███████▋ │ -│ 2000 │ 107478 │ ████████▌ │ -│ 2001 │ 118886 │ █████████▌ │ -│ 2002 │ 137940 │ ███████████ │ -│ 2003 │ 155888 │ ████████████▍ │ +│ 1998 │ 85436 │ ██████▋ │ +│ 1999 │ 96037 │ ███████▋ │ +│ 2000 │ 107479 │ ████████▌ │ +│ 2001 │ 118885 │ █████████▌ │ +│ 2002 │ 137941 │ ███████████ │ +│ 2003 │ 155889 │ ████████████▍ │ │ 2004 │ 178885 │ ██████████████▎ │ -│ 2005 │ 189350 │ ███████████████▏ │ +│ 2005 │ 189351 │ ███████████████▏ │ │ 2006 │ 203528 │ ████████████████▎ │ -│ 2007 │ 219377 │ █████████████████▌ │ +│ 2007 │ 219378 │ █████████████████▌ │ │ 2008 │ 217056 │ █████████████████▎ │ │ 2009 │ 213419 │ █████████████████ │ -│ 2010 │ 236110 │ ██████████████████▊ │ -│ 2011 │ 232804 │ ██████████████████▌ │ -│ 2012 │ 238366 │ ███████████████████ │ +│ 2010 │ 236109 │ ██████████████████▊ │ +│ 2011 │ 232805 │ ██████████████████▌ │ +│ 2012 │ 238367 │ ███████████████████ │ │ 2013 │ 256931 │ ████████████████████▌ │ -│ 2014 │ 279917 │ ██████████████████████▍ │ -│ 2015 │ 297264 │ ███████████████████████▋ │ -│ 2016 │ 313197 │ █████████████████████████ │ -│ 2017 │ 346070 │ ███████████████████████████▋ │ -│ 2018 │ 350117 │ ████████████████████████████ │ -│ 2019 │ 351010 │ ████████████████████████████ │ -│ 2020 │ 368974 │ █████████████████████████████▌ │ -│ 2021 │ 384351 │ ██████████████████████████████▋ │ +│ 2014 │ 279915 │ ██████████████████████▍ │ +│ 2015 │ 297266 │ ███████████████████████▋ │ +│ 2016 │ 313201 │ █████████████████████████ │ +│ 2017 │ 346097 │ ███████████████████████████▋ │ +│ 2018 │ 350116 │ ████████████████████████████ │ +│ 2019 │ 351013 │ ████████████████████████████ │ +│ 2020 │ 369420 │ █████████████████████████████▌ │ +│ 2021 │ 386903 │ ██████████████████████████████▊ │ └──────┴────────┴────────────────────────────────────────┘ - -27 rows in set. Elapsed: 0.003 sec. Processed 106.87 thousand rows, 3.21 MB (31.92 million rows/s., 959.03 MB/s.) ``` -### Average price per year in London +### Запрос 2. Средняя цена за год в Лондоне -Query: +Запрос: ```sql SELECT @@ -456,45 +462,45 @@ GROUP BY year ORDER BY year ASC; ``` -Result: +Результат: ```text ┌─year─┬───price─┬─bar(round(avg(price)), 0, 2000000, 100)───────────────┐ -│ 1995 │ 109112 │ █████▍ │ +│ 1995 │ 109116 │ █████▍ │ │ 1996 │ 118667 │ █████▊ │ │ 1997 │ 136518 │ ██████▋ │ │ 1998 │ 152983 │ ███████▋ │ -│ 1999 │ 180633 │ █████████ │ -│ 2000 │ 215830 │ ██████████▋ │ -│ 2001 │ 232996 │ ███████████▋ │ -│ 2002 │ 263672 │ █████████████▏ │ +│ 1999 │ 180637 │ █████████ │ +│ 2000 │ 215838 │ ██████████▋ │ +│ 2001 │ 232994 │ ███████████▋ │ +│ 2002 │ 263670 │ █████████████▏ │ │ 2003 │ 278394 │ █████████████▊ │ -│ 2004 │ 304665 │ ███████████████▏ │ +│ 2004 │ 304666 │ ███████████████▏ │ │ 2005 │ 322875 │ ████████████████▏ │ -│ 2006 │ 356192 │ █████████████████▋ │ -│ 2007 │ 404055 │ ████████████████████▏ │ +│ 2006 │ 356191 │ █████████████████▋ │ +│ 2007 │ 404054 │ ████████████████████▏ │ │ 2008 │ 420741 │ █████████████████████ │ -│ 2009 │ 427754 │ █████████████████████▍ │ +│ 2009 │ 427753 │ █████████████████████▍ │ │ 2010 │ 480306 │ ████████████████████████ │ │ 2011 │ 496274 │ ████████████████████████▋ │ -│ 2012 │ 519441 │ █████████████████████████▊ │ -│ 2013 │ 616209 │ ██████████████████████████████▋ │ -│ 2014 │ 724144 │ ████████████████████████████████████▏ │ -│ 2015 │ 792112 │ ███████████████████████████████████████▌ │ -│ 2016 │ 843568 │ ██████████████████████████████████████████▏ │ -│ 2017 │ 982566 │ █████████████████████████████████████████████████▏ │ -│ 2018 │ 1016845 │ ██████████████████████████████████████████████████▋ │ -│ 2019 │ 1043277 │ ████████████████████████████████████████████████████▏ │ -│ 2020 │ 1003963 │ ██████████████████████████████████████████████████▏ │ -│ 2021 │ 940794 │ ███████████████████████████████████████████████ │ +│ 2012 │ 519442 │ █████████████████████████▊ │ +│ 2013 │ 616212 │ ██████████████████████████████▋ │ +│ 2014 │ 724154 │ ████████████████████████████████████▏ │ +│ 2015 │ 792129 │ ███████████████████████████████████████▌ │ +│ 2016 │ 843655 │ ██████████████████████████████████████████▏ │ +│ 2017 │ 982642 │ █████████████████████████████████████████████████▏ │ +│ 2018 │ 1016835 │ ██████████████████████████████████████████████████▋ │ +│ 2019 │ 1042849 │ ████████████████████████████████████████████████████▏ │ +│ 2020 │ 1011889 │ ██████████████████████████████████████████████████▌ │ +│ 2021 │ 960343 │ ████████████████████████████████████████████████ │ └──────┴─────────┴───────────────────────────────────────────────────────┘ - -27 rows in set. Elapsed: 0.005 sec. Processed 106.87 thousand rows, 3.53 MB (23.49 million rows/s., 775.95 MB/s.) ``` -### The most expensive neighborhoods: the condition (date >= '2020-01-01') needs to be modified to match projection dimension (toYear(date) >= 2020) +### Запрос 3. Самые дорогие районы -Query: +Условие (date >= '2020-01-01') необходимо изменить, чтобы оно соответствовало проекции (toYear(date) >= 2020). + +Запрос: ```sql SELECT @@ -513,119 +519,133 @@ ORDER BY price DESC LIMIT 100; ``` -Result: +Результат: ```text ┌─town─────────────────┬─district───────────────┬────c─┬───price─┬─bar(round(avg(price)), 0, 5000000, 100)────────────────────────────┐ -│ LONDON │ CITY OF WESTMINSTER │ 3372 │ 3305225 │ ██████████████████████████████████████████████████████████████████ │ -│ LONDON │ CITY OF LONDON │ 257 │ 3294478 │ █████████████████████████████████████████████████████████████████▊ │ -│ LONDON │ KENSINGTON AND CHELSEA │ 2367 │ 2342422 │ ██████████████████████████████████████████████▋ │ -│ LEATHERHEAD │ ELMBRIDGE │ 108 │ 1927143 │ ██████████████████████████████████████▌ │ -│ VIRGINIA WATER │ RUNNYMEDE │ 142 │ 1868819 │ █████████████████████████████████████▍ │ -│ LONDON │ CAMDEN │ 2815 │ 1736788 │ ██████████████████████████████████▋ │ -│ THORNTON HEATH │ CROYDON │ 521 │ 1733051 │ ██████████████████████████████████▋ │ -│ WINDLESHAM │ SURREY HEATH │ 103 │ 1717255 │ ██████████████████████████████████▎ │ -│ BARNET │ ENFIELD │ 115 │ 1503458 │ ██████████████████████████████ │ -│ OXFORD │ SOUTH OXFORDSHIRE │ 298 │ 1275200 │ █████████████████████████▌ │ -│ LONDON │ ISLINGTON │ 2458 │ 1274308 │ █████████████████████████▍ │ -│ COBHAM │ ELMBRIDGE │ 364 │ 1260005 │ █████████████████████████▏ │ -│ LONDON │ HOUNSLOW │ 618 │ 1215682 │ ████████████████████████▎ │ -│ ASCOT │ WINDSOR AND MAIDENHEAD │ 379 │ 1215146 │ ████████████████████████▎ │ -│ LONDON │ RICHMOND UPON THAMES │ 654 │ 1207551 │ ████████████████████████▏ │ -│ BEACONSFIELD │ BUCKINGHAMSHIRE │ 307 │ 1186220 │ ███████████████████████▋ │ -│ RICHMOND │ RICHMOND UPON THAMES │ 805 │ 1100420 │ ██████████████████████ │ -│ LONDON │ HAMMERSMITH AND FULHAM │ 2888 │ 1062959 │ █████████████████████▎ │ -│ WEYBRIDGE │ ELMBRIDGE │ 607 │ 1027161 │ ████████████████████▌ │ -│ RADLETT │ HERTSMERE │ 265 │ 1015896 │ ████████████████████▎ │ -│ SALCOMBE │ SOUTH HAMS │ 124 │ 1014393 │ ████████████████████▎ │ -│ BURFORD │ WEST OXFORDSHIRE │ 102 │ 993100 │ ███████████████████▋ │ -│ ESHER │ ELMBRIDGE │ 454 │ 969770 │ ███████████████████▍ │ -│ HINDHEAD │ WAVERLEY │ 128 │ 967786 │ ███████████████████▎ │ -│ BROCKENHURST │ NEW FOREST │ 121 │ 967046 │ ███████████████████▎ │ -│ LEATHERHEAD │ GUILDFORD │ 191 │ 964489 │ ███████████████████▎ │ -│ GERRARDS CROSS │ BUCKINGHAMSHIRE │ 376 │ 958555 │ ███████████████████▏ │ -│ EAST MOLESEY │ ELMBRIDGE │ 181 │ 943457 │ ██████████████████▋ │ -│ OLNEY │ MILTON KEYNES │ 220 │ 942892 │ ██████████████████▋ │ -│ CHALFONT ST GILES │ BUCKINGHAMSHIRE │ 135 │ 926950 │ ██████████████████▌ │ -│ HENLEY-ON-THAMES │ SOUTH OXFORDSHIRE │ 509 │ 905732 │ ██████████████████ │ -│ KINGSTON UPON THAMES │ KINGSTON UPON THAMES │ 889 │ 899689 │ █████████████████▊ │ -│ BELVEDERE │ BEXLEY │ 313 │ 895336 │ █████████████████▊ │ -│ CRANBROOK │ TUNBRIDGE WELLS │ 404 │ 888190 │ █████████████████▋ │ -│ LONDON │ EALING │ 2460 │ 865893 │ █████████████████▎ │ -│ MAIDENHEAD │ BUCKINGHAMSHIRE │ 114 │ 863814 │ █████████████████▎ │ -│ LONDON │ MERTON │ 1958 │ 857192 │ █████████████████▏ │ -│ GUILDFORD │ WAVERLEY │ 131 │ 854447 │ █████████████████ │ -│ LONDON │ HACKNEY │ 3088 │ 846571 │ ████████████████▊ │ -│ LYMM │ WARRINGTON │ 285 │ 839920 │ ████████████████▋ │ -│ HARPENDEN │ ST ALBANS │ 606 │ 836994 │ ████████████████▋ │ -│ LONDON │ WANDSWORTH │ 6113 │ 832292 │ ████████████████▋ │ -│ LONDON │ SOUTHWARK │ 3612 │ 831319 │ ████████████████▋ │ -│ BERKHAMSTED │ DACORUM │ 502 │ 830356 │ ████████████████▌ │ -│ KINGS LANGLEY │ DACORUM │ 137 │ 821358 │ ████████████████▍ │ -│ TONBRIDGE │ TUNBRIDGE WELLS │ 339 │ 806736 │ ████████████████▏ │ -│ EPSOM │ REIGATE AND BANSTEAD │ 157 │ 805903 │ ████████████████ │ -│ WOKING │ GUILDFORD │ 161 │ 803283 │ ████████████████ │ -│ STOCKBRIDGE │ TEST VALLEY │ 168 │ 801973 │ ████████████████ │ -│ TEDDINGTON │ RICHMOND UPON THAMES │ 539 │ 798591 │ ███████████████▊ │ -│ OXFORD │ VALE OF WHITE HORSE │ 329 │ 792907 │ ███████████████▋ │ -│ LONDON │ BARNET │ 3624 │ 789583 │ ███████████████▋ │ -│ TWICKENHAM │ RICHMOND UPON THAMES │ 1090 │ 787760 │ ███████████████▋ │ -│ LUTON │ CENTRAL BEDFORDSHIRE │ 196 │ 786051 │ ███████████████▋ │ -│ TONBRIDGE │ MAIDSTONE │ 277 │ 785746 │ ███████████████▋ │ -│ TOWCESTER │ WEST NORTHAMPTONSHIRE │ 186 │ 783532 │ ███████████████▋ │ -│ LONDON │ LAMBETH │ 4832 │ 783422 │ ███████████████▋ │ -│ LUTTERWORTH │ HARBOROUGH │ 515 │ 781775 │ ███████████████▋ │ -│ WOODSTOCK │ WEST OXFORDSHIRE │ 135 │ 777499 │ ███████████████▌ │ -│ ALRESFORD │ WINCHESTER │ 196 │ 775577 │ ███████████████▌ │ -│ LONDON │ NEWHAM │ 2942 │ 768551 │ ███████████████▎ │ -│ ALDERLEY EDGE │ CHESHIRE EAST │ 168 │ 768280 │ ███████████████▎ │ -│ MARLOW │ BUCKINGHAMSHIRE │ 301 │ 762784 │ ███████████████▎ │ -│ BILLINGSHURST │ CHICHESTER │ 134 │ 760920 │ ███████████████▏ │ -│ LONDON │ TOWER HAMLETS │ 4183 │ 759635 │ ███████████████▏ │ -│ MIDHURST │ CHICHESTER │ 245 │ 759101 │ ███████████████▏ │ -│ THAMES DITTON │ ELMBRIDGE │ 227 │ 753347 │ ███████████████ │ -│ POTTERS BAR │ WELWYN HATFIELD │ 163 │ 752926 │ ███████████████ │ -│ REIGATE │ REIGATE AND BANSTEAD │ 555 │ 740961 │ ██████████████▋ │ -│ TADWORTH │ REIGATE AND BANSTEAD │ 477 │ 738997 │ ██████████████▋ │ -│ SEVENOAKS │ SEVENOAKS │ 1074 │ 734658 │ ██████████████▋ │ -│ PETWORTH │ CHICHESTER │ 138 │ 732432 │ ██████████████▋ │ -│ BOURNE END │ BUCKINGHAMSHIRE │ 127 │ 730742 │ ██████████████▌ │ -│ PURLEY │ CROYDON │ 540 │ 727721 │ ██████████████▌ │ -│ OXTED │ TANDRIDGE │ 320 │ 726078 │ ██████████████▌ │ -│ LONDON │ HARINGEY │ 2988 │ 724573 │ ██████████████▍ │ -│ BANSTEAD │ REIGATE AND BANSTEAD │ 373 │ 713834 │ ██████████████▎ │ -│ PINNER │ HARROW │ 480 │ 712166 │ ██████████████▏ │ -│ MALMESBURY │ WILTSHIRE │ 293 │ 707747 │ ██████████████▏ │ -│ RICKMANSWORTH │ THREE RIVERS │ 732 │ 705400 │ ██████████████ │ -│ SLOUGH │ BUCKINGHAMSHIRE │ 359 │ 705002 │ ██████████████ │ -│ GREAT MISSENDEN │ BUCKINGHAMSHIRE │ 214 │ 704904 │ ██████████████ │ -│ READING │ SOUTH OXFORDSHIRE │ 295 │ 701697 │ ██████████████ │ -│ HYTHE │ FOLKESTONE AND HYTHE │ 457 │ 700334 │ ██████████████ │ -│ WELWYN │ WELWYN HATFIELD │ 217 │ 699649 │ █████████████▊ │ -│ CHIGWELL │ EPPING FOREST │ 242 │ 697869 │ █████████████▊ │ -│ BARNET │ BARNET │ 906 │ 695680 │ █████████████▊ │ -│ HASLEMERE │ CHICHESTER │ 120 │ 694028 │ █████████████▊ │ -│ LEATHERHEAD │ MOLE VALLEY │ 748 │ 692026 │ █████████████▋ │ -│ LONDON │ BRENT │ 1945 │ 690799 │ █████████████▋ │ -│ HASLEMERE │ WAVERLEY │ 258 │ 690765 │ █████████████▋ │ -│ NORTHWOOD │ HILLINGDON │ 252 │ 690753 │ █████████████▋ │ -│ WALTON-ON-THAMES │ ELMBRIDGE │ 871 │ 689431 │ █████████████▋ │ -│ INGATESTONE │ BRENTWOOD │ 150 │ 688345 │ █████████████▋ │ -│ OXFORD │ OXFORD │ 1761 │ 686114 │ █████████████▋ │ -│ CHISLEHURST │ BROMLEY │ 410 │ 682892 │ █████████████▋ │ -│ KINGS LANGLEY │ THREE RIVERS │ 109 │ 682320 │ █████████████▋ │ -│ ASHTEAD │ MOLE VALLEY │ 280 │ 680483 │ █████████████▌ │ -│ WOKING │ SURREY HEATH │ 269 │ 679035 │ █████████████▌ │ -│ ASCOT │ BRACKNELL FOREST │ 160 │ 678632 │ █████████████▌ │ +│ LONDON │ CITY OF WESTMINSTER │ 3606 │ 3280239 │ █████████████████████████████████████████████████████████████████▌ │ +│ LONDON │ CITY OF LONDON │ 274 │ 3160502 │ ███████████████████████████████████████████████████████████████▏ │ +│ LONDON │ KENSINGTON AND CHELSEA │ 2550 │ 2308478 │ ██████████████████████████████████████████████▏ │ +│ LEATHERHEAD │ ELMBRIDGE │ 114 │ 1897407 │ █████████████████████████████████████▊ │ +│ LONDON │ CAMDEN │ 3033 │ 1805404 │ ████████████████████████████████████ │ +│ VIRGINIA WATER │ RUNNYMEDE │ 156 │ 1753247 │ ███████████████████████████████████ │ +│ WINDLESHAM │ SURREY HEATH │ 108 │ 1677613 │ █████████████████████████████████▌ │ +│ THORNTON HEATH │ CROYDON │ 546 │ 1671721 │ █████████████████████████████████▍ │ +│ BARNET │ ENFIELD │ 124 │ 1505840 │ ██████████████████████████████ │ +│ COBHAM │ ELMBRIDGE │ 387 │ 1237250 │ ████████████████████████▋ │ +│ LONDON │ ISLINGTON │ 2668 │ 1236980 │ ████████████████████████▋ │ +│ OXFORD │ SOUTH OXFORDSHIRE │ 321 │ 1220907 │ ████████████████████████▍ │ +│ LONDON │ RICHMOND UPON THAMES │ 704 │ 1215551 │ ████████████████████████▎ │ +│ LONDON │ HOUNSLOW │ 671 │ 1207493 │ ████████████████████████▏ │ +│ ASCOT │ WINDSOR AND MAIDENHEAD │ 407 │ 1183299 │ ███████████████████████▋ │ +│ BEACONSFIELD │ BUCKINGHAMSHIRE │ 330 │ 1175615 │ ███████████████████████▌ │ +│ RICHMOND │ RICHMOND UPON THAMES │ 874 │ 1110444 │ ██████████████████████▏ │ +│ LONDON │ HAMMERSMITH AND FULHAM │ 3086 │ 1053983 │ █████████████████████ │ +│ SURBITON │ ELMBRIDGE │ 100 │ 1011800 │ ████████████████████▏ │ +│ RADLETT │ HERTSMERE │ 283 │ 1011712 │ ████████████████████▏ │ +│ SALCOMBE │ SOUTH HAMS │ 127 │ 1011624 │ ████████████████████▏ │ +│ WEYBRIDGE │ ELMBRIDGE │ 655 │ 1007265 │ ████████████████████▏ │ +│ ESHER │ ELMBRIDGE │ 485 │ 986581 │ ███████████████████▋ │ +│ LEATHERHEAD │ GUILDFORD │ 202 │ 977320 │ ███████████████████▌ │ +│ BURFORD │ WEST OXFORDSHIRE │ 111 │ 966893 │ ███████████████████▎ │ +│ BROCKENHURST │ NEW FOREST │ 129 │ 956675 │ ███████████████████▏ │ +│ HINDHEAD │ WAVERLEY │ 137 │ 953753 │ ███████████████████ │ +│ GERRARDS CROSS │ BUCKINGHAMSHIRE │ 419 │ 951121 │ ███████████████████ │ +│ EAST MOLESEY │ ELMBRIDGE │ 192 │ 936769 │ ██████████████████▋ │ +│ CHALFONT ST GILES │ BUCKINGHAMSHIRE │ 146 │ 925515 │ ██████████████████▌ │ +│ LONDON │ TOWER HAMLETS │ 4388 │ 918304 │ ██████████████████▎ │ +│ OLNEY │ MILTON KEYNES │ 235 │ 910646 │ ██████████████████▏ │ +│ HENLEY-ON-THAMES │ SOUTH OXFORDSHIRE │ 540 │ 902418 │ ██████████████████ │ +│ LONDON │ SOUTHWARK │ 3885 │ 892997 │ █████████████████▋ │ +│ KINGSTON UPON THAMES │ KINGSTON UPON THAMES │ 960 │ 885969 │ █████████████████▋ │ +│ LONDON │ EALING │ 2658 │ 871755 │ █████████████████▍ │ +│ CRANBROOK │ TUNBRIDGE WELLS │ 431 │ 862348 │ █████████████████▏ │ +│ LONDON │ MERTON │ 2099 │ 859118 │ █████████████████▏ │ +│ BELVEDERE │ BEXLEY │ 346 │ 842423 │ ████████████████▋ │ +│ GUILDFORD │ WAVERLEY │ 143 │ 841277 │ ████████████████▋ │ +│ HARPENDEN │ ST ALBANS │ 657 │ 841216 │ ████████████████▋ │ +│ LONDON │ HACKNEY │ 3307 │ 837090 │ ████████████████▋ │ +│ LONDON │ WANDSWORTH │ 6566 │ 832663 │ ████████████████▋ │ +│ MAIDENHEAD │ BUCKINGHAMSHIRE │ 123 │ 824299 │ ████████████████▍ │ +│ KINGS LANGLEY │ DACORUM │ 145 │ 821331 │ ████████████████▍ │ +│ BERKHAMSTED │ DACORUM │ 543 │ 818415 │ ████████████████▎ │ +│ GREAT MISSENDEN │ BUCKINGHAMSHIRE │ 226 │ 802807 │ ████████████████ │ +│ BILLINGSHURST │ CHICHESTER │ 144 │ 797829 │ ███████████████▊ │ +│ WOKING │ GUILDFORD │ 176 │ 793494 │ ███████████████▋ │ +│ STOCKBRIDGE │ TEST VALLEY │ 178 │ 793269 │ ███████████████▋ │ +│ EPSOM │ REIGATE AND BANSTEAD │ 172 │ 791862 │ ███████████████▋ │ +│ TONBRIDGE │ TUNBRIDGE WELLS │ 360 │ 787876 │ ███████████████▋ │ +│ TEDDINGTON │ RICHMOND UPON THAMES │ 595 │ 786492 │ ███████████████▋ │ +│ TWICKENHAM │ RICHMOND UPON THAMES │ 1155 │ 786193 │ ███████████████▋ │ +│ LYNDHURST │ NEW FOREST │ 102 │ 785593 │ ███████████████▋ │ +│ LONDON │ LAMBETH │ 5228 │ 774574 │ ███████████████▍ │ +│ LONDON │ BARNET │ 3955 │ 773259 │ ███████████████▍ │ +│ OXFORD │ VALE OF WHITE HORSE │ 353 │ 772088 │ ███████████████▍ │ +│ TONBRIDGE │ MAIDSTONE │ 305 │ 770740 │ ███████████████▍ │ +│ LUTTERWORTH │ HARBOROUGH │ 538 │ 768634 │ ███████████████▎ │ +│ WOODSTOCK │ WEST OXFORDSHIRE │ 140 │ 766037 │ ███████████████▎ │ +│ MIDHURST │ CHICHESTER │ 257 │ 764815 │ ███████████████▎ │ +│ MARLOW │ BUCKINGHAMSHIRE │ 327 │ 761876 │ ███████████████▏ │ +│ LONDON │ NEWHAM │ 3237 │ 761784 │ ███████████████▏ │ +│ ALDERLEY EDGE │ CHESHIRE EAST │ 178 │ 757318 │ ███████████████▏ │ +│ LUTON │ CENTRAL BEDFORDSHIRE │ 212 │ 754283 │ ███████████████ │ +│ PETWORTH │ CHICHESTER │ 154 │ 754220 │ ███████████████ │ +│ ALRESFORD │ WINCHESTER │ 219 │ 752718 │ ███████████████ │ +│ POTTERS BAR │ WELWYN HATFIELD │ 174 │ 748465 │ ██████████████▊ │ +│ HASLEMERE │ CHICHESTER │ 128 │ 746907 │ ██████████████▊ │ +│ TADWORTH │ REIGATE AND BANSTEAD │ 502 │ 743252 │ ██████████████▋ │ +│ THAMES DITTON │ ELMBRIDGE │ 244 │ 741913 │ ██████████████▋ │ +│ REIGATE │ REIGATE AND BANSTEAD │ 581 │ 738198 │ ██████████████▋ │ +│ BOURNE END │ BUCKINGHAMSHIRE │ 138 │ 735190 │ ██████████████▋ │ +│ SEVENOAKS │ SEVENOAKS │ 1156 │ 730018 │ ██████████████▌ │ +│ OXTED │ TANDRIDGE │ 336 │ 729123 │ ██████████████▌ │ +│ INGATESTONE │ BRENTWOOD │ 166 │ 728103 │ ██████████████▌ │ +│ LONDON │ BRENT │ 2079 │ 720605 │ ██████████████▍ │ +│ LONDON │ HARINGEY │ 3216 │ 717780 │ ██████████████▎ │ +│ PURLEY │ CROYDON │ 575 │ 716108 │ ██████████████▎ │ +│ WELWYN │ WELWYN HATFIELD │ 222 │ 710603 │ ██████████████▏ │ +│ RICKMANSWORTH │ THREE RIVERS │ 798 │ 704571 │ ██████████████ │ +│ BANSTEAD │ REIGATE AND BANSTEAD │ 401 │ 701293 │ ██████████████ │ +│ CHIGWELL │ EPPING FOREST │ 261 │ 701203 │ ██████████████ │ +│ PINNER │ HARROW │ 528 │ 698885 │ █████████████▊ │ +│ HASLEMERE │ WAVERLEY │ 280 │ 696659 │ █████████████▊ │ +│ SLOUGH │ BUCKINGHAMSHIRE │ 396 │ 694917 │ █████████████▊ │ +│ WALTON-ON-THAMES │ ELMBRIDGE │ 946 │ 692395 │ █████████████▋ │ +│ READING │ SOUTH OXFORDSHIRE │ 318 │ 691988 │ █████████████▋ │ +│ NORTHWOOD │ HILLINGDON │ 271 │ 690643 │ █████████████▋ │ +│ FELTHAM │ HOUNSLOW │ 763 │ 688595 │ █████████████▋ │ +│ ASHTEAD │ MOLE VALLEY │ 303 │ 687923 │ █████████████▋ │ +│ BARNET │ BARNET │ 975 │ 686980 │ █████████████▋ │ +│ WOKING │ SURREY HEATH │ 283 │ 686669 │ █████████████▋ │ +│ MALMESBURY │ WILTSHIRE │ 323 │ 683324 │ █████████████▋ │ +│ AMERSHAM │ BUCKINGHAMSHIRE │ 496 │ 680962 │ █████████████▌ │ +│ CHISLEHURST │ BROMLEY │ 430 │ 680209 │ █████████████▌ │ +│ HYTHE │ FOLKESTONE AND HYTHE │ 490 │ 676908 │ █████████████▌ │ +│ MAYFIELD │ WEALDEN │ 101 │ 676210 │ █████████████▌ │ +│ ASCOT │ BRACKNELL FOREST │ 168 │ 676004 │ █████████████▌ │ └──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ - -100 rows in set. Elapsed: 0.005 sec. Processed 12.85 thousand rows, 813.40 KB (2.73 million rows/s., 172.95 MB/s.) ``` -All 3 queries work much faster and read fewer rows. +Все три запроса работают намного быстрее и читают меньшее количество строк. +```text +Query 1 + +no projection: 27 rows in set. Elapsed: 0.158 sec. Processed 26.32 million rows, 157.93 MB (166.57 million rows/s., 999.39 MB/s.) + projection: 27 rows in set. Elapsed: 0.007 sec. Processed 105.96 thousand rows, 3.33 MB (14.58 million rows/s., 458.13 MB/s.) + + +Query 2 + +no projection: 27 rows in set. Elapsed: 0.163 sec. Processed 26.32 million rows, 80.01 MB (161.75 million rows/s., 491.64 MB/s.) + projection: 27 rows in set. Elapsed: 0.008 sec. Processed 105.96 thousand rows, 3.67 MB (13.29 million rows/s., 459.89 MB/s.) + +Query 3 + +no projection: 100 rows in set. Elapsed: 0.069 sec. Processed 26.32 million rows, 62.47 MB (382.13 million rows/s., 906.93 MB/s.) + projection: 100 rows in set. Elapsed: 0.029 sec. Processed 8.08 thousand rows, 511.08 KB (276.06 thousand rows/s., 17.47 MB/s.) ``` -Q1) -no projection: 27 rows in set. Elapsed: 0.027 sec. Processed 26.25 million rows, 157.49 MB (955.96 million rows/s., 5.74 GB/s.) - projection: 27 rows in set. Elapsed: 0.003 sec. Processed 106.87 thousand rows, 3.21 MB (31.92 million rows/s., 959.03 MB/s.) -``` + +### Online Playground + +Вы можете протестировать другие запросы к этому набору данным с помощью интерактивного ресурса [Online Playground](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). From 95b00f8fd4464bf2b394e252e25dbea766ad6442 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 13 Sep 2021 19:23:56 +0300 Subject: [PATCH 030/142] Fix some live view tests. --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 9 +++++---- src/DataStreams/PushingToViewsBlockOutputStream.h | 3 ++- src/Processors/Executors/PullingPipelineExecutor.cpp | 5 ----- src/Storages/LiveView/StorageLiveView.cpp | 7 ++++--- 4 files changed, 11 insertions(+), 13 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 6201a06c74d..5f82104d914 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -286,7 +286,8 @@ Chain buildPushingToViewsDrain( ContextPtr context, const ASTPtr & query_ptr, bool no_destination, - ExceptionKeepingTransformRuntimeDataPtr runtime_data) + ExceptionKeepingTransformRuntimeDataPtr runtime_data, + const Block & lv_storage) { checkStackSize(); Chain result_chain; @@ -414,7 +415,7 @@ Chain buildPushingToViewsDrain( type = QueryViewsLogElement::ViewType::LIVE; query = live_view->getInnerQuery(); // Used only to log in system.query_views_log out = buildPushingToViewsDrain( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true, view_runtime_data); + dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true, view_runtime_data, storage_header); } else out = buildPushingToViewsDrain( @@ -438,7 +439,7 @@ Chain buildPushingToViewsDrain( nullptr, std::move(runtime_stats)}); - //if (type == QueryViewsLogElement::ViewType::MATERIALIZED) + if (type == QueryViewsLogElement::ViewType::MATERIALIZED) { auto executing_inner_query = std::make_shared( storage_header, views_data->views.back(), views_data->source_storage_id, views_data->source_metadata_snapshot, views_data->source_storage); @@ -497,7 +498,7 @@ Chain buildPushingToViewsDrain( if (auto * live_view = dynamic_cast(storage.get())) { - auto sink = std::make_shared(storage_header, *live_view, storage, context); + auto sink = std::make_shared(lv_storage, *live_view, storage, context); sink->setRuntimeData(runtime_data); result_chain.addSource(std::move(sink)); } diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index 2bf1f79b556..96ccc0ac5b6 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -51,7 +51,8 @@ Chain buildPushingToViewsDrain( ContextPtr context, const ASTPtr & query_ptr, bool no_destination, - ExceptionKeepingTransformRuntimeDataPtr runtime_data); + ExceptionKeepingTransformRuntimeDataPtr runtime_data, + const Block & lv_storage = {}); class ExecutingInnerQueryFromViewTransform final : public ExceptionKeepingTransform diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index 870787a3806..dc59e0a2f5a 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -6,11 +6,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - PullingPipelineExecutor::PullingPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) { pulling_format = std::make_shared(pipeline.getHeader(), has_data_flag); diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 69390850ccc..d6da59b596f 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -16,6 +16,7 @@ limitations under the License. */ #include #include #include +#include #include #include #include @@ -261,10 +262,10 @@ void StorageLiveView::writeIntoLiveView( } auto pipeline = live_view.completeQuery(std::move(from)); - pipeline.resize(1); - pipeline.setSinks([&](const Block &, Pipe::StreamType) + pipeline.addChain(Chain(std::move(output))); + pipeline.setSinks([&](const Block & cur_header, Pipe::StreamType) { - return std::move(output); + return std::make_shared(cur_header); }); auto executor = pipeline.execute(); From 3b5e6ebcd43c45ca041aa38872b4c2c8513ef616 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 13 Sep 2021 20:31:04 +0300 Subject: [PATCH 031/142] Fix link to prj --- docs/en/getting-started/example-datasets/uk-price-paid.md | 2 +- docs/ru/getting-started/example-datasets/uk-price-paid.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index d1053f008ca..68dad008812 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -351,7 +351,7 @@ Result: ## Let's Speed Up Queries Using Projections -[Projections](https://../../sql-reference/statements/alter/projection/) allow to improve queries speed by storing pre-aggregated data. +[Projections](../../sql-reference/statements/alter/projection.md) allow to improve queries speed by storing pre-aggregated data. ### Build a Projection diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index 7f5f383a844..4d89cf7e24d 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -357,7 +357,7 @@ LIMIT 100; ## Ускорьте запросы с помощью проекций -[Проекции](https://../../sql-reference/statements/alter/projection/) позволяют повысить скорость запросов за счет хранения предварительно агрегированных данных. +[Проекции](../../sql-reference/statements/alter/projection.md) позволяют повысить скорость запросов за счет хранения предварительно агрегированных данных. ### Создайте проекцию From 9700e2495a86aae786fae065e3e4b90c97eebd7a Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 13 Sep 2021 20:52:16 +0300 Subject: [PATCH 032/142] Fix errors --- docs/en/getting-started/example-datasets/opensky.md | 10 +++++++--- .../example-datasets/uk-price-paid.md | 4 +++- docs/ru/getting-started/example-datasets/opensky.md | 12 ++++++++---- .../example-datasets/uk-price-paid.md | 4 +++- 4 files changed, 21 insertions(+), 9 deletions(-) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md index 8d9cfd99ec9..51b5073f10e 100644 --- a/docs/en/getting-started/example-datasets/opensky.md +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -5,7 +5,7 @@ toc_title: OpenSky # Crowdsourced air traffic data from The OpenSky Network 2020 -> The data in this dataset is derived and cleaned from the full OpenSky dataset to illustrate the development of air traffic during the COVID-19 pandemic. It spans all flights seen by the network's more than 2500 members since 1 January 2019. More data will be periodically included in the dataset until the end of the COVID-19 pandemic. +"The data in this dataset is derived and cleaned from the full OpenSky dataset to illustrate the development of air traffic during the COVID-19 pandemic. It spans all flights seen by the network's more than 2500 members since 1 January 2019. More data will be periodically included in the dataset until the end of the COVID-19 pandemic". Source: https://zenodo.org/record/5092942#.YRBCyTpRXYd @@ -84,7 +84,9 @@ SELECT count() FROM opensky; Result: ```text -66010819 +┌──count()─┐ +│ 66010819 │ +└──────────┘ ``` The size of dataset in ClickHouse is just 2.66 GiB, check it. @@ -98,7 +100,9 @@ SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' Result: ```text -2.66 GiB +┌─formatReadableSize(total_bytes)─┐ +│ 2.66 GiB │ +└─────────────────────────────────┘ ``` ## Run Some Queries diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 68dad008812..66e4a9b5826 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -112,9 +112,11 @@ SELECT count() FROM uk_price_paid; Result: +```text ┌──count()─┐ │ 26321785 │ └──────────┘ +``` The size of dataset in ClickHouse is just 278 MiB, check it. @@ -642,4 +644,4 @@ no projection: 100 rows in set. Elapsed: 0.069 sec. Processed 26.32 million rows ### Test It in Playground -The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). +The dataset is also available in the [Online Playground](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index 443337a6640..0eedf104d24 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -5,7 +5,7 @@ toc_title: OpenSky # Набор данных о воздушном движении из сети OpenSky Network 2020 -> Данные в этом наборе получены и отфильтрованы из полного набора данных OpenSky, чтобы проиллюстрировать развитие воздушного движения во время пандемии COVID-19. Набор включает в себя все рейсы, которые видели более 2500 участников сети с 1 января 2019 года. Дополнительные данные будут периодически включаться в набор данных до окончания пандемии COVID-19. +"Данные в этом наборе получены и отфильтрованы из полного набора данных OpenSky, чтобы проиллюстрировать развитие воздушного движения во время пандемии COVID-19. Набор включает в себя все рейсы, которые видели более 2500 участников сети с 1 января 2019 года. Дополнительные данные будут периодически включаться в набор данных до окончания пандемии COVID-19". Источник: https://zenodo.org/record/5092942#.YRBCyTpRXYd @@ -70,7 +70,7 @@ ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c 'gzip -c -d "{}" | clickhou Загрузка параллельными потоками займёт около 24 секунд. -Также вы можете последовательный вариант загрузки: +Также вы можете использовать вариант последовательной загрузки: ```bash for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"; done ``` @@ -86,7 +86,9 @@ SELECT count() FROM opensky; Результат: ```text -66010819 +┌──count()─┐ +│ 66010819 │ +└──────────┘ ``` Размер набора данных в ClickHouse составляет всего 2,66 гигабайта, проверьте это. @@ -100,7 +102,9 @@ SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' Результат: ```text -2.66 GiB +┌─formatReadableSize(total_bytes)─┐ +│ 2.66 GiB │ +└─────────────────────────────────┘ ``` ## Примеры diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index 4d89cf7e24d..a6aa5863a72 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -118,9 +118,11 @@ SELECT count() FROM uk_price_paid; Результат: +```text ┌──count()─┐ │ 26321785 │ └──────────┘ +``` Размер набора данных в ClickHouse составляет всего 278 МБ, проверьте это. @@ -648,4 +650,4 @@ no projection: 100 rows in set. Elapsed: 0.069 sec. Processed 26.32 million rows ### Online Playground -Вы можете протестировать другие запросы к этому набору данным с помощью интерактивного ресурса [Online Playground](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). +Этот набор данных доступен в [Online Playground](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). From 638c172564008edce8d6105aea00f3048c7dbdb9 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 13 Sep 2021 21:03:43 +0300 Subject: [PATCH 033/142] Update toc --- docs/ru/getting-started/example-datasets/opensky.md | 4 ++-- docs/ru/getting-started/example-datasets/uk-price-paid.md | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index 0eedf104d24..3eb5943963b 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -1,9 +1,9 @@ --- toc_priority: 20 -toc_title: OpenSky +toc_title: Набор данных о воздушном движении OpenSky Network 2020 --- -# Набор данных о воздушном движении из сети OpenSky Network 2020 +# Набор данных о воздушном движении OpenSky Network 2020 "Данные в этом наборе получены и отфильтрованы из полного набора данных OpenSky, чтобы проиллюстрировать развитие воздушного движения во время пандемии COVID-19. Набор включает в себя все рейсы, которые видели более 2500 участников сети с 1 января 2019 года. Дополнительные данные будут периодически включаться в набор данных до окончания пандемии COVID-19". diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index a6aa5863a72..e05a10ae613 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -1,11 +1,11 @@ --- toc_priority: 20 -toc_title: Данные о недвижимости в Великобритании +toc_title: Набор данных о стоимости недвижимости в Великобритании --- -# Данные о недвижимости в Великобритании +# Набор данных о стоимости недвижимости в Великобритании -Набор данных содержит данные о ценах, уплаченных за недвижимость в Англии и Уэльсе. +Набор данных содержит данные о стоимости недвижимости в Англии и Уэльсе. Данные доступны с 1995 года. Размер набора данных в несжатом виде составляет около 4 гигабайт, а в ClickHouse он займет около 278 Мбайт. From c4cf7808508e0e1a7c5b91cf1d095037510123a7 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 13 Sep 2021 21:12:55 +0300 Subject: [PATCH 034/142] Add summary section --- docs/en/getting-started/example-datasets/uk-price-paid.md | 2 ++ docs/ru/getting-started/example-datasets/uk-price-paid.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 66e4a9b5826..3868c324b32 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -622,6 +622,8 @@ Result: └──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ ``` +### Summary + All 3 queries work much faster and read fewer rows. ```text diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index e05a10ae613..56891ac27f0 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -628,6 +628,8 @@ LIMIT 100; └──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ ``` +### Резюме + Все три запроса работают намного быстрее и читают меньшее количество строк. ```text From adfc48c0f1f76c86eac4a4deeba96a22f9f311a7 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 13 Sep 2021 21:21:01 +0300 Subject: [PATCH 035/142] Update index.md --- docs/ru/getting-started/example-datasets/index.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/ru/getting-started/example-datasets/index.md b/docs/ru/getting-started/example-datasets/index.md index e046fae5fa5..00d14a6cb5a 100644 --- a/docs/ru/getting-started/example-datasets/index.md +++ b/docs/ru/getting-started/example-datasets/index.md @@ -11,13 +11,14 @@ toc_title: "Введение" - [Анонимизированные данные Яндекс.Метрики](metrica.md) - [Star Schema Benchmark](star-schema.md) +- [Набор данных кулинарных рецептов](recipes.md) - [WikiStat](wikistat.md) - [Терабайт логов кликов от Criteo](criteo.md) - [AMPLab Big Data Benchmark](amplab-benchmark.md) - [Данные о такси в Нью-Йорке](nyc-taxi.md) +- [Набор данных о воздушном движении OpenSky Network 2020](../../getting-started/example-datasets/opensky.md) +- [Данные о стоимости недвижимости в Великобритании](../../getting-started/example-datasets/uk-price-paid.md) - [OnTime](ontime.md) -- [OpenSky](../../getting-started/example-datasets/opensky.md) -- [Данные о недвижимости в Великобритании](../../getting-started/example-datasets/uk-price-paid.md) - [Вышки сотовой связи](../../getting-started/example-datasets/cell-towers.md) [Оригинальная статья](https://clickhouse.tech/docs/ru/getting_started/example_datasets) From 2b6a471a8df933207f6e36aaf8487dd5489fcae2 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 14 Sep 2021 11:20:58 +0300 Subject: [PATCH 036/142] Updating ru --- .../example-datasets/opensky.md | 11 +++++----- .../example-datasets/opensky.md | 20 +++++++++---------- .../example-datasets/uk-price-paid.md | 18 ++++++----------- 3 files changed, 20 insertions(+), 29 deletions(-) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md index 51b5073f10e..ede9a4093d7 100644 --- a/docs/en/getting-started/example-datasets/opensky.md +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -56,14 +56,13 @@ Upload data into ClickHouse in parallel: ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c 'gzip -c -d "{}" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"' ``` -Here we pass the list of files (`ls -1 flightlist_*.csv.gz`) to `xargs` for parallel processing. +- Here we pass the list of files (`ls -1 flightlist_*.csv.gz`) to `xargs` for parallel processing. `xargs -P100` specifies to use up to 100 parallel workers but as we only have 30 files, the number of workers will be only 30. +- For every file, `xargs` will run a script with `bash -c`. The script has substitution in form of `{}` and the `xargs` command will substitute the filename to it (we have asked it for xargs with `-I{}`). +- The script will decompress the file (`gzip -c -d "{}"`) to standard output (`-c` parameter) and the output is redirected to `clickhouse-client`. +- We also asked to parse DateTime fields with extended parser (`--date_time_input_format best_effort`) to recognize ISO-8601 format with timezone offsets. -For every file, `xargs` will run a script with `bash -c`. The script has substitution in form of `{}` and the `xargs` command will substitute the filename to it (we have asked it for xargs with `-I{}`). - -The script will decompress the file (`gzip -c -d "{}"`) to standard output (`-c` parameter) and the output is redirected to `clickhouse-client`. - -Finally, `clickhouse-client` will do insertion. It will read input data in [CSVWithNames](../../interfaces/formats.md#csvwithnames) format. We also asked to parse DateTime fields with extended parser (`--date_time_input_format best_effort`) to recognize ISO-8601 format with timezone offsets. +Finally, `clickhouse-client` will do insertion. It will read input data in [CSVWithNames](../../interfaces/formats.md#csvwithnames) format. Parallel upload takes 24 seconds. diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index 3eb5943963b..c8f1a19645a 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -22,7 +22,7 @@ https://doi.org/10.5194/essd-13-357-2021 wget -O- https://zenodo.org/record/5092942 | grep -oP 'https://zenodo.org/record/5092942/files/flightlist_\d+_\d+\.csv\.gz' | xargs wget ``` -Загрузка займет около 2 минут при хорошем подключении к Интернету. Будут загружено 30 файлов, общим размером 4,3 ГБ. +Загрузка займет около 2 минут при хорошем подключении к Интернету. Будет загружено 30 файлов, общим размером 4,3 ГБ. ## Создайте таблицу @@ -56,26 +56,24 @@ CREATE TABLE opensky ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c 'gzip -c -d "{}" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"' ``` -Здесь мы передаем список файлов (`ls -1 flightlist_*.csv.gz`) в `xargs` для параллельной обработки. - -`xargs -P100` указывает на использование до 100 параллельных обработчиков, но поскольку у нас всего 30 файлов, то количество обработчиков будет всего 30. - -Для каждого файла `xargs` будет запускать скрипт с `bash -c`. Сценарий имеет подстановку в виде ` {}`, а команда `xargs` заменяет имя файла (мы указали это для xargs с помощью `-I{}`). - -Скрипт распакует файл (`gzip -c -d "{}"`) в стандартный вывод (параметр`-c`) и перенаправит его в `clickhouse-client`. +- Список файлов передаётся (`ls -1 flightlist_*.csv.gz`) в `xargs` для параллельной обработки. +- `xargs -P100` указывает на использование до 100 параллельных обработчиков, но поскольку у нас всего 30 файлов, то количество обработчиков будет всего 30. +- Для каждого файла `xargs` будет запускать скрипт с `bash -c`. Сценарий имеет подстановку в виде ` {}`, а команда `xargs` заменяет имя файла (мы указали это для xargs с помощью `-I{}`). +- Скрипт распакует файл (`gzip -c -d "{}"`) в стандартный вывод (параметр`-c`) и перенаправит его в `clickhouse-client`. +- Чтобы распознать формат ISO-8601 со смещениями часовых поясов в полях типа `DateTime`, указывается параметр парсера `--date_time_input_format best_effort`. В итоге: клиент clickhouse добавит данные в таблицу `opensky`. Входные данные импортируются в формате [CSVWithNames](../../interfaces/formats.md#csvwithnames). -Чтобы распознать формат ISO-8601 со смещениями часовых поясов в полях типа `DateTime`, указывается параметр парсера `--date_time_input_format best_effort`. Загрузка параллельными потоками займёт около 24 секунд. Также вы можете использовать вариант последовательной загрузки: + ```bash for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"; done ``` -## Проверьте загруженную информацию +## Проверьте импортированные данные Запрос: @@ -91,7 +89,7 @@ SELECT count() FROM opensky; └──────────┘ ``` -Размер набора данных в ClickHouse составляет всего 2,66 гигабайта, проверьте это. +Проверьте, размер набора данных в ClickHouse составляет всего 2,66 ГБ. Запрос: diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index 56891ac27f0..3d3f99445ae 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -5,19 +5,13 @@ toc_title: Набор данных о стоимости недвижимост # Набор данных о стоимости недвижимости в Великобритании -Набор данных содержит данные о стоимости недвижимости в Англии и Уэльсе. -Данные доступны с 1995 года. +Набор данных содержит данные о стоимости недвижимости в Англии и Уэльсе. Данные доступны с 1995 года. Размер набора данных в несжатом виде составляет около 4 гигабайт, а в ClickHouse он займет около 278 Мбайт. - Источник: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads Описание полей таблицы: https://www.gov.uk/guidance/about-the-price-paid-data -Содержит данные HM Land Registry data © Авторское право короны и право на базу данных 2021. - -Crown Авторское право Короны и право на базу данных 2021. Эти данные лицензированы в соответствии с лицензией Open Government Licence v3.0. - - +Набор содержит данные HM Land Registry data © Crown copyright and database right 2021. Эти данные лицензированы в соответствии с Open Government Licence v3.0. ## Загрузите набор данных @@ -54,18 +48,18 @@ CREATE TABLE uk_price_paid ## Обработайте и импортируйте данные -Мы будем использовать инструмент `clickhouse-local` для предварительной обработки данных и `clickhouse-client` для загрузки данных. +В этом примере используется `clickhouse-local` для предварительной обработки данных и `clickhouse-client` для импорта данных. -В этом примере мы определяем структуру исходных данных из CSV-файла и указываем запрос для предварительной обработки данных с помощью `clickhouse-local`. +Указывается структура исходных данных CSV-файла и запрос для предварительной обработки данных с помощью `clickhouse-local`. Предварительная обработка включает: - разделение почтового индекса на два разных столбца `postcode1` и `postcode2`, что лучше подходит для хранения данных и выполнения запросов к ним; - преобразование поля `time` в `Date`, поскольку оно содержит только время 00:00; - поле `uuid` игнорируется, потому что оно не будет использовано нами для анализа; -- преобразование полей `type` and `duration` в более читаемые поля типа `Enum` с помощью функции `transform`; +- преобразование полей `type` и `duration` в более читаемые поля типа `Enum` с помощью функции `transform`; - преобразование полей `is_new` и `category` из односимвольной строки (`Y`/`N` и `A`/`B`) в поле `UInt8` со значениями 0 и 1 соответственно. -Предварительно обработанные данные передаются непосредственно в `clickhouse-client` для вставки в таблицу ClickHouse потоковым способом. +Обработанные данные передаются в `clickhouse-client` и импортируются в таблицу ClickHouse потоковым способом. ```bash clickhouse-local --input-format CSV --structure ' From 6a32a949e6d23b87295e4f59202cfeca752376ed Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Sep 2021 17:22:26 +0300 Subject: [PATCH 037/142] Fix read_rows for live view. --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 5f82104d914..1686e4ee0e2 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -271,7 +271,9 @@ public: void consume(Chunk chunk) override { + Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); StorageLiveView::writeIntoLiveView(live_view, getHeader().cloneWithColumns(chunk.detachColumns()), context); + CurrentThread::updateProgressIn(local_progress); } private: From 7fc830ae2b5e6b6d8abe3b74248f218380db960f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Sep 2021 19:28:41 +0300 Subject: [PATCH 038/142] Rename QueryPipeline to QueryPipelineBuilder. --- programs/client/Client.cpp | 4 +- src/DataStreams/BlockIO.h | 8 +- src/Dictionaries/CacheDictionary.cpp | 2 +- src/Dictionaries/PolygonDictionary.cpp | 2 +- src/Dictionaries/RangeHashedDictionary.cpp | 2 +- src/Interpreters/MutationsInterpreter.h | 6 +- src/Interpreters/executeQuery.cpp | 4 +- .../PipelineExecutingBlockInputStream.cpp | 6 +- .../PipelineExecutingBlockInputStream.h | 6 +- .../PullingAsyncPipelineExecutor.cpp | 4 +- .../Executors/PullingAsyncPipelineExecutor.h | 6 +- .../Executors/PullingPipelineExecutor.cpp | 4 +- .../Executors/PullingPipelineExecutor.h | 6 +- src/Processors/Pipe.cpp | 2 +- src/Processors/Pipe.h | 4 +- src/Processors/PipelineResourcesHolder.h | 2 +- ...yPipeline.cpp => QueryPipelineBuilder.cpp} | 76 +++++++++---------- ...QueryPipeline.h => QueryPipelineBuilder.h} | 32 ++++---- src/Processors/QueryPlan/AggregatingStep.cpp | 4 +- src/Processors/QueryPlan/AggregatingStep.h | 3 +- src/Processors/QueryPlan/ArrayJoinStep.cpp | 8 +- src/Processors/QueryPlan/ArrayJoinStep.h | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 10 +-- src/Processors/QueryPlan/CreatingSetsStep.h | 4 +- src/Processors/QueryPlan/CubeStep.cpp | 8 +- src/Processors/QueryPlan/CubeStep.h | 2 +- src/Processors/QueryPlan/DistinctStep.cpp | 8 +- src/Processors/QueryPlan/DistinctStep.h | 2 +- src/Processors/QueryPlan/ExpressionStep.cpp | 4 +- src/Processors/QueryPlan/ExpressionStep.h | 2 +- src/Processors/QueryPlan/ExtremesStep.cpp | 4 +- src/Processors/QueryPlan/ExtremesStep.h | 2 +- src/Processors/QueryPlan/FillingStep.cpp | 8 +- src/Processors/QueryPlan/FillingStep.h | 2 +- src/Processors/QueryPlan/FilterStep.cpp | 8 +- src/Processors/QueryPlan/FilterStep.h | 2 +- .../QueryPlan/FinishSortingStep.cpp | 8 +- src/Processors/QueryPlan/FinishSortingStep.h | 2 +- src/Processors/QueryPlan/IQueryPlanStep.h | 8 +- src/Processors/QueryPlan/ISourceStep.cpp | 6 +- src/Processors/QueryPlan/ISourceStep.h | 4 +- .../QueryPlan/ITransformingStep.cpp | 4 +- src/Processors/QueryPlan/ITransformingStep.h | 4 +- .../QueryPlan/IntersectOrExceptStep.cpp | 8 +- .../QueryPlan/IntersectOrExceptStep.h | 2 +- src/Processors/QueryPlan/JoinStep.cpp | 12 +-- src/Processors/QueryPlan/JoinStep.h | 4 +- src/Processors/QueryPlan/LimitByStep.cpp | 8 +- src/Processors/QueryPlan/LimitByStep.h | 2 +- src/Processors/QueryPlan/LimitStep.cpp | 4 +- src/Processors/QueryPlan/LimitStep.h | 2 +- src/Processors/QueryPlan/MergeSortingStep.cpp | 8 +- src/Processors/QueryPlan/MergeSortingStep.h | 3 +- .../QueryPlan/MergingAggregatedStep.cpp | 4 +- .../QueryPlan/MergingAggregatedStep.h | 2 +- .../QueryPlan/MergingSortedStep.cpp | 4 +- src/Processors/QueryPlan/MergingSortedStep.h | 4 +- src/Processors/QueryPlan/OffsetStep.cpp | 4 +- src/Processors/QueryPlan/OffsetStep.h | 2 +- .../QueryPlan/PartialSortingStep.cpp | 12 +-- src/Processors/QueryPlan/PartialSortingStep.h | 2 +- src/Processors/QueryPlan/QueryPlan.h | 6 +- .../QueryPlan/ReadFromMergeTree.cpp | 4 +- src/Processors/QueryPlan/ReadFromMergeTree.h | 2 +- .../QueryPlan/ReadFromPreparedSource.cpp | 4 +- .../QueryPlan/ReadFromPreparedSource.h | 2 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 4 +- src/Processors/QueryPlan/ReadFromRemote.h | 2 +- src/Processors/QueryPlan/ReadNothingStep.cpp | 4 +- src/Processors/QueryPlan/ReadNothingStep.h | 2 +- src/Processors/QueryPlan/RollupStep.cpp | 8 +- src/Processors/QueryPlan/RollupStep.h | 2 +- .../QueryPlan/SettingQuotaAndLimitsStep.cpp | 4 +- .../QueryPlan/SettingQuotaAndLimitsStep.h | 2 +- src/Processors/QueryPlan/TotalsHavingStep.cpp | 4 +- src/Processors/QueryPlan/TotalsHavingStep.h | 3 +- src/Processors/QueryPlan/UnionStep.cpp | 8 +- src/Processors/QueryPlan/UnionStep.h | 2 +- src/Processors/QueryPlan/WindowStep.cpp | 4 +- src/Processors/QueryPlan/WindowStep.h | 2 +- src/Storages/IStorage.h | 6 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 2 +- src/Storages/StorageMerge.cpp | 2 +- src/Storages/StorageS3.h | 2 +- src/Storages/StorageURL.cpp | 6 +- 85 files changed, 231 insertions(+), 238 deletions(-) rename src/Processors/{QueryPipeline.cpp => QueryPipelineBuilder.cpp} (86%) rename src/Processors/{QueryPipeline.h => QueryPipelineBuilder.h} (87%) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 568928c28a3..444c033318c 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -33,7 +33,7 @@ #include #include #include -#include +#include #include #include #include @@ -2034,7 +2034,7 @@ private: }); } - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(std::move(pipe)); PullingAsyncPipelineExecutor executor(pipeline); diff --git a/src/DataStreams/BlockIO.h b/src/DataStreams/BlockIO.h index aa4d481f68b..41dd65e8782 100644 --- a/src/DataStreams/BlockIO.h +++ b/src/DataStreams/BlockIO.h @@ -4,7 +4,7 @@ #include -#include +#include #include @@ -29,10 +29,10 @@ struct BlockIO Chain out; BlockInputStreamPtr in; - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; /// Callbacks for query logging could be set here. - std::function finish_callback; + std::function finish_callback; std::function exception_callback; /// When it is true, don't bother sending any non-empty blocks to the out stream @@ -43,7 +43,7 @@ struct BlockIO { if (finish_callback) { - QueryPipeline * pipeline_ptr = nullptr; + QueryPipelineBuilder * pipeline_ptr = nullptr; if (pipeline.initialized()) pipeline_ptr = &pipeline; diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 0b2044cfe2c..9d994209d49 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -14,7 +14,7 @@ #include #include -#include +#include namespace ProfileEvents { diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index f10aa071442..504319ce688 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -165,7 +165,7 @@ void IPolygonDictionary::blockToAttributes(const DB::Block & block) void IPolygonDictionary::loadData() { - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(source_ptr->loadAll()); PullingPipelineExecutor executor(pipeline); diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index 390871661c7..5ef2397fc5a 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -303,7 +303,7 @@ void RangeHashedDictionary::createAttributes() template void RangeHashedDictionary::loadData() { - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(source_ptr->loadAll()); PullingPipelineExecutor executor(pipeline); diff --git a/src/Interpreters/MutationsInterpreter.h b/src/Interpreters/MutationsInterpreter.h index 4f8960ae8f7..b0540f7d2ed 100644 --- a/src/Interpreters/MutationsInterpreter.h +++ b/src/Interpreters/MutationsInterpreter.h @@ -14,8 +14,8 @@ namespace DB class Context; class QueryPlan; -class QueryPipeline; -using QueryPipelinePtr = std::unique_ptr; +class QueryPipelineBuilder; +using QueryPipelineBuilderPtr = std::unique_ptr; /// Return false if the data isn't going to be changed by mutations. bool isStorageTouchedByMutations( @@ -84,7 +84,7 @@ private: struct Stage; ASTPtr prepareInterpreterSelectQuery(std::vector &prepared_stages, bool dry_run); - QueryPipelinePtr addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const; + QueryPipelineBuilderPtr addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const; std::optional getStorageSortDescriptionIfPossible(const Block & header) const; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index a16c6163188..31491fa12f8 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -587,7 +587,7 @@ static std::tuple executeQueryImpl( res = interpreter->execute(); } - QueryPipeline & pipeline = res.pipeline; + QueryPipelineBuilder & pipeline = res.pipeline; bool use_processors = pipeline.initialized(); if (const auto * insert_interpreter = typeid_cast(&*interpreter)) @@ -759,7 +759,7 @@ static std::tuple executeQueryImpl( log_queries_min_query_duration_ms = settings.log_queries_min_query_duration_ms.totalMilliseconds(), status_info_to_query_log ] - (IBlockInputStream * stream_in, QueryPipeline * query_pipeline) mutable + (IBlockInputStream * stream_in, QueryPipelineBuilder * query_pipeline) mutable { QueryStatus * process_list_elem = context->getProcessListElement(); diff --git a/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp b/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp index 736ad1ecefe..9881ce1806f 100644 --- a/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp +++ b/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include namespace DB { @@ -11,8 +11,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -PipelineExecutingBlockInputStream::PipelineExecutingBlockInputStream(QueryPipeline pipeline_) - : pipeline(std::make_unique(std::move(pipeline_))) +PipelineExecutingBlockInputStream::PipelineExecutingBlockInputStream(QueryPipelineBuilder pipeline_) + : pipeline(std::make_unique(std::move(pipeline_))) { } diff --git a/src/Processors/Executors/PipelineExecutingBlockInputStream.h b/src/Processors/Executors/PipelineExecutingBlockInputStream.h index 68497938ad4..ca9614c7ee9 100644 --- a/src/Processors/Executors/PipelineExecutingBlockInputStream.h +++ b/src/Processors/Executors/PipelineExecutingBlockInputStream.h @@ -4,7 +4,7 @@ namespace DB { -class QueryPipeline; +class QueryPipelineBuilder; class PullingAsyncPipelineExecutor; class PullingPipelineExecutor; @@ -13,7 +13,7 @@ class PullingPipelineExecutor; class PipelineExecutingBlockInputStream : public IBlockInputStream { public: - explicit PipelineExecutingBlockInputStream(QueryPipeline pipeline_); + explicit PipelineExecutingBlockInputStream(QueryPipelineBuilder pipeline_); ~PipelineExecutingBlockInputStream() override; String getName() const override { return "PipelineExecuting"; } @@ -32,7 +32,7 @@ protected: Block readImpl() override; private: - std::unique_ptr pipeline; + std::unique_ptr pipeline; /// One of executors is used. std::unique_ptr executor; /// for single thread. std::unique_ptr async_executor; /// for many threads. diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 8ecbe75af3a..92dd2501245 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include @@ -36,7 +36,7 @@ struct PullingAsyncPipelineExecutor::Data } }; -PullingAsyncPipelineExecutor::PullingAsyncPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) +PullingAsyncPipelineExecutor::PullingAsyncPipelineExecutor(QueryPipelineBuilder & pipeline_) : pipeline(pipeline_) { if (!pipeline.isCompleted()) { diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.h b/src/Processors/Executors/PullingAsyncPipelineExecutor.h index 2ce75aecab7..17cb6a6a7ae 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.h +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.h @@ -4,7 +4,7 @@ namespace DB { -class QueryPipeline; +class QueryPipelineBuilder; class Block; class Chunk; class LazyOutputFormat; @@ -20,7 +20,7 @@ struct BlockStreamProfileInfo; class PullingAsyncPipelineExecutor { public: - explicit PullingAsyncPipelineExecutor(QueryPipeline & pipeline_); + explicit PullingAsyncPipelineExecutor(QueryPipelineBuilder & pipeline_); ~PullingAsyncPipelineExecutor(); /// Get structure of returned block or chunk. @@ -50,7 +50,7 @@ public: struct Data; private: - QueryPipeline & pipeline; + QueryPipelineBuilder & pipeline; std::shared_ptr lazy_format; std::unique_ptr data; }; diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index dc59e0a2f5a..f13cfce3195 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -1,12 +1,12 @@ #include #include -#include +#include #include namespace DB { -PullingPipelineExecutor::PullingPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) +PullingPipelineExecutor::PullingPipelineExecutor(QueryPipelineBuilder & pipeline_) : pipeline(pipeline_) { pulling_format = std::make_shared(pipeline.getHeader(), has_data_flag); pipeline.setOutputFormat(pulling_format); diff --git a/src/Processors/Executors/PullingPipelineExecutor.h b/src/Processors/Executors/PullingPipelineExecutor.h index 878d66bd3d4..3646905f9f9 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.h +++ b/src/Processors/Executors/PullingPipelineExecutor.h @@ -7,7 +7,7 @@ namespace DB class Block; class Chunk; -class QueryPipeline; +class QueryPipelineBuilder; class PipelineExecutor; class PullingOutputFormat; struct BlockStreamProfileInfo; @@ -23,7 +23,7 @@ using PipelineExecutorPtr = std::shared_ptr; class PullingPipelineExecutor { public: - explicit PullingPipelineExecutor(QueryPipeline & pipeline_); + explicit PullingPipelineExecutor(QueryPipelineBuilder & pipeline_); ~PullingPipelineExecutor(); /// Get structure of returned block or chunk. @@ -50,7 +50,7 @@ public: private: std::atomic_bool has_data_flag = false; - QueryPipeline & pipeline; + QueryPipelineBuilder & pipeline; std::shared_ptr pulling_format; PipelineExecutorPtr executor; }; diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index 5038ee9f100..ec288484ca3 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -777,7 +777,7 @@ void Pipe::setOutputFormat(ProcessorPtr output) auto * format = dynamic_cast(output.get()); if (!format) - throw Exception("IOutputFormat processor expected for QueryPipeline::setOutputFormat.", + throw Exception("IOutputFormat processor expected for QueryPipelineBuilder::setOutputFormat.", ErrorCodes::LOGICAL_ERROR); auto & main = format->getPort(IOutputFormat::PortKind::Main); diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 1568622fbd5..66dcce9e277 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -14,8 +14,6 @@ struct StreamLocalLimits; class Pipe; using Pipes = std::vector; -class QueryPipeline; - using OutputPortRawPtrs = std::vector; /// Pipe is a set of processors which represents the part of pipeline. @@ -145,7 +143,7 @@ private: void setSinks(const Pipe::ProcessorGetterWithStreamKind & getter); void setOutputFormat(ProcessorPtr output); - friend class QueryPipeline; + friend class QueryPipelineBuilder; }; } diff --git a/src/Processors/PipelineResourcesHolder.h b/src/Processors/PipelineResourcesHolder.h index fe8e65996d0..9fb1438424a 100644 --- a/src/Processors/PipelineResourcesHolder.h +++ b/src/Processors/PipelineResourcesHolder.h @@ -5,7 +5,7 @@ namespace DB { -class QueryPipeline; +class QueryPipelineBuilder; class IStorage; using StoragePtr = std::shared_ptr; diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipelineBuilder.cpp similarity index 86% rename from src/Processors/QueryPipeline.cpp rename to src/Processors/QueryPipelineBuilder.cpp index afba81bfee9..b3f7290b4d3 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipelineBuilder.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -30,18 +30,18 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void QueryPipeline::addQueryPlan(std::unique_ptr plan) +void QueryPipelineBuilder::addQueryPlan(std::unique_ptr plan) { pipe.addQueryPlan(std::move(plan)); } -void QueryPipeline::checkInitialized() +void QueryPipelineBuilder::checkInitialized() { if (!initialized()) throw Exception("QueryPipeline wasn't initialized.", ErrorCodes::LOGICAL_ERROR); } -void QueryPipeline::checkInitializedAndNotCompleted() +void QueryPipelineBuilder::checkInitializedAndNotCompleted() { checkInitialized(); @@ -68,7 +68,7 @@ static void checkSource(const ProcessorPtr & source, bool can_have_totals) toString(source->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); } -void QueryPipeline::init(Pipe pipe_) +void QueryPipelineBuilder::init(Pipe pipe_) { if (initialized()) throw Exception("Pipeline has already been initialized.", ErrorCodes::LOGICAL_ERROR); @@ -79,43 +79,43 @@ void QueryPipeline::init(Pipe pipe_) pipe = std::move(pipe_); } -void QueryPipeline::reset() +void QueryPipelineBuilder::reset() { Pipe pipe_to_destroy(std::move(pipe)); - *this = QueryPipeline(); + *this = QueryPipelineBuilder(); } -void QueryPipeline::addSimpleTransform(const Pipe::ProcessorGetter & getter) +void QueryPipelineBuilder::addSimpleTransform(const Pipe::ProcessorGetter & getter) { checkInitializedAndNotCompleted(); pipe.addSimpleTransform(getter); } -void QueryPipeline::addSimpleTransform(const Pipe::ProcessorGetterWithStreamKind & getter) +void QueryPipelineBuilder::addSimpleTransform(const Pipe::ProcessorGetterWithStreamKind & getter) { checkInitializedAndNotCompleted(); pipe.addSimpleTransform(getter); } -void QueryPipeline::addTransform(ProcessorPtr transform) +void QueryPipelineBuilder::addTransform(ProcessorPtr transform) { checkInitializedAndNotCompleted(); pipe.addTransform(std::move(transform)); } -void QueryPipeline::addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes) +void QueryPipelineBuilder::addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes) { checkInitializedAndNotCompleted(); pipe.addTransform(std::move(transform), totals, extremes); } -void QueryPipeline::addChains(std::vector chains) +void QueryPipelineBuilder::addChains(std::vector chains) { checkInitializedAndNotCompleted(); pipe.addChains(std::move(chains)); } -void QueryPipeline::addChain(Chain chain) +void QueryPipelineBuilder::addChain(Chain chain) { checkInitializedAndNotCompleted(); std::vector chains; @@ -124,19 +124,19 @@ void QueryPipeline::addChain(Chain chain) pipe.addChains(std::move(chains)); } -void QueryPipeline::transform(const Transformer & transformer) +void QueryPipelineBuilder::transform(const Transformer & transformer) { checkInitializedAndNotCompleted(); pipe.transform(transformer); } -void QueryPipeline::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) +void QueryPipelineBuilder::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) { checkInitializedAndNotCompleted(); pipe.setSinks(getter); } -void QueryPipeline::addDelayedStream(ProcessorPtr source) +void QueryPipelineBuilder::addDelayedStream(ProcessorPtr source) { checkInitializedAndNotCompleted(); @@ -150,18 +150,18 @@ void QueryPipeline::addDelayedStream(ProcessorPtr source) addTransform(std::move(processor)); } -void QueryPipeline::addMergingAggregatedMemoryEfficientTransform(AggregatingTransformParamsPtr params, size_t num_merging_processors) +void QueryPipelineBuilder::addMergingAggregatedMemoryEfficientTransform(AggregatingTransformParamsPtr params, size_t num_merging_processors) { DB::addMergingAggregatedMemoryEfficientTransform(pipe, std::move(params), num_merging_processors); } -void QueryPipeline::resize(size_t num_streams, bool force, bool strict) +void QueryPipelineBuilder::resize(size_t num_streams, bool force, bool strict) { checkInitializedAndNotCompleted(); pipe.resize(num_streams, force, strict); } -void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform) +void QueryPipelineBuilder::addTotalsHavingTransform(ProcessorPtr transform) { checkInitializedAndNotCompleted(); @@ -178,7 +178,7 @@ void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform) pipe.addTransform(std::move(transform), totals_port, nullptr); } -void QueryPipeline::addDefaultTotals() +void QueryPipelineBuilder::addDefaultTotals() { checkInitializedAndNotCompleted(); @@ -200,13 +200,13 @@ void QueryPipeline::addDefaultTotals() pipe.addTotalsSource(std::move(source)); } -void QueryPipeline::dropTotalsAndExtremes() +void QueryPipelineBuilder::dropTotalsAndExtremes() { pipe.dropTotals(); pipe.dropExtremes(); } -void QueryPipeline::addExtremesTransform() +void QueryPipelineBuilder::addExtremesTransform() { checkInitializedAndNotCompleted(); @@ -222,7 +222,7 @@ void QueryPipeline::addExtremesTransform() pipe.addTransform(std::move(transform), nullptr, port); } -void QueryPipeline::setOutputFormat(ProcessorPtr output) +void QueryPipelineBuilder::setOutputFormat(ProcessorPtr output) { checkInitializedAndNotCompleted(); @@ -237,8 +237,8 @@ void QueryPipeline::setOutputFormat(ProcessorPtr output) initRowsBeforeLimit(); } -QueryPipeline QueryPipeline::unitePipelines( - std::vector> pipelines, +QueryPipelineBuilder QueryPipelineBuilder::unitePipelines( + std::vector> pipelines, size_t max_threads_limit, Processors * collected_processors) { @@ -271,7 +271,7 @@ QueryPipeline QueryPipeline::unitePipelines( max_threads_limit = pipeline.max_threads; } - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe::unitePipes(std::move(pipes), collected_processors, false)); if (will_limit_max_threads) @@ -283,9 +283,9 @@ QueryPipeline QueryPipeline::unitePipelines( return pipeline; } -std::unique_ptr QueryPipeline::joinPipelines( - std::unique_ptr left, - std::unique_ptr right, +std::unique_ptr QueryPipelineBuilder::joinPipelines( + std::unique_ptr left, + std::unique_ptr right, JoinPtr join, size_t max_block_size, Processors * collected_processors) @@ -375,7 +375,7 @@ std::unique_ptr QueryPipeline::joinPipelines( return left; } -void QueryPipeline::addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, ContextPtr context) +void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, ContextPtr context) { resize(1); @@ -394,7 +394,7 @@ void QueryPipeline::addCreatingSetsTransform(const Block & res_header, SubqueryF pipe.addTransform(std::move(transform), totals_port, nullptr); } -void QueryPipeline::addPipelineBefore(QueryPipeline pipeline) +void QueryPipelineBuilder::addPipelineBefore(QueryPipelineBuilder pipeline) { checkInitializedAndNotCompleted(); if (pipeline.getHeader()) @@ -409,14 +409,14 @@ void QueryPipeline::addPipelineBefore(QueryPipeline pipeline) Pipes pipes; pipes.emplace_back(std::move(pipe)); - pipes.emplace_back(QueryPipeline::getPipe(std::move(pipeline))); + pipes.emplace_back(QueryPipelineBuilder::getPipe(std::move(pipeline))); pipe = Pipe::unitePipes(std::move(pipes), collected_processors, true); auto processor = std::make_shared(getHeader(), pipe.numOutputPorts(), delayed_streams, true); addTransform(std::move(processor)); } -void QueryPipeline::setProgressCallback(const ProgressCallback & callback) +void QueryPipelineBuilder::setProgressCallback(const ProgressCallback & callback) { for (auto & processor : pipe.processors) { @@ -425,7 +425,7 @@ void QueryPipeline::setProgressCallback(const ProgressCallback & callback) } } -void QueryPipeline::setProcessListElement(QueryStatus * elem) +void QueryPipelineBuilder::setProcessListElement(QueryStatus * elem) { process_list_element = elem; @@ -436,7 +436,7 @@ void QueryPipeline::setProcessListElement(QueryStatus * elem) } } -void QueryPipeline::initRowsBeforeLimit() +void QueryPipelineBuilder::initRowsBeforeLimit() { RowsBeforeLimitCounterPtr rows_before_limit_at_least; @@ -530,7 +530,7 @@ void QueryPipeline::initRowsBeforeLimit() output_format->setRowsBeforeLimitCounter(rows_before_limit_at_least); } -PipelineExecutorPtr QueryPipeline::execute() +PipelineExecutorPtr QueryPipelineBuilder::execute() { if (!isCompleted()) throw Exception("Cannot execute pipeline because it is not completed.", ErrorCodes::LOGICAL_ERROR); @@ -538,13 +538,13 @@ PipelineExecutorPtr QueryPipeline::execute() return std::make_shared(pipe.processors, process_list_element); } -void QueryPipeline::setCollectedProcessors(Processors * processors) +void QueryPipelineBuilder::setCollectedProcessors(Processors * processors) { pipe.collected_processors = processors; } -QueryPipelineProcessorsCollector::QueryPipelineProcessorsCollector(QueryPipeline & pipeline_, IQueryPlanStep * step_) +QueryPipelineProcessorsCollector::QueryPipelineProcessorsCollector(QueryPipelineBuilder & pipeline_, IQueryPlanStep * step_) : pipeline(pipeline_), step(step_) { pipeline.setCollectedProcessors(&processors); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipelineBuilder.h similarity index 87% rename from src/Processors/QueryPipeline.h rename to src/Processors/QueryPipelineBuilder.h index ebd9141e370..fd44aea2d33 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipelineBuilder.h @@ -29,15 +29,15 @@ struct ExpressionActionsSettings; class IJoin; using JoinPtr = std::shared_ptr; -class QueryPipeline +class QueryPipelineBuilder { public: - QueryPipeline() = default; - ~QueryPipeline() = default; - QueryPipeline(QueryPipeline &&) = default; - QueryPipeline(const QueryPipeline &) = delete; - QueryPipeline & operator= (QueryPipeline && rhs) = default; - QueryPipeline & operator= (const QueryPipeline & rhs) = delete; + QueryPipelineBuilder() = default; + ~QueryPipelineBuilder() = default; + QueryPipelineBuilder(QueryPipelineBuilder &&) = default; + QueryPipelineBuilder(const QueryPipelineBuilder &) = delete; + QueryPipelineBuilder & operator= (QueryPipelineBuilder && rhs) = default; + QueryPipelineBuilder & operator= (const QueryPipelineBuilder & rhs) = delete; /// All pipes must have same header. void init(Pipe pipe); @@ -91,16 +91,16 @@ public: /// Unite several pipelines together. Result pipeline would have common_header structure. /// If collector is used, it will collect only newly-added processors, but not processors from pipelines. - static QueryPipeline unitePipelines( - std::vector> pipelines, + static QueryPipelineBuilder unitePipelines( + std::vector> pipelines, size_t max_threads_limit = 0, Processors * collected_processors = nullptr); /// Join two pipelines together using JoinPtr. /// If collector is used, it will collect only newly-added processors, but not processors from pipelines. - static std::unique_ptr joinPipelines( - std::unique_ptr left, - std::unique_ptr right, + static std::unique_ptr joinPipelines( + std::unique_ptr left, + std::unique_ptr right, JoinPtr join, size_t max_block_size, Processors * collected_processors = nullptr); @@ -108,7 +108,7 @@ public: /// Add other pipeline and execute it before current one. /// Pipeline must have empty header, it should not generate any chunk. /// This is used for CreatingSets. - void addPipelineBefore(QueryPipeline pipeline); + void addPipelineBefore(QueryPipelineBuilder pipeline); void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, ContextPtr context); @@ -154,7 +154,7 @@ public: } /// Convert query pipeline to pipe. - static Pipe getPipe(QueryPipeline pipeline) { return std::move(pipeline.pipe); } + static Pipe getPipe(QueryPipelineBuilder pipeline) { return std::move(pipeline.pipe); } private: @@ -182,13 +182,13 @@ private: class QueryPipelineProcessorsCollector { public: - explicit QueryPipelineProcessorsCollector(QueryPipeline & pipeline_, IQueryPlanStep * step_ = nullptr); + explicit QueryPipelineProcessorsCollector(QueryPipelineBuilder & pipeline_, IQueryPlanStep * step_ = nullptr); ~QueryPipelineProcessorsCollector(); Processors detachProcessors(size_t group = 0); private: - QueryPipeline & pipeline; + QueryPipelineBuilder & pipeline; IQueryPlanStep * step; Processors processors; }; diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 772390acb32..023f9016cc4 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -46,7 +46,7 @@ AggregatingStep::AggregatingStep( { } -void AggregatingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { QueryPipelineProcessorsCollector collector(pipeline, this); diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 696aabd4de7..8583e5be485 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -27,7 +27,7 @@ public: String getName() const override { return "Aggregating"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; @@ -57,4 +57,3 @@ private: }; } - diff --git a/src/Processors/QueryPlan/ArrayJoinStep.cpp b/src/Processors/QueryPlan/ArrayJoinStep.cpp index fa9ea298319..35b974baa83 100644 --- a/src/Processors/QueryPlan/ArrayJoinStep.cpp +++ b/src/Processors/QueryPlan/ArrayJoinStep.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -46,11 +46,11 @@ void ArrayJoinStep::updateInputStream(DataStream input_stream, Block result_head res_header = std::move(result_header); } -void ArrayJoinStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) +void ArrayJoinStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) { - bool on_totals = stream_type == QueryPipeline::StreamType::Totals; + bool on_totals = stream_type == QueryPipelineBuilder::StreamType::Totals; return std::make_shared(header, array_join, on_totals); }); diff --git a/src/Processors/QueryPlan/ArrayJoinStep.h b/src/Processors/QueryPlan/ArrayJoinStep.h index b3e08c2023c..83df4d021e8 100644 --- a/src/Processors/QueryPlan/ArrayJoinStep.h +++ b/src/Processors/QueryPlan/ArrayJoinStep.h @@ -13,7 +13,7 @@ public: explicit ArrayJoinStep(const DataStream & input_stream_, ArrayJoinActionPtr array_join_); String getName() const override { return "ArrayJoin"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 2f3287d178b..83a4c291bf2 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -44,7 +44,7 @@ CreatingSetStep::CreatingSetStep( { } -void CreatingSetStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, getContext()); } @@ -85,7 +85,7 @@ CreatingSetsStep::CreatingSetsStep(DataStreams input_streams_) input_streams[i].header.dumpStructure()); } -QueryPipelinePtr CreatingSetsStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &) +QueryPipelineBuilderPtr CreatingSetsStep::updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) { if (pipelines.empty()) throw Exception("CreatingSetsStep cannot be created with no inputs", ErrorCodes::LOGICAL_ERROR); @@ -96,11 +96,11 @@ QueryPipelinePtr CreatingSetsStep::updatePipeline(QueryPipelines pipelines, cons pipelines.erase(pipelines.begin()); - QueryPipeline delayed_pipeline; + QueryPipelineBuilder delayed_pipeline; if (pipelines.size() > 1) { QueryPipelineProcessorsCollector collector(delayed_pipeline, this); - delayed_pipeline = QueryPipeline::unitePipelines(std::move(pipelines)); + delayed_pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines)); processors = collector.detachProcessors(); } else diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index fa6d34ef667..8d20c764e8a 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -21,7 +21,7 @@ public: String getName() const override { return "CreatingSet"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; @@ -39,7 +39,7 @@ public: String getName() const override { return "CreatingSets"; } - QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &) override; + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) override; void describePipeline(FormatSettings & settings) const override; diff --git a/src/Processors/QueryPlan/CubeStep.cpp b/src/Processors/QueryPlan/CubeStep.cpp index 1a3016b7106..3d61d3ef36b 100644 --- a/src/Processors/QueryPlan/CubeStep.cpp +++ b/src/Processors/QueryPlan/CubeStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { @@ -30,13 +30,13 @@ CubeStep::CubeStep(const DataStream & input_stream_, AggregatingTransformParamsP output_stream->distinct_columns.insert(params->params.src_header.getByPosition(key).name); } -void CubeStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void CubeStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { pipeline.resize(1); - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { - if (stream_type == QueryPipeline::StreamType::Totals) + if (stream_type == QueryPipelineBuilder::StreamType::Totals) return nullptr; return std::make_shared(header, std::move(params)); diff --git a/src/Processors/QueryPlan/CubeStep.h b/src/Processors/QueryPlan/CubeStep.h index 0e06ffc598a..45077d78a90 100644 --- a/src/Processors/QueryPlan/CubeStep.h +++ b/src/Processors/QueryPlan/CubeStep.h @@ -17,7 +17,7 @@ public: String getName() const override { return "Cube"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; const Aggregator::Params & getParams() const; private: diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index 5edd2f52f47..d53d1fa9310 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include @@ -63,7 +63,7 @@ DistinctStep::DistinctStep( } } -void DistinctStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { if (checkColumnsAlreadyDistinct(columns, input_streams.front().distinct_columns)) return; @@ -71,9 +71,9 @@ void DistinctStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryP if (!pre_distinct) pipeline.resize(1); - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { - if (stream_type != QueryPipeline::StreamType::Main) + if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; return std::make_shared(header, set_size_limits, limit_hint, columns); diff --git a/src/Processors/QueryPlan/DistinctStep.h b/src/Processors/QueryPlan/DistinctStep.h index 815601d6253..b08e93dffa9 100644 --- a/src/Processors/QueryPlan/DistinctStep.h +++ b/src/Processors/QueryPlan/DistinctStep.h @@ -18,7 +18,7 @@ public: String getName() const override { return "Distinct"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 656dcd46fe9..b4ff1a1281c 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -52,7 +52,7 @@ void ExpressionStep::updateInputStream(DataStream input_stream, bool keep_header input_streams.emplace_back(std::move(input_stream)); } -void ExpressionStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) +void ExpressionStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { auto expression = std::make_shared(actions_dag, settings.getActionsSettings()); diff --git a/src/Processors/QueryPlan/ExpressionStep.h b/src/Processors/QueryPlan/ExpressionStep.h index 43272e19536..94c2ba21bc1 100644 --- a/src/Processors/QueryPlan/ExpressionStep.h +++ b/src/Processors/QueryPlan/ExpressionStep.h @@ -18,7 +18,7 @@ public: explicit ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_dag_); String getName() const override { return "Expression"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; void updateInputStream(DataStream input_stream, bool keep_header); diff --git a/src/Processors/QueryPlan/ExtremesStep.cpp b/src/Processors/QueryPlan/ExtremesStep.cpp index d3ec403f37e..117ccd414ca 100644 --- a/src/Processors/QueryPlan/ExtremesStep.cpp +++ b/src/Processors/QueryPlan/ExtremesStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { @@ -25,7 +25,7 @@ ExtremesStep::ExtremesStep(const DataStream & input_stream_) { } -void ExtremesStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void ExtremesStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { pipeline.addExtremesTransform(); } diff --git a/src/Processors/QueryPlan/ExtremesStep.h b/src/Processors/QueryPlan/ExtremesStep.h index 960b046b955..7898796306c 100644 --- a/src/Processors/QueryPlan/ExtremesStep.h +++ b/src/Processors/QueryPlan/ExtremesStep.h @@ -11,7 +11,7 @@ public: String getName() const override { return "Extremes"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; }; } diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index ba3588efa72..204559ecc3b 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include @@ -36,11 +36,11 @@ FillingStep::FillingStep(const DataStream & input_stream_, SortDescription sort_ throw Exception("FillingStep expects single input", ErrorCodes::LOGICAL_ERROR); } -void FillingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void FillingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { - bool on_totals = stream_type == QueryPipeline::StreamType::Totals; + bool on_totals = stream_type == QueryPipelineBuilder::StreamType::Totals; return std::make_shared(header, sort_description, on_totals); }); } diff --git a/src/Processors/QueryPlan/FillingStep.h b/src/Processors/QueryPlan/FillingStep.h index f4c6782e9df..6a5bca1890d 100644 --- a/src/Processors/QueryPlan/FillingStep.h +++ b/src/Processors/QueryPlan/FillingStep.h @@ -13,7 +13,7 @@ public: String getName() const override { return "Filling"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 15fd5c7b673..483055810cf 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -65,13 +65,13 @@ void FilterStep::updateInputStream(DataStream input_stream, bool keep_header) input_streams.emplace_back(std::move(input_stream)); } -void FilterStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) +void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { auto expression = std::make_shared(actions_dag, settings.getActionsSettings()); - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) { - bool on_totals = stream_type == QueryPipeline::StreamType::Totals; + bool on_totals = stream_type == QueryPipelineBuilder::StreamType::Totals; return std::make_shared(header, expression, filter_column_name, remove_filter_column, on_totals); }); diff --git a/src/Processors/QueryPlan/FilterStep.h b/src/Processors/QueryPlan/FilterStep.h index d01d128a08c..7ac5bc036e0 100644 --- a/src/Processors/QueryPlan/FilterStep.h +++ b/src/Processors/QueryPlan/FilterStep.h @@ -18,7 +18,7 @@ public: bool remove_filter_column_); String getName() const override { return "Filter"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; void updateInputStream(DataStream input_stream, bool keep_header); diff --git a/src/Processors/QueryPlan/FinishSortingStep.cpp b/src/Processors/QueryPlan/FinishSortingStep.cpp index 718eeb96cd8..c219c09f3bd 100644 --- a/src/Processors/QueryPlan/FinishSortingStep.cpp +++ b/src/Processors/QueryPlan/FinishSortingStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -54,7 +54,7 @@ void FinishSortingStep::updateLimit(size_t limit_) } } -void FinishSortingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void FinishSortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { bool need_finish_sorting = (prefix_description.size() < result_description.size()); if (pipeline.getNumStreams() > 1) @@ -74,9 +74,9 @@ void FinishSortingStep::transformPipeline(QueryPipeline & pipeline, const BuildQ if (need_finish_sorting) { - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { - if (stream_type != QueryPipeline::StreamType::Main) + if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; return std::make_shared(header, result_description, limit); diff --git a/src/Processors/QueryPlan/FinishSortingStep.h b/src/Processors/QueryPlan/FinishSortingStep.h index 5ea3a6d91b5..ac34aea9df4 100644 --- a/src/Processors/QueryPlan/FinishSortingStep.h +++ b/src/Processors/QueryPlan/FinishSortingStep.h @@ -19,7 +19,7 @@ public: String getName() const override { return "FinishSorting"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; diff --git a/src/Processors/QueryPlan/IQueryPlanStep.h b/src/Processors/QueryPlan/IQueryPlanStep.h index 9ff2b22e5b8..17bf6a4054e 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.h +++ b/src/Processors/QueryPlan/IQueryPlanStep.h @@ -8,9 +8,9 @@ namespace JSONBuilder { class JSONMap; } namespace DB { -class QueryPipeline; -using QueryPipelinePtr = std::unique_ptr; -using QueryPipelines = std::vector; +class QueryPipelineBuilder; +using QueryPipelineBuilderPtr = std::unique_ptr; +using QueryPipelineBuilders = std::vector; class IProcessor; using ProcessorPtr = std::shared_ptr; @@ -80,7 +80,7 @@ public: /// * header from each pipeline is the same as header from corresponding input_streams /// Result pipeline must contain any number of streams with compatible output header is hasOutputStream(), /// or pipeline should be completed otherwise. - virtual QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) = 0; + virtual QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings & settings) = 0; const DataStreams & getInputStreams() const { return input_streams; } diff --git a/src/Processors/QueryPlan/ISourceStep.cpp b/src/Processors/QueryPlan/ISourceStep.cpp index ec82e42fa34..61c0a9254cd 100644 --- a/src/Processors/QueryPlan/ISourceStep.cpp +++ b/src/Processors/QueryPlan/ISourceStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { @@ -9,9 +9,9 @@ ISourceStep::ISourceStep(DataStream output_stream_) output_stream = std::move(output_stream_); } -QueryPipelinePtr ISourceStep::updatePipeline(QueryPipelines, const BuildQueryPipelineSettings & settings) +QueryPipelineBuilderPtr ISourceStep::updatePipeline(QueryPipelineBuilders, const BuildQueryPipelineSettings & settings) { - auto pipeline = std::make_unique(); + auto pipeline = std::make_unique(); QueryPipelineProcessorsCollector collector(*pipeline, this); initializePipeline(*pipeline, settings); auto added_processors = collector.detachProcessors(); diff --git a/src/Processors/QueryPlan/ISourceStep.h b/src/Processors/QueryPlan/ISourceStep.h index fbef0fcce38..08c939b626d 100644 --- a/src/Processors/QueryPlan/ISourceStep.h +++ b/src/Processors/QueryPlan/ISourceStep.h @@ -10,9 +10,9 @@ class ISourceStep : public IQueryPlanStep public: explicit ISourceStep(DataStream output_stream_); - QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override; + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings & settings) override; - virtual void initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) = 0; + virtual void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) = 0; void describePipeline(FormatSettings & settings) const override; diff --git a/src/Processors/QueryPlan/ITransformingStep.cpp b/src/Processors/QueryPlan/ITransformingStep.cpp index e71afd94c46..1c7f836378f 100644 --- a/src/Processors/QueryPlan/ITransformingStep.cpp +++ b/src/Processors/QueryPlan/ITransformingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { @@ -36,7 +36,7 @@ DataStream ITransformingStep::createOutputStream( } -QueryPipelinePtr ITransformingStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) +QueryPipelineBuilderPtr ITransformingStep::updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings & settings) { if (collect_processors) { diff --git a/src/Processors/QueryPlan/ITransformingStep.h b/src/Processors/QueryPlan/ITransformingStep.h index 9abe025729d..d87ca05d4bc 100644 --- a/src/Processors/QueryPlan/ITransformingStep.h +++ b/src/Processors/QueryPlan/ITransformingStep.h @@ -48,9 +48,9 @@ public: ITransformingStep(DataStream input_stream, Block output_header, Traits traits, bool collect_processors_ = true); - QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override; + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings & settings) override; - virtual void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) = 0; + virtual void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) = 0; const TransformTraits & getTransformTraits() const { return transform_traits; } const DataStreamTraits & getDataStreamTraits() const { return data_stream_traits; } diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp index d1bb1eb41e9..a4d81e69fe0 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.cpp +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -39,9 +39,9 @@ IntersectOrExceptStep::IntersectOrExceptStep( output_stream = DataStream{.header = header}; } -QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &) +QueryPipelineBuilderPtr IntersectOrExceptStep::updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) { - auto pipeline = std::make_unique(); + auto pipeline = std::make_unique(); QueryPipelineProcessorsCollector collector(*pipeline, this); if (pipelines.empty()) @@ -72,7 +72,7 @@ QueryPipelinePtr IntersectOrExceptStep::updatePipeline(QueryPipelines pipelines, cur_pipeline->addTransform(std::make_shared(header, cur_pipeline->getNumStreams(), 1)); } - *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); + *pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines), max_threads); pipeline->addTransform(std::make_shared(header, current_operator)); processors = collector.detachProcessors(); diff --git a/src/Processors/QueryPlan/IntersectOrExceptStep.h b/src/Processors/QueryPlan/IntersectOrExceptStep.h index 9e87c921ab2..b2738cb297f 100644 --- a/src/Processors/QueryPlan/IntersectOrExceptStep.h +++ b/src/Processors/QueryPlan/IntersectOrExceptStep.h @@ -16,7 +16,7 @@ public: String getName() const override { return "IntersectOrExcept"; } - QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings & settings) override; + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings & settings) override; void describePipeline(FormatSettings & settings) const override; diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 736d7eb37c1..9c5f8ae2e5f 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include @@ -26,12 +26,12 @@ JoinStep::JoinStep( }; } -QueryPipelinePtr JoinStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &) +QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) { if (pipelines.size() != 2) throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); - return QueryPipeline::joinPipelines(std::move(pipelines[0]), std::move(pipelines[1]), join, max_block_size, &processors); + return QueryPipelineBuilder::joinPipelines(std::move(pipelines[0]), std::move(pipelines[1]), join, max_block_size, &processors); } void JoinStep::describePipeline(FormatSettings & settings) const @@ -67,7 +67,7 @@ FilledJoinStep::FilledJoinStep(const DataStream & input_stream_, JoinPtr join_, throw Exception(ErrorCodes::LOGICAL_ERROR, "FilledJoinStep expects Join to be filled"); } -void FilledJoinStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void FilledJoinStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { bool default_totals = false; if (!pipeline.hasTotals() && join->getTotals()) @@ -78,9 +78,9 @@ void FilledJoinStep::transformPipeline(QueryPipeline & pipeline, const BuildQuer auto finish_counter = std::make_shared(pipeline.getNumStreams()); - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) { - bool on_totals = stream_type == QueryPipeline::StreamType::Totals; + bool on_totals = stream_type == QueryPipelineBuilder::StreamType::Totals; auto counter = on_totals ? nullptr : finish_counter; return std::make_shared(header, join, max_block_size, on_totals, default_totals, counter); }); diff --git a/src/Processors/QueryPlan/JoinStep.h b/src/Processors/QueryPlan/JoinStep.h index 6430f7cbd59..71537f29a8e 100644 --- a/src/Processors/QueryPlan/JoinStep.h +++ b/src/Processors/QueryPlan/JoinStep.h @@ -20,7 +20,7 @@ public: String getName() const override { return "Join"; } - QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &) override; + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) override; void describePipeline(FormatSettings & settings) const override; @@ -40,7 +40,7 @@ public: FilledJoinStep(const DataStream & input_stream_, JoinPtr join_, size_t max_block_size_); String getName() const override { return "FilledJoin"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; private: JoinPtr join; diff --git a/src/Processors/QueryPlan/LimitByStep.cpp b/src/Processors/QueryPlan/LimitByStep.cpp index 8ded0784b41..12ad933a159 100644 --- a/src/Processors/QueryPlan/LimitByStep.cpp +++ b/src/Processors/QueryPlan/LimitByStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include @@ -34,13 +34,13 @@ LimitByStep::LimitByStep( } -void LimitByStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void LimitByStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { pipeline.resize(1); - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { - if (stream_type != QueryPipeline::StreamType::Main) + if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; return std::make_shared(header, group_length, group_offset, columns); diff --git a/src/Processors/QueryPlan/LimitByStep.h b/src/Processors/QueryPlan/LimitByStep.h index 1b574cd02a1..eb91be8a814 100644 --- a/src/Processors/QueryPlan/LimitByStep.h +++ b/src/Processors/QueryPlan/LimitByStep.h @@ -14,7 +14,7 @@ public: String getName() const override { return "LimitBy"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; diff --git a/src/Processors/QueryPlan/LimitStep.cpp b/src/Processors/QueryPlan/LimitStep.cpp index 5f5a0bd0d64..3db59e0684a 100644 --- a/src/Processors/QueryPlan/LimitStep.cpp +++ b/src/Processors/QueryPlan/LimitStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -43,7 +43,7 @@ void LimitStep::updateInputStream(DataStream input_stream) output_stream = createOutputStream(input_streams.front(), output_stream->header, getDataStreamTraits()); } -void LimitStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void LimitStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { auto transform = std::make_shared( pipeline.getHeader(), limit, offset, pipeline.getNumStreams(), always_read_till_end, with_ties, description); diff --git a/src/Processors/QueryPlan/LimitStep.h b/src/Processors/QueryPlan/LimitStep.h index 772ba0722a7..f5bceeb29c7 100644 --- a/src/Processors/QueryPlan/LimitStep.h +++ b/src/Processors/QueryPlan/LimitStep.h @@ -18,7 +18,7 @@ public: String getName() const override { return "Limit"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; diff --git a/src/Processors/QueryPlan/MergeSortingStep.cpp b/src/Processors/QueryPlan/MergeSortingStep.cpp index c9e141281f4..820bbc31b74 100644 --- a/src/Processors/QueryPlan/MergeSortingStep.cpp +++ b/src/Processors/QueryPlan/MergeSortingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -57,11 +57,11 @@ void MergeSortingStep::updateLimit(size_t limit_) } } -void MergeSortingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void MergeSortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { - if (stream_type == QueryPipeline::StreamType::Totals) + if (stream_type == QueryPipelineBuilder::StreamType::Totals) return nullptr; return std::make_shared( diff --git a/src/Processors/QueryPlan/MergeSortingStep.h b/src/Processors/QueryPlan/MergeSortingStep.h index dcecdffd122..d5daa041256 100644 --- a/src/Processors/QueryPlan/MergeSortingStep.h +++ b/src/Processors/QueryPlan/MergeSortingStep.h @@ -24,7 +24,7 @@ public: String getName() const override { return "MergeSorting"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; @@ -45,4 +45,3 @@ private: }; } - diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index 71efb37b363..d02be59ae84 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -40,7 +40,7 @@ MergingAggregatedStep::MergingAggregatedStep( output_stream->distinct_columns.insert(params->params.intermediate_header.getByPosition(key).name); } -void MergingAggregatedStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { if (!memory_efficient_aggregation) { diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.h b/src/Processors/QueryPlan/MergingAggregatedStep.h index 2e94d536a8c..9171512571a 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.h +++ b/src/Processors/QueryPlan/MergingAggregatedStep.h @@ -21,7 +21,7 @@ public: String getName() const override { return "MergingAggregated"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; diff --git a/src/Processors/QueryPlan/MergingSortedStep.cpp b/src/Processors/QueryPlan/MergingSortedStep.cpp index 7e866f4ccd2..87d1af4d2bd 100644 --- a/src/Processors/QueryPlan/MergingSortedStep.cpp +++ b/src/Processors/QueryPlan/MergingSortedStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -47,7 +47,7 @@ void MergingSortedStep::updateLimit(size_t limit_) } } -void MergingSortedStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void MergingSortedStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { /// If there are several streams, then we merge them into one if (pipeline.getNumStreams() > 1) diff --git a/src/Processors/QueryPlan/MergingSortedStep.h b/src/Processors/QueryPlan/MergingSortedStep.h index 4f82e3830d0..5d27e59ab76 100644 --- a/src/Processors/QueryPlan/MergingSortedStep.h +++ b/src/Processors/QueryPlan/MergingSortedStep.h @@ -19,7 +19,7 @@ public: String getName() const override { return "MergingSorted"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; @@ -34,5 +34,3 @@ private: }; } - - diff --git a/src/Processors/QueryPlan/OffsetStep.cpp b/src/Processors/QueryPlan/OffsetStep.cpp index 34ddb687ddd..b48327eb36c 100644 --- a/src/Processors/QueryPlan/OffsetStep.cpp +++ b/src/Processors/QueryPlan/OffsetStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include @@ -29,7 +29,7 @@ OffsetStep::OffsetStep(const DataStream & input_stream_, size_t offset_) { } -void OffsetStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void OffsetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { auto transform = std::make_shared( pipeline.getHeader(), offset, pipeline.getNumStreams()); diff --git a/src/Processors/QueryPlan/OffsetStep.h b/src/Processors/QueryPlan/OffsetStep.h index a10fcc7baec..488c55b6460 100644 --- a/src/Processors/QueryPlan/OffsetStep.h +++ b/src/Processors/QueryPlan/OffsetStep.h @@ -13,7 +13,7 @@ public: String getName() const override { return "Offset"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; diff --git a/src/Processors/QueryPlan/PartialSortingStep.cpp b/src/Processors/QueryPlan/PartialSortingStep.cpp index d713a63028a..cf7cb157e4c 100644 --- a/src/Processors/QueryPlan/PartialSortingStep.cpp +++ b/src/Processors/QueryPlan/PartialSortingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -47,11 +47,11 @@ void PartialSortingStep::updateLimit(size_t limit_) } } -void PartialSortingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void PartialSortingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { - if (stream_type != QueryPipeline::StreamType::Main) + if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; return std::make_shared(header, sort_description, limit); @@ -61,9 +61,9 @@ void PartialSortingStep::transformPipeline(QueryPipeline & pipeline, const Build limits.mode = LimitsMode::LIMITS_CURRENT; //-V1048 limits.size_limits = size_limits; - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { - if (stream_type != QueryPipeline::StreamType::Main) + if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; auto transform = std::make_shared(header, limits); diff --git a/src/Processors/QueryPlan/PartialSortingStep.h b/src/Processors/QueryPlan/PartialSortingStep.h index aeca42f7096..bd8fd30ce02 100644 --- a/src/Processors/QueryPlan/PartialSortingStep.h +++ b/src/Processors/QueryPlan/PartialSortingStep.h @@ -18,7 +18,7 @@ public: String getName() const override { return "PartialSorting"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 95034d34c9c..9a9730914bc 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -17,8 +17,8 @@ class DataStream; class IQueryPlanStep; using QueryPlanStepPtr = std::unique_ptr; -class QueryPipeline; -using QueryPipelinePtr = std::unique_ptr; +class QueryPipelineBuilder; +using QueryPipelineBuilderPtr = std::unique_ptr; class WriteBuffer; @@ -56,7 +56,7 @@ public: void optimize(const QueryPlanOptimizationSettings & optimization_settings); - QueryPipelinePtr buildQueryPipeline( + QueryPipelineBuilderPtr buildQueryPipeline( const QueryPlanOptimizationSettings & optimization_settings, const BuildQueryPipelineSettings & build_pipeline_settings); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 86d4a7c93b1..4e1e66c48d8 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -927,7 +927,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const return std::get(result_ptr->result); } -void ReadFromMergeTree::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { auto result = getAnalysisResult(); LOG_DEBUG( diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index fc06314ee0c..46b62467ae0 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -102,7 +102,7 @@ public: String getName() const override { return "ReadFromMergeTree"; } - void initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(FormatSettings & format_settings) const override; void describeIndexes(FormatSettings & format_settings) const override; diff --git a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp index 0d1a0fdc619..c8213d58db6 100644 --- a/src/Processors/QueryPlan/ReadFromPreparedSource.cpp +++ b/src/Processors/QueryPlan/ReadFromPreparedSource.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { @@ -11,7 +11,7 @@ ReadFromPreparedSource::ReadFromPreparedSource(Pipe pipe_, std::shared_ptrbuildQueryPipeline( + return QueryPipelineBuilder::getPipe(std::move(*plan->buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)))); } @@ -220,7 +220,7 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::IStreamFactory:: addConvertingActions(pipes.back(), output_stream->header); } -void ReadFromRemote::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void ReadFromRemote::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { Pipes pipes; for (const auto & shard : shards) diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index ba0060d5470..f963164dd3f 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -34,7 +34,7 @@ public: String getName() const override { return "ReadFromRemote"; } - void initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; private: ClusterProxy::IStreamFactory::Shards shards; diff --git a/src/Processors/QueryPlan/ReadNothingStep.cpp b/src/Processors/QueryPlan/ReadNothingStep.cpp index c0c97e5d523..7019b88f0b2 100644 --- a/src/Processors/QueryPlan/ReadNothingStep.cpp +++ b/src/Processors/QueryPlan/ReadNothingStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include namespace DB @@ -10,7 +10,7 @@ ReadNothingStep::ReadNothingStep(Block output_header) { } -void ReadNothingStep::initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void ReadNothingStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { pipeline.init(Pipe(std::make_shared(getOutputStream().header))); } diff --git a/src/Processors/QueryPlan/ReadNothingStep.h b/src/Processors/QueryPlan/ReadNothingStep.h index 4c5b4adb7ce..dad554e3d15 100644 --- a/src/Processors/QueryPlan/ReadNothingStep.h +++ b/src/Processors/QueryPlan/ReadNothingStep.h @@ -12,7 +12,7 @@ public: String getName() const override { return "ReadNothing"; } - void initializePipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; }; } diff --git a/src/Processors/QueryPlan/RollupStep.cpp b/src/Processors/QueryPlan/RollupStep.cpp index 45573b352d6..114fe661c70 100644 --- a/src/Processors/QueryPlan/RollupStep.cpp +++ b/src/Processors/QueryPlan/RollupStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { @@ -30,13 +30,13 @@ RollupStep::RollupStep(const DataStream & input_stream_, AggregatingTransformPar output_stream->distinct_columns.insert(params->params.src_header.getByPosition(key).name); } -void RollupStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void RollupStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { pipeline.resize(1); - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { - if (stream_type == QueryPipeline::StreamType::Totals) + if (stream_type == QueryPipelineBuilder::StreamType::Totals) return nullptr; return std::make_shared(header, std::move(params)); diff --git a/src/Processors/QueryPlan/RollupStep.h b/src/Processors/QueryPlan/RollupStep.h index 21faf539990..2ff3040d7a7 100644 --- a/src/Processors/QueryPlan/RollupStep.h +++ b/src/Processors/QueryPlan/RollupStep.h @@ -16,7 +16,7 @@ public: String getName() const override { return "Rollup"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; private: AggregatingTransformParamsPtr params; diff --git a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp index 734e6db318d..47f8187c3aa 100644 --- a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp +++ b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include namespace DB @@ -39,7 +39,7 @@ SettingQuotaAndLimitsStep::SettingQuotaAndLimitsStep( { } -void SettingQuotaAndLimitsStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void SettingQuotaAndLimitsStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { /// Table lock is stored inside pipeline here. pipeline.setLimits(limits); diff --git a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h index 3c73c208b70..b36ddfb3768 100644 --- a/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h +++ b/src/Processors/QueryPlan/SettingQuotaAndLimitsStep.h @@ -32,7 +32,7 @@ public: String getName() const override { return "SettingQuotaAndLimits"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; private: ContextPtr context; diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index db82538d5a0..e0388ed480e 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -49,7 +49,7 @@ TotalsHavingStep::TotalsHavingStep( { } -void TotalsHavingStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) +void TotalsHavingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { auto expression_actions = actions_dag ? std::make_shared(actions_dag, settings.getActionsSettings()) : nullptr; diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index bc053c96970..3d79d47ad22 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -24,7 +24,7 @@ public: String getName() const override { return "TotalsHaving"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings & settings) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; @@ -41,4 +41,3 @@ private: }; } - diff --git a/src/Processors/QueryPlan/UnionStep.cpp b/src/Processors/QueryPlan/UnionStep.cpp index 418e9b4e2e7..6d4036b1272 100644 --- a/src/Processors/QueryPlan/UnionStep.cpp +++ b/src/Processors/QueryPlan/UnionStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -37,9 +37,9 @@ UnionStep::UnionStep(DataStreams input_streams_, size_t max_threads_) output_stream = DataStream{.header = header}; } -QueryPipelinePtr UnionStep::updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &) +QueryPipelineBuilderPtr UnionStep::updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) { - auto pipeline = std::make_unique(); + auto pipeline = std::make_unique(); QueryPipelineProcessorsCollector collector(*pipeline, this); if (pipelines.empty()) @@ -71,7 +71,7 @@ QueryPipelinePtr UnionStep::updatePipeline(QueryPipelines pipelines, const Build } } - *pipeline = QueryPipeline::unitePipelines(std::move(pipelines), max_threads); + *pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines), max_threads); processors = collector.detachProcessors(); return pipeline; diff --git a/src/Processors/QueryPlan/UnionStep.h b/src/Processors/QueryPlan/UnionStep.h index 81bd033d045..c23223bc6fa 100644 --- a/src/Processors/QueryPlan/UnionStep.h +++ b/src/Processors/QueryPlan/UnionStep.h @@ -13,7 +13,7 @@ public: String getName() const override { return "Union"; } - QueryPipelinePtr updatePipeline(QueryPipelines pipelines, const BuildQueryPipelineSettings &) override; + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) override; void describePipeline(FormatSettings & settings) const override; diff --git a/src/Processors/QueryPlan/WindowStep.cpp b/src/Processors/QueryPlan/WindowStep.cpp index 29f2999ec83..ca09f4a9474 100644 --- a/src/Processors/QueryPlan/WindowStep.cpp +++ b/src/Processors/QueryPlan/WindowStep.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -63,7 +63,7 @@ WindowStep::WindowStep(const DataStream & input_stream_, } -void WindowStep::transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) +void WindowStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { // This resize is needed for cases such as `over ()` when we don't have a // sort node, and the input might have multiple streams. The sort node would diff --git a/src/Processors/QueryPlan/WindowStep.h b/src/Processors/QueryPlan/WindowStep.h index b5018b1d5a7..a65b157f481 100644 --- a/src/Processors/QueryPlan/WindowStep.h +++ b/src/Processors/QueryPlan/WindowStep.h @@ -20,7 +20,7 @@ public: String getName() const override { return "Window"; } - void transformPipeline(QueryPipeline & pipeline, const BuildQueryPipelineSettings &) override; + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index e1c0f030954..5e61466d2b6 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -54,8 +54,8 @@ using QueryPlanPtr = std::unique_ptr; class SinkToStorage; using SinkToStoragePtr = std::shared_ptr; -class QueryPipeline; -using QueryPipelinePtr = std::unique_ptr; +class QueryPipelineBuilder; +using QueryPipelineBuilderPtr = std::unique_ptr; class IStoragePolicy; using StoragePolicyPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 997e6e8bb74..88e27cef7bf 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index c1066329e6f..3a09e01f9d8 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -380,7 +380,7 @@ Pipe StorageMerge::createSources( if (!storage) { - pipe = QueryPipeline::getPipe(InterpreterSelectQuery( + pipe = QueryPipelineBuilder::getPipe(InterpreterSelectQuery( modified_query_info.query, modified_context, std::make_shared(header), SelectQueryOptions(processed_stage).analyze()).execute().pipeline); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index cfd7e496928..f3387301663 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -82,7 +82,7 @@ private: std::unique_ptr read_buf; - std::unique_ptr pipeline; + std::unique_ptr pipeline; std::unique_ptr reader; bool initialized = false; bool with_file_column = false; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 68d9d64297a..16b20ac78e0 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -20,7 +20,7 @@ #include #include -#include +#include #include #include #include @@ -105,7 +105,7 @@ namespace compression_method); auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size, format_settings); - pipeline = std::make_unique(); + pipeline = std::make_unique(); pipeline->init(Pipe(input_format)); pipeline->addSimpleTransform([&](const Block & cur_header) @@ -139,7 +139,7 @@ namespace private: String name; std::unique_ptr read_buf; - std::unique_ptr pipeline; + std::unique_ptr pipeline; std::unique_ptr reader; }; } From b9972146207933eb5b0f39d91e9e92a8a8c1d171 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Sep 2021 20:48:18 +0300 Subject: [PATCH 039/142] Rename QueryPipeline to QueryPipelineBuilder. --- programs/copier/ClusterCopier.cpp | 4 ++-- programs/obfuscator/Obfuscator.cpp | 6 +++--- src/Client/Connection.cpp | 8 ++++---- .../PushingToViewsBlockOutputStream.cpp | 2 +- src/DataStreams/ShellCommandSource.h | 4 ++-- src/DataStreams/TemporaryFileStream.h | 4 ++-- src/Databases/MySQL/DatabaseMySQL.cpp | 4 ++-- src/Databases/MySQL/FetchTablesColumnsList.cpp | 4 ++-- src/Databases/MySQL/MaterializeMetadata.cpp | 14 +++++++------- .../MySQL/MaterializedMySQLSyncThread.cpp | 8 ++++---- src/Dictionaries/CacheDictionary.cpp | 2 +- src/Dictionaries/ClickHouseDictionarySource.cpp | 6 +++--- src/Dictionaries/DictionaryHelpers.h | 4 ++-- src/Dictionaries/DirectDictionary.cpp | 6 +++--- src/Dictionaries/FlatDictionary.cpp | 6 +++--- src/Dictionaries/HashedDictionary.cpp | 4 ++-- src/Dictionaries/IPAddressDictionary.cpp | 2 +- src/Dictionaries/readInvalidateQuery.cpp | 4 ++-- src/Interpreters/InterpreterExplainQuery.cpp | 4 ++-- src/Interpreters/InterpreterInsertQuery.cpp | 4 ++-- src/Interpreters/MergeJoin.cpp | 4 ++-- src/Interpreters/MutationsInterpreter.cpp | 4 ++-- src/Interpreters/SortedBlocksWriter.cpp | 16 ++++++++-------- src/Interpreters/executeDDLQueryOnCluster.cpp | 2 +- src/Interpreters/executeQuery.h | 1 - src/Processors/QueryPlan/QueryPlan.cpp | 10 +++++----- src/Server/GRPCServer.cpp | 8 ++++---- src/Storages/IStorage.h | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 2 +- src/Storages/LiveView/StorageLiveView.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 ++-- .../MaterializedPostgreSQLConsumer.cpp | 2 +- .../PostgreSQL/PostgreSQLReplicationHandler.cpp | 4 ++-- src/Storages/StorageDistributed.cpp | 12 ++++++------ src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageFile.cpp | 6 +++--- src/Storages/StorageMerge.cpp | 2 +- src/Storages/StorageS3.cpp | 4 ++-- 38 files changed, 93 insertions(+), 94 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index d8d6cf6e867..bdca1e097c7 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include @@ -1527,7 +1527,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( context_insert->setSettings(task_cluster->settings_push); /// Custom INSERT SELECT implementation - QueryPipeline input; + QueryPipelineBuilder input; Chain output; { BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index b1acc34ef93..d7f7494a756 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -26,7 +26,7 @@ #include #include #include -#include +#include #include #include #include @@ -1162,7 +1162,7 @@ try Pipe pipe(FormatFactory::instance().getInput(input_format, file_in, header, context, max_block_size)); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(std::move(pipe)); PullingPipelineExecutor executor(pipeline); @@ -1200,7 +1200,7 @@ try }); } - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(std::move(pipe)); BlockOutputStreamPtr output = context->getOutputStreamParallelIfPossible(output_format, file_out, header); diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 366e61bc8e2..9cf0ddcdb48 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -23,7 +23,7 @@ #include #include #include -#include +#include #include #include #include @@ -689,14 +689,14 @@ void Connection::sendExternalTablesData(ExternalTablesData & data) if (!elem->pipe) elem->pipe = elem->creating_pipe_callback(); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(std::move(*elem->pipe)); elem->pipe.reset(); pipeline.resize(1); auto sink = std::make_shared(pipeline.getHeader(), *this, *elem, std::move(on_cancel)); - pipeline.setSinks([&](const Block &, QueryPipeline::StreamType type) -> ProcessorPtr + pipeline.setSinks([&](const Block &, QueryPipelineBuilder::StreamType type) -> ProcessorPtr { - if (type != QueryPipeline::StreamType::Main) + if (type != QueryPipelineBuilder::StreamType::Main) return nullptr; return sink; }); diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 1686e4ee0e2..c370f547f7f 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -409,7 +409,7 @@ Chain buildPushingToViewsDrain( ASTPtr insert_query_ptr(insert.release()); InterpreterInsertQuery interpreter(insert_query_ptr, insert_context, false, false, false, view_runtime_data); BlockIO io = interpreter.execute(); - io.out.attachResources(QueryPipeline::getPipe(std::move(io.pipeline)).detachResources()); + io.out.attachResources(QueryPipelineBuilder::getPipe(std::move(io.pipeline)).detachResources()); out = std::move(io.out); } else if (auto * live_view = dynamic_cast(dependent_table.get())) diff --git a/src/DataStreams/ShellCommandSource.h b/src/DataStreams/ShellCommandSource.h index 21d0acaf81a..752d3209fb2 100644 --- a/src/DataStreams/ShellCommandSource.h +++ b/src/DataStreams/ShellCommandSource.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include @@ -175,7 +175,7 @@ private: std::shared_ptr process_pool; - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; std::unique_ptr executor; std::vector send_data_threads; diff --git a/src/DataStreams/TemporaryFileStream.h b/src/DataStreams/TemporaryFileStream.h index ec38f6c1baa..74e00477789 100644 --- a/src/DataStreams/TemporaryFileStream.h +++ b/src/DataStreams/TemporaryFileStream.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include @@ -35,7 +35,7 @@ struct TemporaryFileStream {} /// Flush data from input stream into file for future reading - static void write(const std::string & path, const Block & header, QueryPipeline pipeline, const std::string & codec) + static void write(const std::string & path, const Block & header, QueryPipelineBuilder pipeline, const std::string & codec) { WriteBufferFromFile file_buf(path); CompressedWriteBuffer compressed_buf(file_buf, CompressionCodecFactory::instance().get(codec, {})); diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 793b4f15d60..c02d4acc7ae 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -13,7 +13,7 @@ # include # include # include -# include +# include # include # include # include @@ -284,7 +284,7 @@ std::map DatabaseMySQL::fetchTablesWithModificationTime(ContextP std::map tables_with_modification_time; StreamSettings mysql_input_stream_settings(local_context->getSettingsRef()); auto result = std::make_unique(mysql_pool.get(), query.str(), tables_status_sample_block, mysql_input_stream_settings); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe(std::move(result))); Block block; diff --git a/src/Databases/MySQL/FetchTablesColumnsList.cpp b/src/Databases/MySQL/FetchTablesColumnsList.cpp index c67dcefb433..8c338dbcbcb 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.cpp +++ b/src/Databases/MySQL/FetchTablesColumnsList.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include @@ -88,7 +88,7 @@ std::map fetchTablesColumnsList( StreamSettings mysql_input_stream_settings(settings); auto result = std::make_unique(pool.get(), query.str(), tables_columns_sample_block, mysql_input_stream_settings); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe(std::move(result))); Block block; diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index f684797c675..ed7a0fa4622 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -44,7 +44,7 @@ static std::unordered_map fetchTablesCreateQuery( connection, "SHOW CREATE TABLE " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(fetch_table_name), show_create_table_header, mysql_input_stream_settings); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe(std::move(show_create_table))); Block create_query_block; @@ -69,7 +69,7 @@ static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entr StreamSettings mysql_input_stream_settings(global_settings); auto input = std::make_unique(connection, query, header, mysql_input_stream_settings); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe(std::move(input))); Block block; @@ -97,7 +97,7 @@ void MaterializeMetadata::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & c StreamSettings mysql_input_stream_settings(settings, false, true); auto input = std::make_unique(connection, "SHOW MASTER STATUS;", header, mysql_input_stream_settings); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe(std::move(input))); Block master_status; @@ -125,7 +125,7 @@ void MaterializeMetadata::fetchMasterVariablesValue(const mysqlxx::PoolWithFailo const String & fetch_query = "SHOW VARIABLES WHERE Variable_name = 'binlog_checksum'"; StreamSettings mysql_input_stream_settings(settings, false, true); auto variables_input = std::make_unique(connection, fetch_query, variables_header, mysql_input_stream_settings); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe(std::move(variables_input))); Block variables_block; @@ -153,7 +153,7 @@ static bool checkSyncUserPrivImpl(const mysqlxx::PoolWithFailover::Entry & conne String grants_query, sub_privs; StreamSettings mysql_input_stream_settings(global_settings); auto input = std::make_unique(connection, "SHOW GRANTS FOR CURRENT_USER();", sync_user_privs_header, mysql_input_stream_settings); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe(std::move(input))); Block block; @@ -204,7 +204,7 @@ bool MaterializeMetadata::checkBinlogFileExists(const mysqlxx::PoolWithFailover: StreamSettings mysql_input_stream_settings(settings, false, true); auto input = std::make_unique(connection, "SHOW MASTER LOGS", logs_header, mysql_input_stream_settings); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe(std::move(input))); Block block; diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 0a1960ee014..dc4bb3862ff 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -9,7 +9,7 @@ # include # include # include -# include +# include # include # include # include @@ -114,7 +114,7 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S {"log_bin_use_v1_row_events", "OFF"} }; - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe(std::move(variables_input))); PullingPipelineExecutor executor(pipeline); @@ -343,7 +343,7 @@ static inline void dumpDataForTables( connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), chain.getInputHeader(), mysql_input_stream_settings); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe(std::move(input))); pipeline.addChain(std::move(chain)); pipeline.setSinks([&](const Block & header, Pipe::StreamType) @@ -808,7 +808,7 @@ void MaterializedMySQLSyncThread::Buffers::commit(ContextPtr context) auto query_context = createQueryContext(context); auto input = std::make_shared(table_name_and_buffer.second->first); auto out = getTableOutput(database, table_name_and_buffer.first, query_context, true); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe(std::move(input))); pipeline.addChain(std::move(out)); pipeline.setSinks([&](const Block & header, Pipe::StreamType) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 9d994209d49..4dbd8461efa 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -570,7 +570,7 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtrloadIds(requested_keys_vector)); diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 7348569442d..22ed68cea1f 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -162,7 +162,7 @@ std::string ClickHouseDictionarySource::toString() const Pipe ClickHouseDictionarySource::createStreamForQuery(const String & query, std::atomic * result_size_hint) { - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; /// Sample block should not contain first row default values auto empty_sample_block = sample_block.cloneEmpty(); @@ -194,7 +194,7 @@ Pipe ClickHouseDictionarySource::createStreamForQuery(const String & query, std: }); } - return QueryPipeline::getPipe(std::move(pipeline)); + return QueryPipelineBuilder::getPipe(std::move(pipeline)); } std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & request) const @@ -203,7 +203,7 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re if (configuration.is_local) { auto query_context = Context::createCopy(context); - auto pipe = QueryPipeline::getPipe(executeQuery(request, query_context, true).pipeline); + auto pipe = QueryPipelineBuilder::getPipe(executeQuery(request, query_context, true).pipeline); return readInvalidateQuery(std::move(pipe)); } else diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 4ed3b74226e..1609c1d6080 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -16,7 +16,7 @@ #include #include #include -#include +#include namespace DB @@ -567,7 +567,7 @@ void mergeBlockWithPipe( auto result_fetched_columns = block_to_update.cloneEmptyColumns(); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(std::move(pipe)); PullingPipelineExecutor executor(pipeline); diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 10e7414b42f..2630146fa3e 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -8,7 +8,7 @@ #include #include -#include +#include #include namespace DB @@ -68,7 +68,7 @@ Columns DirectDictionary::getColumns( size_t dictionary_keys_size = dict_struct.getKeysNames().size(); block_key_columns.reserve(dictionary_keys_size); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(getSourceBlockInputStream(key_columns, requested_keys)); PullingPipelineExecutor executor(pipeline); @@ -185,7 +185,7 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys( size_t dictionary_keys_size = dict_struct.getKeysNames().size(); block_key_columns.reserve(dictionary_keys_size); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(getSourceBlockInputStream(key_columns, requested_keys)); PullingPipelineExecutor executor(pipeline); diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 5ecf3299ea6..48022a0b239 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include #include @@ -322,7 +322,7 @@ void FlatDictionary::updateData() { if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) { - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(source_ptr->loadUpdatedAll()); PullingPipelineExecutor executor(pipeline); @@ -358,7 +358,7 @@ void FlatDictionary::loadData() { if (!source_ptr->hasUpdateField()) { - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(source_ptr->loadAll()); PullingPipelineExecutor executor(pipeline); diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index fd5865e24c0..712333fb38d 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -367,7 +367,7 @@ void HashedDictionary::updateData() if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) { - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(source_ptr->loadUpdatedAll()); PullingPipelineExecutor executor(pipeline); @@ -561,7 +561,7 @@ void HashedDictionary::loadData() { std::atomic new_size = 0; - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; if (configuration.preallocate) pipeline.init(source_ptr->loadAllWithSizeHint(&new_size)); else diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index b688362d048..25c1eba0a7a 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -352,7 +352,7 @@ void IPAddressDictionary::createAttributes() void IPAddressDictionary::loadData() { - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(source_ptr->loadAll()); std::vector ip_records; diff --git a/src/Dictionaries/readInvalidateQuery.cpp b/src/Dictionaries/readInvalidateQuery.cpp index bd1ec0e0983..4623c43c5fa 100644 --- a/src/Dictionaries/readInvalidateQuery.cpp +++ b/src/Dictionaries/readInvalidateQuery.cpp @@ -1,5 +1,5 @@ #include "readInvalidateQuery.h" -#include +#include #include #include #include @@ -17,7 +17,7 @@ namespace ErrorCodes std::string readInvalidateQuery(Pipe pipe) { - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(std::move(pipe)); PullingPipelineExecutor executor(pipeline); diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 2b2ed8040f4..da5c96776fd 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -318,7 +318,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() if (settings.graph) { /// Pipe holds QueryPlan, should not go out-of-scope - auto pipe = QueryPipeline::getPipe(std::move(*pipeline)); + auto pipe = QueryPipelineBuilder::getPipe(std::move(*pipeline)); const auto & processors = pipe.getProcessors(); if (settings.compact) @@ -337,7 +337,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() auto io = insert.execute(); if (io.pipeline.initialized()) { - auto pipe = QueryPipeline::getPipe(std::move(io.pipeline)); + auto pipe = QueryPipelineBuilder::getPipe(std::move(io.pipeline)); const auto & processors = pipe.getProcessors(); printPipeline(processors, buf); } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 4f3a7d3e4f3..65658a0b295 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -355,7 +355,7 @@ BlockIO InterpreterInsertQuery::execute() res.pipeline.addChains(std::move(out_chains)); - res.pipeline.setSinks([&](const Block & cur_header, QueryPipeline::StreamType) -> ProcessorPtr + res.pipeline.setSinks([&](const Block & cur_header, QueryPipelineBuilder::StreamType) -> ProcessorPtr { return std::make_shared(cur_header); }); @@ -396,7 +396,7 @@ BlockIO InterpreterInsertQuery::execute() } if (!res.out.empty()) - res.out.attachResources(QueryPipeline::getPipe(std::move(res.pipeline)).detachResources()); + res.out.attachResources(QueryPipelineBuilder::getPipe(std::move(res.pipeline)).detachResources()); return res; } diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 0150bbe1d93..ba4d23f5491 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include @@ -580,7 +580,7 @@ void MergeJoin::mergeInMemoryRightBlocks() Pipe source(std::make_shared(std::move(right_blocks.blocks))); right_blocks.clear(); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(std::move(source)); /// TODO: there should be no split keys by blocks for RIGHT|FULL JOIN diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index b90a1d10d2d..d008fec0a85 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include @@ -852,7 +852,7 @@ ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector & return select; } -QueryPipelinePtr MutationsInterpreter::addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const +QueryPipelineBuilderPtr MutationsInterpreter::addStreamsForLaterStages(const std::vector & prepared_stages, QueryPlan & plan) const { for (size_t i_stage = 1; i_stage < prepared_stages.size(); ++i_stage) { diff --git a/src/Interpreters/SortedBlocksWriter.cpp b/src/Interpreters/SortedBlocksWriter.cpp index 3ce9f2d1b90..c38402f7b72 100644 --- a/src/Interpreters/SortedBlocksWriter.cpp +++ b/src/Interpreters/SortedBlocksWriter.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -15,7 +15,7 @@ namespace DB namespace { -std::unique_ptr flushToFile(const String & tmp_path, const Block & header, QueryPipeline pipeline, const String & codec) +std::unique_ptr flushToFile(const String & tmp_path, const Block & header, QueryPipelineBuilder pipeline, const String & codec) { auto tmp_file = createTemporaryFile(tmp_path); @@ -24,7 +24,7 @@ std::unique_ptr flushToFile(const String & tmp_path, const Block return tmp_file; } -SortedBlocksWriter::SortedFiles flushToManyFiles(const String & tmp_path, const Block & header, QueryPipeline pipeline, +SortedBlocksWriter::SortedFiles flushToManyFiles(const String & tmp_path, const Block & header, QueryPipelineBuilder pipeline, const String & codec, std::function callback = [](const Block &){}) { std::vector> files; @@ -38,7 +38,7 @@ SortedBlocksWriter::SortedFiles flushToManyFiles(const String & tmp_path, const callback(block); - QueryPipeline one_block_pipeline; + QueryPipelineBuilder one_block_pipeline; Chunk chunk(block.getColumns(), block.rows()); one_block_pipeline.init(Pipe(std::make_shared(block.cloneEmpty(), std::move(chunk)))); auto tmp_file = flushToFile(tmp_path, header, std::move(one_block_pipeline), codec); @@ -126,7 +126,7 @@ SortedBlocksWriter::TmpFilePtr SortedBlocksWriter::flush(const BlocksList & bloc if (pipes.empty()) return {}; - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe::unitePipes(std::move(pipes))); if (pipeline.getNumStreams() > 1) @@ -179,7 +179,7 @@ SortedBlocksWriter::PremergedFiles SortedBlocksWriter::premerge() if (pipes.size() == num_files_for_merge || &file == &files.back()) { - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe::unitePipes(std::move(pipes))); pipes = Pipes(); @@ -212,7 +212,7 @@ SortedBlocksWriter::PremergedFiles SortedBlocksWriter::premerge() SortedBlocksWriter::SortedFiles SortedBlocksWriter::finishMerge(std::function callback) { PremergedFiles files = premerge(); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(std::move(files.pipe)); if (pipeline.getNumStreams() > 1) @@ -293,7 +293,7 @@ Block SortedBlocksBuffer::mergeBlocks(Blocks && blocks) const Blocks tmp_blocks; - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe::unitePipes(std::move(pipes))); if (pipeline.getNumStreams() > 1) diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 180a4f9af3e..255d10d14cf 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -216,7 +216,7 @@ BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & en io.pipeline.init(Pipe{processor}); if (context->getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::NONE) - io.pipeline.setSinks([](const Block & header, QueryPipeline::StreamType){ return std::make_shared(header); }); + io.pipeline.setSinks([](const Block & header, QueryPipelineBuilder::StreamType){ return std::make_shared(header); }); return io; } diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index 9672c4e7517..64261d044c7 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -2,7 +2,6 @@ #include #include -#include namespace DB { diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index bc3b8458531..144783fc5ba 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -134,7 +134,7 @@ void QueryPlan::addStep(QueryPlanStepPtr step) " input expected", ErrorCodes::LOGICAL_ERROR); } -QueryPipelinePtr QueryPlan::buildQueryPipeline( +QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline( const QueryPlanOptimizationSettings & optimization_settings, const BuildQueryPipelineSettings & build_pipeline_settings) { @@ -144,10 +144,10 @@ QueryPipelinePtr QueryPlan::buildQueryPipeline( struct Frame { Node * node = {}; - QueryPipelines pipelines = {}; + QueryPipelineBuilders pipelines = {}; }; - QueryPipelinePtr last_pipeline; + QueryPipelineBuilderPtr last_pipeline; std::stack stack; stack.push(Frame{.node = root}); @@ -193,7 +193,7 @@ Pipe QueryPlan::convertToPipe( if (isCompleted()) throw Exception("Cannot convert completed QueryPlan to Pipe", ErrorCodes::LOGICAL_ERROR); - return QueryPipeline::getPipe(std::move(*buildQueryPipeline(optimization_settings, build_pipeline_settings))); + return QueryPipelineBuilder::getPipe(std::move(*buildQueryPipeline(optimization_settings, build_pipeline_settings))); } void QueryPlan::addInterpreterContext(std::shared_ptr context) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index a150521ff19..6631da60385 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -27,7 +27,7 @@ #include #include #include -#include +#include #include #include #include @@ -589,7 +589,7 @@ namespace std::optional read_buffer; std::optional write_buffer; - std::unique_ptr pipeline; + std::unique_ptr pipeline; std::unique_ptr pipeline_executor; BlockOutputStreamPtr block_output_stream; bool need_input_data_from_insert_query = true; @@ -896,7 +896,7 @@ namespace }); assert(!pipeline); - pipeline = std::make_unique(); + pipeline = std::make_unique(); auto source = FormatFactory::instance().getInput( input_format, *read_buffer, header, query_context, query_context->getSettings().max_insert_block_size); pipeline->init(Pipe(source)); @@ -986,7 +986,7 @@ namespace format, data, metadata_snapshot->getSampleBlock(), external_table_context, external_table_context->getSettings().max_insert_block_size); - QueryPipeline cur_pipeline; + QueryPipelineBuilder cur_pipeline; cur_pipeline.init(Pipe(std::move(in))); cur_pipeline.addTransform(std::move(sink)); cur_pipeline.setSinks([&](const Block & header, Pipe::StreamType) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 5e61466d2b6..8c8f113fa8f 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -359,7 +359,7 @@ public: * * Returns query pipeline if distributed writing is possible, and nullptr otherwise. */ - virtual QueryPipelinePtr distributedWrite( + virtual QueryPipelineBuilderPtr distributedWrite( const ASTInsertQuery & /*query*/, ContextPtr /*context*/) { diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index d6da59b596f..ebfb01f0934 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -143,7 +143,7 @@ Pipes StorageLiveView::blocksToPipes(BlocksPtrs blocks, Block & sample_block) } /// Complete query using input streams from mergeable blocks -QueryPipeline StorageLiveView::completeQuery(Pipes pipes) +QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes) { //FIXME it's dangerous to create Context on stack auto block_context = Context::createCopy(getContext()); diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 15afc642989..b08b034ec3f 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -167,7 +167,7 @@ public: /// Collect mergeable blocks and their sample. Must be called holding mutex MergeableBlocksPtr collectMergeableBlocks(ContextPtr context); /// Complete query using input streams from mergeable blocks - QueryPipeline completeQuery(Pipes pipes); + QueryPipelineBuilder completeQuery(Pipes pipes); void setMergeableBlocks(MergeableBlocksPtr blocks) { mergeable_blocks = blocks; } std::shared_ptr getActivePtr() { return active_ptr; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 18691c5b571..56383ae7d5d 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -924,7 +924,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor break; } - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe::unitePipes(std::move(pipes))); pipeline.addTransform(std::move(merged_transform)); pipeline.setMaxThreads(1); @@ -1049,7 +1049,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor column_part_source->setProgressCallback( MergeProgressCallback(merge_entry, watch_prev_elapsed, column_progress)); - QueryPipeline column_part_pipeline; + QueryPipelineBuilder column_part_pipeline; column_part_pipeline.init(Pipe(std::move(column_part_source))); column_part_pipeline.setMaxThreads(1); diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 8f47daa6059..83835a048c3 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -490,7 +490,7 @@ void MaterializedPostgreSQLConsumer::syncTables() result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); assertBlocksHaveEqualStructure(input->getPort().getHeader(), block_io.out.getInputHeader(), "postgresql replica table sync"); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe(std::move(input))); pipeline.addChain(std::move(block_io.out)); pipeline.setSinks([&](const Block & header, Pipe::StreamType) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 277f1e2358c..8490d39c917 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,7 +1,7 @@ #include "PostgreSQLReplicationHandler.h" #include -#include +#include #include #include #include @@ -251,7 +251,7 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(String & snapshot_name auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); auto input = std::make_unique>(tx, query_str, sample_block, DEFAULT_BLOCK_SIZE); - QueryPipeline pipeline; + QueryPipelineBuilder pipeline; pipeline.init(Pipe(std::move(input))); assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out.getInputHeader(), "postgresql replica load from snapshot"); pipeline.addChain(std::move(block_io.out)); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 87be17de4ce..60459900688 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -696,7 +696,7 @@ SinkToStoragePtr StorageDistributed::write(const ASTPtr &, const StorageMetadata } -QueryPipelinePtr StorageDistributed::distributedWrite(const ASTInsertQuery & query, ContextPtr local_context) +QueryPipelineBuilderPtr StorageDistributed::distributedWrite(const ASTInsertQuery & query, ContextPtr local_context) { const Settings & settings = local_context->getSettingsRef(); std::shared_ptr storage_src; @@ -740,7 +740,7 @@ QueryPipelinePtr StorageDistributed::distributedWrite(const ASTInsertQuery & que const auto & cluster = getCluster(); const auto & shards_info = cluster->getShardsInfo(); - std::vector> pipelines; + std::vector> pipelines; String new_query_str = queryToString(new_query); for (size_t shard_index : collections::range(0, shards_info.size())) @@ -749,7 +749,7 @@ QueryPipelinePtr StorageDistributed::distributedWrite(const ASTInsertQuery & que if (shard_info.isLocal()) { InterpreterInsertQuery interpreter(new_query, local_context); - pipelines.emplace_back(std::make_unique(interpreter.execute().pipeline)); + pipelines.emplace_back(std::make_unique(interpreter.execute().pipeline)); } else { @@ -762,16 +762,16 @@ QueryPipelinePtr StorageDistributed::distributedWrite(const ASTInsertQuery & que /// INSERT SELECT query returns empty block auto remote_query_executor = std::make_shared(shard_info.pool, std::move(connections), new_query_str, Block{}, local_context); - pipelines.emplace_back(std::make_unique()); + pipelines.emplace_back(std::make_unique()); pipelines.back()->init(Pipe(std::make_shared(remote_query_executor, false, settings.async_socket_for_remote))); - pipelines.back()->setSinks([](const Block & header, QueryPipeline::StreamType) -> ProcessorPtr + pipelines.back()->setSinks([](const Block & header, QueryPipelineBuilder::StreamType) -> ProcessorPtr { return std::make_shared(header); }); } } - return std::make_unique(QueryPipeline::unitePipelines(std::move(pipelines))); + return std::make_unique(QueryPipelineBuilder::unitePipelines(std::move(pipelines))); } diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 80800e50059..3f785d54422 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -83,7 +83,7 @@ public: SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override; - QueryPipelinePtr distributedWrite(const ASTInsertQuery & query, ContextPtr context) override; + QueryPipelineBuilderPtr distributedWrite(const ASTInsertQuery & query, ContextPtr context) override; /// Removes temporary data in local filesystem. void truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) override; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 296ec7cfe24..997f52e5d70 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -336,7 +336,7 @@ public: /// Special case for distributed format. Defaults are not needed here. if (storage->format_name == "Distributed") { - pipeline = std::make_unique(); + pipeline = std::make_unique(); pipeline->init(Pipe(StorageDistributedDirectoryMonitor::createSourceFromFile(current_path))); reader = std::make_unique(*pipeline); continue; @@ -394,7 +394,7 @@ public: auto format = FormatFactory::instance().getInput( storage->format_name, *read_buf, get_block_for_format(), context, max_block_size, storage->format_settings); - pipeline = std::make_unique(); + pipeline = std::make_unique(); pipeline->init(Pipe(format)); if (columns_description.hasDefaults()) @@ -454,7 +454,7 @@ private: String current_path; Block sample_block; std::unique_ptr read_buf; - std::unique_ptr pipeline; + std::unique_ptr pipeline; std::unique_ptr reader; ColumnsDescription columns_description; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 3a09e01f9d8..3e8c2738eb8 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -424,7 +424,7 @@ Pipe StorageMerge::createSources( InterpreterSelectQuery interpreter{modified_query_info.query, modified_context, SelectQueryOptions(processed_stage)}; - pipe = QueryPipeline::getPipe(interpreter.execute().pipeline); + pipe = QueryPipelineBuilder::getPipe(interpreter.execute().pipeline); /** Materialization is needed, since from distributed storage the constants come materialized. * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 765f45be165..5061d7a58b9 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -34,7 +34,7 @@ #include #include -#include +#include #include #include @@ -233,7 +233,7 @@ bool StorageS3Source::initialize() std::make_unique(client, bucket, current_key, max_single_read_retries, DBMS_DEFAULT_BUFFER_SIZE), chooseCompressionMethod(current_key, compression_hint)); auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, getContext(), max_block_size, format_settings); - pipeline = std::make_unique(); + pipeline = std::make_unique(); pipeline->init(Pipe(input_format)); if (columns_desc.hasDefaults()) From 69c896df668ce9fbdd83c17f8d2196d000249ebb Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:45:11 +0300 Subject: [PATCH 040/142] Update docs/ru/getting-started/example-datasets/opensky.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/opensky.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index c8f1a19645a..34785cda1ad 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -22,7 +22,7 @@ https://doi.org/10.5194/essd-13-357-2021 wget -O- https://zenodo.org/record/5092942 | grep -oP 'https://zenodo.org/record/5092942/files/flightlist_\d+_\d+\.csv\.gz' | xargs wget ``` -Загрузка займет около 2 минут при хорошем подключении к Интернету. Будет загружено 30 файлов, общим размером 4,3 ГБ. +Загрузка займет около 2 минут при хорошем подключении к интернету. Будет загружено 30 файлов общим размером 4,3 ГБ. ## Создайте таблицу From 53c99d748d14aef4d2f069ba5c3e74713cc6b3d6 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:45:20 +0300 Subject: [PATCH 041/142] Update docs/ru/getting-started/example-datasets/opensky.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/opensky.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index 34785cda1ad..aa8ee3b2ff2 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -50,7 +50,7 @@ CREATE TABLE opensky ## Импортируйте данные в ClickHouse -Загрузите параллельными потоками данные в ClickHouse: +Загрузите данные в ClickHouse параллельными потоками: ```bash ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c 'gzip -c -d "{}" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"' From d196b7abbc6ad74d32f4a9e3e9f6eeaa306b2d3e Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:45:37 +0300 Subject: [PATCH 042/142] Update docs/ru/getting-started/example-datasets/opensky.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/opensky.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index aa8ee3b2ff2..abde28f478c 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -58,7 +58,7 @@ ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c 'gzip -c -d "{}" | clickhou - Список файлов передаётся (`ls -1 flightlist_*.csv.gz`) в `xargs` для параллельной обработки. - `xargs -P100` указывает на использование до 100 параллельных обработчиков, но поскольку у нас всего 30 файлов, то количество обработчиков будет всего 30. -- Для каждого файла `xargs` будет запускать скрипт с `bash -c`. Сценарий имеет подстановку в виде ` {}`, а команда `xargs` заменяет имя файла (мы указали это для xargs с помощью `-I{}`). +- Для каждого файла `xargs` будет запускать скрипт с `bash -c`. Сценарий имеет подстановку в виде `{}`, а команда `xargs` заменяет имя файла на указанные в подстановке символы (мы указали это для `xargs` с помощью `-I{}`). - Скрипт распакует файл (`gzip -c -d "{}"`) в стандартный вывод (параметр`-c`) и перенаправит его в `clickhouse-client`. - Чтобы распознать формат ISO-8601 со смещениями часовых поясов в полях типа `DateTime`, указывается параметр парсера `--date_time_input_format best_effort`. From 7419f76c5fa14f835a8caaf426e727dae71b02dd Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:45:45 +0300 Subject: [PATCH 043/142] Update docs/ru/getting-started/example-datasets/opensky.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/opensky.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index abde28f478c..5e94e4f2fc9 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -59,7 +59,7 @@ ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c 'gzip -c -d "{}" | clickhou - Список файлов передаётся (`ls -1 flightlist_*.csv.gz`) в `xargs` для параллельной обработки. - `xargs -P100` указывает на использование до 100 параллельных обработчиков, но поскольку у нас всего 30 файлов, то количество обработчиков будет всего 30. - Для каждого файла `xargs` будет запускать скрипт с `bash -c`. Сценарий имеет подстановку в виде `{}`, а команда `xargs` заменяет имя файла на указанные в подстановке символы (мы указали это для `xargs` с помощью `-I{}`). -- Скрипт распакует файл (`gzip -c -d "{}"`) в стандартный вывод (параметр`-c`) и перенаправит его в `clickhouse-client`. +- Скрипт распакует файл (`gzip -c -d "{}"`) в стандартный вывод (параметр `-c`) и перенаправит его в `clickhouse-client`. - Чтобы распознать формат ISO-8601 со смещениями часовых поясов в полях типа `DateTime`, указывается параметр парсера `--date_time_input_format best_effort`. В итоге: клиент clickhouse добавит данные в таблицу `opensky`. Входные данные импортируются в формате [CSVWithNames](../../interfaces/formats.md#csvwithnames). From cdbbf76d4c912f9e692580d1c2f4b90c29144a54 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:45:56 +0300 Subject: [PATCH 044/142] Update docs/en/getting-started/example-datasets/opensky.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/getting-started/example-datasets/opensky.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md index ede9a4093d7..119b5cc4b41 100644 --- a/docs/en/getting-started/example-datasets/opensky.md +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -58,9 +58,9 @@ ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c 'gzip -c -d "{}" | clickhou - Here we pass the list of files (`ls -1 flightlist_*.csv.gz`) to `xargs` for parallel processing. `xargs -P100` specifies to use up to 100 parallel workers but as we only have 30 files, the number of workers will be only 30. -- For every file, `xargs` will run a script with `bash -c`. The script has substitution in form of `{}` and the `xargs` command will substitute the filename to it (we have asked it for xargs with `-I{}`). +- For every file, `xargs` will run a script with `bash -c`. The script has substitution in form of `{}` and the `xargs` command will substitute the filename to it (we have asked it for `xargs` with `-I{}`). - The script will decompress the file (`gzip -c -d "{}"`) to standard output (`-c` parameter) and the output is redirected to `clickhouse-client`. -- We also asked to parse DateTime fields with extended parser (`--date_time_input_format best_effort`) to recognize ISO-8601 format with timezone offsets. +- We also asked to parse `DateTime` fields with extended parser (`--date_time_input_format best_effort`) to recognize ISO-8601 format with timezone offsets. Finally, `clickhouse-client` will do insertion. It will read input data in [CSVWithNames](../../interfaces/formats.md#csvwithnames) format. From 07f57ba6a0fa1622f1f7619a9535adfb32ceb74b Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:46:07 +0300 Subject: [PATCH 045/142] Update docs/ru/getting-started/example-datasets/opensky.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/opensky.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index 5e94e4f2fc9..830e43058c1 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -89,7 +89,7 @@ SELECT count() FROM opensky; └──────────┘ ``` -Проверьте, размер набора данных в ClickHouse составляет всего 2,66 ГБ. +Убедитесь, что размер набора данных в ClickHouse составляет всего 2,66 ГБ. Запрос: From 23a1f7e1685a490a26f3e1f7ec3bd479b22649b2 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:46:17 +0300 Subject: [PATCH 046/142] Update docs/ru/getting-started/example-datasets/opensky.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/opensky.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index 830e43058c1..8cfca4507b1 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -418,4 +418,4 @@ ORDER BY k ASC; ### Online Playground -Вы можете протестировать другие запросы к этому набору данным с помощью интерактивного ресурса [Online Playground](https://gh-api.clickhouse.tech/play?user=play). Например, [вот так](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICBvcmlnaW4sCiAgICBjb3VudCgpLAogICAgcm91bmQoYXZnKGdlb0Rpc3RhbmNlKGxvbmdpdHVkZV8xLCBsYXRpdHVkZV8xLCBsb25naXR1ZGVfMiwgbGF0aXR1ZGVfMikpKSBBUyBkaXN0YW5jZSwKICAgIGJhcihkaXN0YW5jZSwgMCwgMTAwMDAwMDAsIDEwMCkgQVMgYmFyCkZST00gb3BlbnNreQpXSEVSRSBvcmlnaW4gIT0gJycKR1JPVVAgQlkgb3JpZ2luCk9SREVSIEJZIGNvdW50KCkgREVTQwpMSU1JVCAxMDA=). Однако, обратите внимание, что здесь нельзя создавать временные таблицы. +Вы можете протестировать другие запросы к этому набору данным с помощью интерактивного ресурса [Online Playground](https://gh-api.clickhouse.tech/play?user=play). Например, [вот так](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICBvcmlnaW4sCiAgICBjb3VudCgpLAogICAgcm91bmQoYXZnKGdlb0Rpc3RhbmNlKGxvbmdpdHVkZV8xLCBsYXRpdHVkZV8xLCBsb25naXR1ZGVfMiwgbGF0aXR1ZGVfMikpKSBBUyBkaXN0YW5jZSwKICAgIGJhcihkaXN0YW5jZSwgMCwgMTAwMDAwMDAsIDEwMCkgQVMgYmFyCkZST00gb3BlbnNreQpXSEVSRSBvcmlnaW4gIT0gJycKR1JPVVAgQlkgb3JpZ2luCk9SREVSIEJZIGNvdW50KCkgREVTQwpMSU1JVCAxMDA=). Однако обратите внимание, что здесь нельзя создавать временные таблицы. From 8d3fa1d387d9473afd6cbccac45549f570cad61f Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:46:24 +0300 Subject: [PATCH 047/142] Update docs/ru/getting-started/example-datasets/uk-price-paid.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/uk-price-paid.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index 3d3f99445ae..80e525e9b49 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -5,7 +5,7 @@ toc_title: Набор данных о стоимости недвижимост # Набор данных о стоимости недвижимости в Великобритании -Набор данных содержит данные о стоимости недвижимости в Англии и Уэльсе. Данные доступны с 1995 года. +Набор содержит данные о стоимости недвижимости в Англии и Уэльсе. Данные доступны с 1995 года. Размер набора данных в несжатом виде составляет около 4 гигабайт, а в ClickHouse он займет около 278 Мбайт. Источник: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads From 7a2c4f2b9a1c9ae111489f14ac40c1f7dc122776 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:46:30 +0300 Subject: [PATCH 048/142] Update docs/ru/getting-started/example-datasets/uk-price-paid.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/uk-price-paid.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index 80e525e9b49..5cfd814f5e7 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -6,7 +6,7 @@ toc_title: Набор данных о стоимости недвижимост # Набор данных о стоимости недвижимости в Великобритании Набор содержит данные о стоимости недвижимости в Англии и Уэльсе. Данные доступны с 1995 года. -Размер набора данных в несжатом виде составляет около 4 гигабайт, а в ClickHouse он займет около 278 Мбайт. +Размер набора данных в несжатом виде составляет около 4 GiB, а в ClickHouse он займет около 278 MiB. Источник: https://www.gov.uk/government/statistical-data-sets/price-paid-data-downloads Описание полей таблицы: https://www.gov.uk/guidance/about-the-price-paid-data From a1665f91d92db365e4a70d31900bf3c5aec08812 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:46:38 +0300 Subject: [PATCH 049/142] Update docs/ru/getting-started/example-datasets/uk-price-paid.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/uk-price-paid.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index 5cfd814f5e7..7bd1cb04672 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -21,7 +21,7 @@ toc_title: Набор данных о стоимости недвижимост wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.com/pp-complete.csv ``` -Загрузка займет около 2 минут при хорошем подключении к Интернету. +Загрузка займет около 2 минут при хорошем подключении к интернету. ## Создайте таблицу From 07fe5b22bb00c634167140d4320ae5001baefe9d Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:46:59 +0300 Subject: [PATCH 050/142] Update docs/ru/getting-started/example-datasets/uk-price-paid.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/uk-price-paid.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index 7bd1cb04672..61987d7bac3 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -54,7 +54,7 @@ CREATE TABLE uk_price_paid Предварительная обработка включает: - разделение почтового индекса на два разных столбца `postcode1` и `postcode2`, что лучше подходит для хранения данных и выполнения запросов к ним; -- преобразование поля `time` в `Date`, поскольку оно содержит только время 00:00; +- преобразование поля `time` в дату, поскольку оно содержит только время 00:00; - поле `uuid` игнорируется, потому что оно не будет использовано нами для анализа; - преобразование полей `type` и `duration` в более читаемые поля типа `Enum` с помощью функции `transform`; - преобразование полей `is_new` и `category` из односимвольной строки (`Y`/`N` и `A`/`B`) в поле `UInt8` со значениями 0 и 1 соответственно. From a171cd2ffa6701dd6fd9ac8300b5df9fab090684 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:47:06 +0300 Subject: [PATCH 051/142] Update docs/ru/getting-started/example-datasets/uk-price-paid.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/uk-price-paid.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index 61987d7bac3..be03e32233c 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -55,7 +55,7 @@ CREATE TABLE uk_price_paid Предварительная обработка включает: - разделение почтового индекса на два разных столбца `postcode1` и `postcode2`, что лучше подходит для хранения данных и выполнения запросов к ним; - преобразование поля `time` в дату, поскольку оно содержит только время 00:00; -- поле `uuid` игнорируется, потому что оно не будет использовано нами для анализа; +- поле `uuid` игнорируется, потому что оно не будет использовано для анализа; - преобразование полей `type` и `duration` в более читаемые поля типа `Enum` с помощью функции `transform`; - преобразование полей `is_new` и `category` из односимвольной строки (`Y`/`N` и `A`/`B`) в поле `UInt8` со значениями 0 и 1 соответственно. From 40eddc5404ba3d7a11235e2514d263da966081c4 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:47:13 +0300 Subject: [PATCH 052/142] Update docs/ru/getting-started/example-datasets/uk-price-paid.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/uk-price-paid.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index be03e32233c..44c33973f93 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -118,7 +118,7 @@ SELECT count() FROM uk_price_paid; └──────────┘ ``` -Размер набора данных в ClickHouse составляет всего 278 МБ, проверьте это. +Размер набора данных в ClickHouse составляет всего 278 MiB, проверьте это. Запрос: From 9c3cc13653b5c83f26ac9b6e5c7c4f93bdcd9137 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:50:43 +0300 Subject: [PATCH 053/142] Update docs/ru/getting-started/example-datasets/opensky.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/opensky.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index 8cfca4507b1..8b51e737f87 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -57,7 +57,7 @@ ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c 'gzip -c -d "{}" | clickhou ``` - Список файлов передаётся (`ls -1 flightlist_*.csv.gz`) в `xargs` для параллельной обработки. -- `xargs -P100` указывает на использование до 100 параллельных обработчиков, но поскольку у нас всего 30 файлов, то количество обработчиков будет всего 30. +- `xargs -P100` указывает на возможность использования до 100 параллельных обработчиков, но поскольку у нас всего 30 файлов, то количество обработчиков будет всего 30. - Для каждого файла `xargs` будет запускать скрипт с `bash -c`. Сценарий имеет подстановку в виде `{}`, а команда `xargs` заменяет имя файла на указанные в подстановке символы (мы указали это для `xargs` с помощью `-I{}`). - Скрипт распакует файл (`gzip -c -d "{}"`) в стандартный вывод (параметр `-c`) и перенаправит его в `clickhouse-client`. - Чтобы распознать формат ISO-8601 со смещениями часовых поясов в полях типа `DateTime`, указывается параметр парсера `--date_time_input_format best_effort`. From e6167327436a45c79a016e17e7d7dccb800dfba5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 15 Sep 2021 22:35:48 +0300 Subject: [PATCH 054/142] Small refactoring. --- programs/client/Client.cpp | 3 +- programs/obfuscator/Obfuscator.cpp | 6 +- src/Access/DiskAccessStorage.cpp | 1 + src/DataStreams/BlockIO.cpp | 20 -- src/DataStreams/BlockIO.h | 24 +- .../PushingToViewsBlockOutputStream.cpp | 35 +- .../IInterpreterUnionOrSelectQuery.cpp | 14 + .../IInterpreterUnionOrSelectQuery.h | 1 + src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Interpreters/InterpreterCheckQuery.cpp | 4 +- src/Interpreters/InterpreterDescribeQuery.cpp | 6 +- src/Interpreters/InterpreterDescribeQuery.h | 2 +- src/Interpreters/InterpreterExistsQuery.cpp | 10 +- src/Interpreters/InterpreterExistsQuery.h | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 20 +- src/Interpreters/InterpreterExplainQuery.h | 4 +- src/Interpreters/InterpreterInsertQuery.cpp | 243 ++++++++------ src/Interpreters/InterpreterInsertQuery.h | 20 +- .../InterpreterKillQueryQuery.cpp | 41 +-- .../InterpreterSelectIntersectExceptQuery.cpp | 6 +- src/Interpreters/InterpreterSelectQuery.cpp | 4 +- .../InterpreterSelectWithUnionQuery.cpp | 6 +- .../InterpreterShowAccessQuery.cpp | 8 +- src/Interpreters/InterpreterShowAccessQuery.h | 2 +- ...InterpreterShowCreateAccessEntityQuery.cpp | 8 +- .../InterpreterShowCreateAccessEntityQuery.h | 2 +- .../InterpreterShowCreateQuery.cpp | 10 +- src/Interpreters/InterpreterShowCreateQuery.h | 2 +- .../InterpreterShowGrantsQuery.cpp | 8 +- src/Interpreters/InterpreterShowGrantsQuery.h | 2 +- src/Interpreters/InterpreterWatchQuery.cpp | 2 +- src/Interpreters/ProcessList.cpp | 11 - src/Interpreters/ProcessList.h | 8 - src/Interpreters/SystemLog.h | 2 +- src/Interpreters/executeQuery.cpp | 239 ++++--------- .../Executors/CompletedPipelineExecutor.cpp | 16 + .../Executors/CompletedPipelineExecutor.h | 19 ++ .../PullingAsyncPipelineExecutor.cpp | 25 +- .../Executors/PullingAsyncPipelineExecutor.h | 6 +- .../Executors/PullingPipelineExecutor.cpp | 22 +- .../Executors/PullingPipelineExecutor.h | 6 +- .../PushingAsyncPipelineExecutor.cpp | 23 +- .../Executors/PushingAsyncPipelineExecutor.h | 7 +- .../Executors/PushingPipelineExecutor.cpp | 22 +- .../Executors/PushingPipelineExecutor.h | 7 +- src/Processors/Formats/IOutputFormat.h | 2 + src/Processors/Formats/LazyOutputFormat.h | 2 + src/Processors/Formats/PullingOutputFormat.h | 2 + src/Processors/Pipe.h | 1 + src/Processors/QueryPipeline.cpp | 313 ++++++++++++++++++ src/Processors/QueryPipeline.h | 117 +++++++ src/Processors/QueryPipelineBuilder.cpp | 7 + src/Processors/QueryPipelineBuilder.h | 2 + src/Server/TCPHandler.cpp | 90 +---- src/Storages/Kafka/StorageKafka.cpp | 4 +- .../MaterializedPostgreSQLConsumer.cpp | 17 +- .../PostgreSQLReplicationHandler.cpp | 24 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 9 +- 58 files changed, 897 insertions(+), 624 deletions(-) create mode 100644 src/Processors/Executors/CompletedPipelineExecutor.cpp create mode 100644 src/Processors/Executors/CompletedPipelineExecutor.h create mode 100644 src/Processors/QueryPipeline.cpp create mode 100644 src/Processors/QueryPipeline.h diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 444c033318c..8b96f81cd83 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -2034,8 +2034,7 @@ private: }); } - QueryPipelineBuilder pipeline; - pipeline.init(std::move(pipe)); + QueryPipeline pipeline(std::move(pipe)); PullingAsyncPipelineExecutor executor(pipeline); Block block; diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index d7f7494a756..748c26675fd 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1162,8 +1162,7 @@ try Pipe pipe(FormatFactory::instance().getInput(input_format, file_in, header, context, max_block_size)); - QueryPipelineBuilder pipeline; - pipeline.init(std::move(pipe)); + QueryPipeline pipeline(std::move(pipe)); PullingPipelineExecutor executor(pipeline); Block block; @@ -1200,8 +1199,7 @@ try }); } - QueryPipelineBuilder pipeline; - pipeline.init(std::move(pipe)); + QueryPipeline pipeline(std::move(pipe)); BlockOutputStreamPtr output = context->getOutputStreamParallelIfPossible(output_format, file_out, header); diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 5fb6f48e1ee..6c995f52f2b 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include diff --git a/src/DataStreams/BlockIO.cpp b/src/DataStreams/BlockIO.cpp index 496bcf84877..0c3ad92ff18 100644 --- a/src/DataStreams/BlockIO.cpp +++ b/src/DataStreams/BlockIO.cpp @@ -10,22 +10,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -BlockInputStreamPtr BlockIO::getInputStream() -{ - if (!out.empty()) - throw Exception("Cannot get input stream from BlockIO because output stream is not empty", - ErrorCodes::LOGICAL_ERROR); - - if (in) - return in; - - if (pipeline.initialized()) - return std::make_shared(std::move(pipeline)); - - throw Exception("Cannot get input stream from BlockIO because query pipeline was not initialized", - ErrorCodes::LOGICAL_ERROR); -} - void BlockIO::reset() { /** process_list_entry should be destroyed after in, after out and after pipeline, @@ -38,8 +22,6 @@ void BlockIO::reset() */ /// TODO simplify it all - out.reset(); - in.reset(); if (process_list_entry) process_list_entry->get().releaseQueryStreams(); pipeline.reset(); @@ -57,8 +39,6 @@ BlockIO & BlockIO::operator= (BlockIO && rhs) reset(); process_list_entry = std::move(rhs.process_list_entry); - in = std::move(rhs.in); - out = std::move(rhs.out); pipeline = std::move(rhs.pipeline); finish_callback = std::move(rhs.finish_callback); diff --git a/src/DataStreams/BlockIO.h b/src/DataStreams/BlockIO.h index 41dd65e8782..d699d525f2f 100644 --- a/src/DataStreams/BlockIO.h +++ b/src/DataStreams/BlockIO.h @@ -1,11 +1,7 @@ #pragma once -#include - #include - -#include -#include +#include namespace DB @@ -26,14 +22,11 @@ struct BlockIO std::shared_ptr process_list_entry; - Chain out; - BlockInputStreamPtr in; - - QueryPipelineBuilder pipeline; + QueryPipeline pipeline; /// Callbacks for query logging could be set here. - std::function finish_callback; - std::function exception_callback; + std::function finish_callback; + std::function exception_callback; /// When it is true, don't bother sending any non-empty blocks to the out stream bool null_format = false; @@ -43,11 +36,7 @@ struct BlockIO { if (finish_callback) { - QueryPipelineBuilder * pipeline_ptr = nullptr; - if (pipeline.initialized()) - pipeline_ptr = &pipeline; - - finish_callback(in.get(), pipeline_ptr); + finish_callback(pipeline); } } @@ -57,9 +46,6 @@ struct BlockIO exception_callback(); } - /// Returns in or converts pipeline to stream. Throws if out is not empty. - BlockInputStreamPtr getInputStream(); - private: void reset(); }; diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index c370f547f7f..8a720916776 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -387,30 +387,22 @@ Chain buildPushingToViewsDrain( query = dependent_metadata_snapshot->getSelectQuery().inner_query; target_name = inner_table_id.getFullTableName(); - std::unique_ptr insert = std::make_unique(); - insert->table_id = inner_table_id; - /// Get list of columns we get from select query. auto header = InterpreterSelectQuery(query, select_context, SelectQueryOptions().analyze()) .getSampleBlock(); /// Insert only columns returned by select. - auto list = std::make_shared(); + Names insert_columns; const auto & inner_table_columns = inner_metadata_snapshot->getColumns(); for (const auto & column : header) { /// But skip columns which storage doesn't have. if (inner_table_columns.hasPhysical(column.name)) - list->children.emplace_back(std::make_shared(column.name)); + insert_columns.emplace_back(column.name); } - insert->columns = std::move(list); - - ASTPtr insert_query_ptr(insert.release()); - InterpreterInsertQuery interpreter(insert_query_ptr, insert_context, false, false, false, view_runtime_data); - BlockIO io = interpreter.execute(); - io.out.attachResources(QueryPipelineBuilder::getPipe(std::move(io.pipeline)).detachResources()); - out = std::move(io.out); + InterpreterInsertQuery interpreter(nullptr, insert_context, false, false, false); + out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, view_runtime_data); } else if (auto * live_view = dynamic_cast(dependent_table.get())) { @@ -547,34 +539,35 @@ static void process(Block & block, ViewRuntimeData & view, const StorageID & sou /// - These objects live inside query pipeline (DataStreams) and the reference become dangling. InterpreterSelectQuery select(view.query, local_context, SelectQueryOptions()); - auto io = select.execute(); - io.pipeline.resize(1); + auto pipeline = select.buildQueryPipeline(); + pipeline.resize(1); /// Squashing is needed here because the materialized 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). - io.pipeline.addTransform(std::make_shared( - io.pipeline.getHeader(), + pipeline.addTransform(std::make_shared( + pipeline.getHeader(), context->getSettingsRef().min_insert_block_size_rows, context->getSettingsRef().min_insert_block_size_bytes)); auto converting = ActionsDAG::makeConvertingActions( - io.pipeline.getHeader().getColumnsWithTypeAndName(), + pipeline.getHeader().getColumnsWithTypeAndName(), view.sample_block.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); - io.pipeline.addTransform(std::make_shared( - io.pipeline.getHeader(), + pipeline.addTransform(std::make_shared( + pipeline.getHeader(), std::make_shared(std::move(converting)))); - io.pipeline.setProgressCallback([context](const Progress & progress) + pipeline.setProgressCallback([context](const Progress & progress) { CurrentThread::updateProgressIn(progress); if (auto callback = context->getProgressCallback()) callback(progress); }); - PullingPipelineExecutor executor(io.pipeline); + auto query_pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline)); + PullingPipelineExecutor executor(query_pipeline); if (!executor.pull(block)) { block.clear(); diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index 7233ab332dd..62a7f0bdf7e 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -1,5 +1,8 @@ #include #include +#include +#include +#include namespace DB { @@ -9,4 +12,15 @@ void IInterpreterUnionOrSelectQuery::extendQueryLogElemImpl(QueryLogElement & el elem.query_kind = "Select"; } + +QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline() +{ + QueryPlan query_plan; + + buildQueryPlan(query_plan); + + return std::move(*query_plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)))); +} + } diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index cc960e748f6..db9cc086e35 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -28,6 +28,7 @@ public: } virtual void buildQueryPlan(QueryPlan & query_plan) = 0; + QueryPipelineBuilder buildQueryPipeline(); virtual void ignoreWithTotals() = 0; diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index c10f4d33f6d..64214fe61d8 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -121,7 +121,7 @@ BlockIO InterpreterAlterQuery::execute() table->checkAlterPartitionIsPossible(partition_commands, metadata_snapshot, getContext()->getSettingsRef()); auto partition_commands_pipe = table->alterPartition(metadata_snapshot, partition_commands, getContext()); if (!partition_commands_pipe.empty()) - res.pipeline.init(std::move(partition_commands_pipe)); + res.pipeline = QueryPipeline(std::move(partition_commands_pipe)); } if (!live_view_commands.empty()) diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index e8a4f884dd0..9fd318ee4cf 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include @@ -72,7 +72,7 @@ BlockIO InterpreterCheckQuery::execute() } BlockIO res; - res.in = std::make_shared(block); + res.pipeline = QueryPipeline(std::make_shared(std::move(block))); return res; } diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 705e52da72c..beaa870f1a2 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -60,7 +60,7 @@ Block InterpreterDescribeQuery::getSampleBlock() } -BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() +QueryPipeline InterpreterDescribeQuery::executeImpl() { ColumnsDescription columns; @@ -119,7 +119,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() res_columns[6]->insertDefault(); } - return std::make_shared(sample_block.cloneWithColumns(std::move(res_columns))); + return QueryPipeline(std::make_shared(sample_block.cloneWithColumns(std::move(res_columns)))); } } diff --git a/src/Interpreters/InterpreterDescribeQuery.h b/src/Interpreters/InterpreterDescribeQuery.h index 627d1ca0353..f5074dbdc93 100644 --- a/src/Interpreters/InterpreterDescribeQuery.h +++ b/src/Interpreters/InterpreterDescribeQuery.h @@ -21,7 +21,7 @@ public: private: ASTPtr query_ptr; - BlockInputStreamPtr executeImpl(); + QueryPipeline executeImpl(); }; diff --git a/src/Interpreters/InterpreterExistsQuery.cpp b/src/Interpreters/InterpreterExistsQuery.cpp index 9e254db4bb3..6eb188bce9f 100644 --- a/src/Interpreters/InterpreterExistsQuery.cpp +++ b/src/Interpreters/InterpreterExistsQuery.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include @@ -21,7 +21,7 @@ namespace ErrorCodes BlockIO InterpreterExistsQuery::execute() { BlockIO res; - res.in = executeImpl(); + res.pipeline = executeImpl(); return res; } @@ -35,7 +35,7 @@ Block InterpreterExistsQuery::getSampleBlock() } -BlockInputStreamPtr InterpreterExistsQuery::executeImpl() +QueryPipeline InterpreterExistsQuery::executeImpl() { ASTQueryWithTableAndOutput * exists_query; bool result = false; @@ -76,10 +76,10 @@ BlockInputStreamPtr InterpreterExistsQuery::executeImpl() result = DatabaseCatalog::instance().isDictionaryExist({database, exists_query->table}); } - return std::make_shared(Block{{ + return QueryPipeline(std::make_shared(Block{{ ColumnUInt8::create(1, result), std::make_shared(), - "result" }}); + "result" }})); } } diff --git a/src/Interpreters/InterpreterExistsQuery.h b/src/Interpreters/InterpreterExistsQuery.h index efc664f07c3..5ef940bdd3f 100644 --- a/src/Interpreters/InterpreterExistsQuery.h +++ b/src/Interpreters/InterpreterExistsQuery.h @@ -21,7 +21,7 @@ public: private: ASTPtr query_ptr; - BlockInputStreamPtr executeImpl(); + QueryPipeline executeImpl(); }; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index da5c96776fd..c29eace1b55 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include #include #include #include @@ -73,7 +73,7 @@ namespace BlockIO InterpreterExplainQuery::execute() { BlockIO res; - res.in = executeImpl(); + res.pipeline = executeImpl(); return res; } @@ -240,7 +240,7 @@ ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) } -BlockInputStreamPtr InterpreterExplainQuery::executeImpl() +QueryPipeline InterpreterExplainQuery::executeImpl() { const auto & ast = query->as(); @@ -335,17 +335,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() { InterpreterInsertQuery insert(ast.getExplainedQuery(), getContext()); auto io = insert.execute(); - if (io.pipeline.initialized()) - { - auto pipe = QueryPipelineBuilder::getPipe(std::move(io.pipeline)); - const auto & processors = pipe.getProcessors(); - printPipeline(processors, buf); - } - else - { - const auto & processors = io.out.getProcessors(); - printPipeline(processors, buf); - } + printPipeline(io.pipeline.getProcessors(), buf); } else throw Exception("Only SELECT and INSERT is supported for EXPLAIN PIPELINE query", ErrorCodes::INCORRECT_QUERY); @@ -377,7 +367,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() fillColumn(*res_columns[0], buf.str()); } - return std::make_shared(sample_block.cloneWithColumns(std::move(res_columns))); + return QueryPipeline(std::make_shared(sample_block.cloneWithColumns(std::move(res_columns)))); } } diff --git a/src/Interpreters/InterpreterExplainQuery.h b/src/Interpreters/InterpreterExplainQuery.h index a7f54a10e3e..a640b1c977c 100644 --- a/src/Interpreters/InterpreterExplainQuery.h +++ b/src/Interpreters/InterpreterExplainQuery.h @@ -15,12 +15,12 @@ public: BlockIO execute() override; - static Block getSampleBlock(const ASTExplainQuery::ExplainKind kind); + static Block getSampleBlock(ASTExplainQuery::ExplainKind kind); private: ASTPtr query; - BlockInputStreamPtr executeImpl(); + QueryPipeline executeImpl(); }; diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 65658a0b295..003d29dcc42 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -44,14 +44,12 @@ namespace ErrorCodes } InterpreterInsertQuery::InterpreterInsertQuery( - const ASTPtr & query_ptr_, ContextPtr context_, bool allow_materialized_, bool no_squash_, bool no_destination_, - ExceptionKeepingTransformRuntimeDataPtr runtime_data_) + const ASTPtr & query_ptr_, ContextPtr context_, bool allow_materialized_, bool no_squash_, bool no_destination_) : WithContext(context_) , query_ptr(query_ptr_) , allow_materialized(allow_materialized_) , no_squash(no_squash_) , no_destination(no_destination_) - , runtime_data(std::move(runtime_data_)) { checkStackSize(); } @@ -75,26 +73,37 @@ Block InterpreterInsertQuery::getSampleBlock( const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const { - Block table_sample_non_materialized = metadata_snapshot->getSampleBlockNonMaterialized(); /// If the query does not include information about columns if (!query.columns) { if (no_destination) return metadata_snapshot->getSampleBlockWithVirtuals(table->getVirtuals()); else - return table_sample_non_materialized; + return metadata_snapshot->getSampleBlockNonMaterialized(); } - Block table_sample = metadata_snapshot->getSampleBlock(); - - const auto columns_ast = processColumnTransformers(getContext()->getCurrentDatabase(), table, metadata_snapshot, query.columns); - /// Form the block based on the column names from the query - Block res; + Names names; + const auto columns_ast = processColumnTransformers(getContext()->getCurrentDatabase(), table, metadata_snapshot, query.columns); for (const auto & identifier : columns_ast->children) { std::string current_name = identifier->getColumnName(); + names.emplace_back(std::move(current_name)); + } + return getSampleBlock(names, table, metadata_snapshot); +} + +Block InterpreterInsertQuery::getSampleBlock( + const Names & names, + const StoragePtr & table, + const StorageMetadataPtr & metadata_snapshot) const +{ + Block table_sample = metadata_snapshot->getSampleBlock(); + Block table_sample_non_materialized = metadata_snapshot->getSampleBlockNonMaterialized(); + Block res; + for (const auto & current_name : names) + { /// The table does not have a column with that name if (!table_sample.has(current_name)) throw Exception("No such column " + current_name + " in table " + table->getStorageID().getNameForLogs(), @@ -149,13 +158,93 @@ static bool isTrivialSelect(const ASTPtr & select) return false; }; +Chain InterpreterInsertQuery::buildChain( + const StoragePtr & table, + const StorageMetadataPtr & metadata_snapshot, + const Names & columns, + ExceptionKeepingTransformRuntimeDataPtr runtime_data) +{ + return buildChainImpl(table, metadata_snapshot, getSampleBlock(columns, table, metadata_snapshot), std::move(runtime_data)); +} + +Chain InterpreterInsertQuery::buildChainImpl( + const StoragePtr & table, + const StorageMetadataPtr & metadata_snapshot, + const Block & query_sample_block, + ExceptionKeepingTransformRuntimeDataPtr runtime_data) +{ + auto context = getContext(); + const ASTInsertQuery * query = nullptr; + if (query_ptr) + query = query_ptr->as(); + + const Settings & settings = context->getSettingsRef(); + bool null_as_default = query && query->select && context->getSettingsRef().insert_null_as_default; + + /// We create a pipeline of several streams, into which we will write data. + Chain out; + + /// NOTE: we explicitly ignore bound materialized views when inserting into Kafka Storage. + /// Otherwise we'll get duplicates when MV reads same rows again from Kafka. + if (table->noPushingToViews() && !no_destination) + { + auto sink = table->write(query_ptr, metadata_snapshot, context); + sink->setRuntimeData(runtime_data); + out.addSource(std::move(sink)); + } + else + { + out = buildPushingToViewsDrain(table, metadata_snapshot, context, query_ptr, no_destination, runtime_data); + } + + /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. + + /// Checking constraints. It must be done after calculation of all defaults, so we can check them on calculated columns. + if (const auto & constraints = metadata_snapshot->getConstraints(); !constraints.empty()) + out.addSource(std::make_shared( + table->getStorageID(), out.getInputHeader(), metadata_snapshot->getConstraints(), context)); + + auto adding_missing_defaults_dag = addMissingDefaults( + query_sample_block, + out.getInputHeader().getNamesAndTypesList(), + metadata_snapshot->getColumns(), + context, + null_as_default); + + auto adding_missing_defaults_actions = std::make_shared(adding_missing_defaults_dag); + + /// Actually we don't know structure of input blocks from query/table, + /// because some clients break insertion protocol (columns != header) + out.addSource(std::make_shared(query_sample_block, adding_missing_defaults_actions)); + + /// It's important to squash blocks as early as possible (before other transforms), + /// because other transforms may work inefficient if block size is small. + + /// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side. + /// Client-side bufferization might cause excessive timeouts (especially in case of big blocks). + if (!(settings.insert_distributed_sync && table->isRemote()) && !no_squash && !(query && query->watch)) + { + bool table_prefers_large_blocks = table->prefersLargeBlocks(); + + out.addSource(std::make_shared( + out.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0)); + } + + auto counting = std::make_shared(out.getInputHeader(), runtime_data ? runtime_data->thread_status : nullptr); + counting->setProcessListElement(context->getProcessListElement()); + out.addSource(std::move(counting)); + + return out; +} BlockIO InterpreterInsertQuery::execute() { const Settings & settings = getContext()->getSettingsRef(); auto & query = query_ptr->as(); - BlockIO res; + QueryPipelineBuilder pipeline; StoragePtr table = getTable(query); if (query.partition_by && !table->supportsPartitionBy()) @@ -175,7 +264,7 @@ BlockIO InterpreterInsertQuery::execute() // Distributed INSERT SELECT if (auto maybe_pipeline = table->distributedWrite(query, getContext())) { - res.pipeline = std::move(*maybe_pipeline); + pipeline = std::move(*maybe_pipeline); is_distributed_insert_select = true; } } @@ -184,6 +273,7 @@ BlockIO InterpreterInsertQuery::execute() if (!is_distributed_insert_select || query.watch) { size_t out_streams_size = 1; + if (query.select) { bool is_trivial_insert_select = false; @@ -227,27 +317,27 @@ BlockIO InterpreterInsertQuery::execute() InterpreterSelectWithUnionQuery interpreter_select{ query.select, new_context, SelectQueryOptions(QueryProcessingStage::Complete, 1)}; - res = interpreter_select.execute(); + pipeline = interpreter_select.buildQueryPipeline(); } else { /// Passing 1 as subquery_depth will disable limiting size of intermediate result. InterpreterSelectWithUnionQuery interpreter_select{ query.select, getContext(), SelectQueryOptions(QueryProcessingStage::Complete, 1)}; - res = interpreter_select.execute(); + pipeline = interpreter_select.buildQueryPipeline(); } - res.pipeline.dropTotalsAndExtremes(); + pipeline.dropTotalsAndExtremes(); if (table->supportsParallelInsert() && settings.max_insert_threads > 1) - out_streams_size = std::min(size_t(settings.max_insert_threads), res.pipeline.getNumStreams()); + out_streams_size = std::min(size_t(settings.max_insert_threads), pipeline.getNumStreams()); - res.pipeline.resize(out_streams_size); + pipeline.resize(out_streams_size); /// Allow to insert Nullable into non-Nullable columns, NULL values will be added as defaults values. if (getContext()->getSettingsRef().insert_null_as_default) { - const auto & input_columns = res.pipeline.getHeader().getColumnsWithTypeAndName(); + const auto & input_columns = pipeline.getHeader().getColumnsWithTypeAndName(); const auto & query_columns = query_sample_block.getColumnsWithTypeAndName(); const auto & output_columns = metadata_snapshot->getColumns(); @@ -266,103 +356,56 @@ BlockIO InterpreterInsertQuery::execute() else if (query.watch) { InterpreterWatchQuery interpreter_watch{ query.watch, getContext() }; - res = interpreter_watch.execute(); + pipeline = interpreter_watch.execute(); } for (size_t i = 0; i < out_streams_size; i++) { - /// We create a pipeline of several streams, into which we will write data. - Chain out; - - /// NOTE: we explicitly ignore bound materialized views when inserting into Kafka Storage. - /// Otherwise we'll get duplicates when MV reads same rows again from Kafka. - if (table->noPushingToViews() && !no_destination) - { - auto sink = table->write(query_ptr, metadata_snapshot, getContext()); - sink->setRuntimeData(runtime_data); - out.addSource(std::move(sink)); - } - else - { - out = buildPushingToViewsDrain(table, metadata_snapshot, getContext(), query_ptr, no_destination, runtime_data); - } - - /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. - - /// Checking constraints. It must be done after calculation of all defaults, so we can check them on calculated columns. - if (const auto & constraints = metadata_snapshot->getConstraints(); !constraints.empty()) - out.addSource(std::make_shared( - query.table_id, out.getInputHeader(), metadata_snapshot->getConstraints(), getContext())); - - bool null_as_default = query.select && getContext()->getSettingsRef().insert_null_as_default; - - auto adding_missing_defaults_dag = addMissingDefaults( - query_sample_block, - out.getInputHeader().getNamesAndTypesList(), - metadata_snapshot->getColumns(), - getContext(), - null_as_default); - - auto adding_missing_defaults_actions = std::make_shared(adding_missing_defaults_dag); - - /// Actually we don't know structure of input blocks from query/table, - /// because some clients break insertion protocol (columns != header) - out.addSource(std::make_shared(query_sample_block, adding_missing_defaults_actions)); - - /// It's important to squash blocks as early as possible (before other transforms), - /// because other transforms may work inefficient if block size is small. - - /// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side. - /// Client-side bufferization might cause excessive timeouts (especially in case of big blocks). - if (!(settings.insert_distributed_sync && table->isRemote()) && !no_squash && !query.watch) - { - bool table_prefers_large_blocks = table->prefersLargeBlocks(); - - out.addSource(std::make_shared( - out.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0)); - } - - auto counting = std::make_shared(out.getInputHeader(), runtime_data ? runtime_data->thread_status : nullptr); - counting->setProcessListElement(getContext()->getProcessListElement()); - out.addSource(std::move(counting)); - + auto out = buildChainImpl(table, metadata_snapshot, query_sample_block); out_chains.emplace_back(std::move(out)); } } + pipeline.addStorageHolder(table); + if (const auto * mv = dynamic_cast(table.get())) + { + if (auto inner_table = mv->tryGetTargetTable()) + pipeline.addStorageHolder(inner_table); + } + + BlockIO res; + /// What type of query: INSERT or INSERT SELECT or INSERT WATCH? if (is_distributed_insert_select) { - /// Pipeline was already built. + res.pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline)); } else if (query.select || query.watch) { const auto & header = out_chains.at(0).getInputHeader(); auto actions_dag = ActionsDAG::makeConvertingActions( - res.pipeline.getHeader().getColumnsWithTypeAndName(), + pipeline.getHeader().getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Position); auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes)); - res.pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { return std::make_shared(in_header, actions); }); - auto num_select_threads = res.pipeline.getNumThreads(); + auto num_select_threads = pipeline.getNumThreads(); - res.pipeline.addChains(std::move(out_chains)); + pipeline.addChains(std::move(out_chains)); - res.pipeline.setSinks([&](const Block & cur_header, QueryPipelineBuilder::StreamType) -> ProcessorPtr + pipeline.setSinks([&](const Block & cur_header, QueryPipelineBuilder::StreamType) -> ProcessorPtr { return std::make_shared(cur_header); }); /// Don't use more threads for insert then for select to reduce memory consumption. - if (!settings.parallel_view_processing && res.pipeline.getNumThreads() > num_select_threads) - res.pipeline.setMaxThreads(num_select_threads); + if (!settings.parallel_view_processing && pipeline.getNumThreads() > num_select_threads) + pipeline.setMaxThreads(num_select_threads); if (!allow_materialized) { @@ -370,34 +413,22 @@ BlockIO InterpreterInsertQuery::execute() if (column.default_desc.kind == ColumnDefaultKind::Materialized && header.has(column.name)) throw Exception("Cannot insert column " + column.name + ", because it is MATERIALIZED column.", ErrorCodes::ILLEGAL_COLUMN); } - } - else if (query.data && !query.has_tail) /// can execute without additional data - { - auto pipe = getSourceFromFromASTInsertQuery(query_ptr, nullptr, query_sample_block, getContext(), nullptr); - res.pipeline.init(std::move(pipe)); - res.pipeline.resize(1); - res.pipeline.addChains(std::move(out_chains)); - res.pipeline.setSinks([&](const Block & cur_header, Pipe::StreamType) - { - return std::make_shared(cur_header); - }); + + res.pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline)); } else { - res.out = std::move(out_chains.at(0)); - res.out.setNumThreads(std::min(res.out.getNumThreads(), settings.max_threads)); - } + res.pipeline = QueryPipeline(std::move(out_chains.at(0))); + res.pipeline.setNumThreads(std::min(res.pipeline.getNumThreads(), settings.max_threads)); - res.pipeline.addStorageHolder(table); - if (const auto * mv = dynamic_cast(table.get())) - { - if (auto inner_table = mv->tryGetTargetTable()) - res.pipeline.addStorageHolder(inner_table); + if (query.data && !query.has_tail) + { + /// can execute without additional data + auto pipe = getSourceFromFromASTInsertQuery(query_ptr, nullptr, query_sample_block, getContext(), nullptr); + res.pipeline.complete(std::move(pipe)); + } } - if (!res.out.empty()) - res.out.attachResources(QueryPipelineBuilder::getPipe(std::move(res.pipeline)).detachResources()); - return res; } diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 380d7550ed8..0454e92c90a 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -12,6 +12,8 @@ namespace DB struct ExceptionKeepingTransformRuntimeData; using ExceptionKeepingTransformRuntimeDataPtr = std::shared_ptr; +class Chain; + /** Interprets the INSERT query. */ class InterpreterInsertQuery : public IInterpreter, WithContext @@ -22,8 +24,7 @@ public: ContextPtr context_, bool allow_materialized_ = false, bool no_squash_ = false, - bool no_destination_ = false, - ExceptionKeepingTransformRuntimeDataPtr runtime_data = nullptr); + bool no_destination_ = false); /** Prepare a request for execution. Return block streams * - the stream into which you can write data to execute the query, if INSERT; @@ -31,20 +32,33 @@ public: * Or nothing if the request INSERT SELECT (self-sufficient query - does not accept the input data, does not return the result). */ BlockIO execute() override; + Chain buildChain(); StorageID getDatabaseTable() const; + Chain buildChain( + const StoragePtr & table, + const StorageMetadataPtr & metadata_snapshot, + const Names & columns, + ExceptionKeepingTransformRuntimeDataPtr runtime_data = nullptr); + void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const override; private: StoragePtr getTable(ASTInsertQuery & query); Block getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const; + Block getSampleBlock(const Names & names, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const; ASTPtr query_ptr; const bool allow_materialized; const bool no_squash; const bool no_destination; - ExceptionKeepingTransformRuntimeDataPtr runtime_data; + + Chain buildChainImpl( + const StoragePtr & table, + const StorageMetadataPtr & metadata_snapshot, + const Block & query_sample_block, + ExceptionKeepingTransformRuntimeDataPtr runtime_data = nullptr); }; diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index d43d697fcd5..7aad817488b 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -16,7 +16,9 @@ #include #include #include -#include +#include +#include +#include #include #include #include @@ -121,15 +123,15 @@ static QueryDescriptors extractQueriesExceptMeAndCheckAccess(const Block & proce } -class SyncKillQueryInputStream : public IBlockInputStream +class SyncKillQuerySource : public SourceWithProgress { public: - SyncKillQueryInputStream(ProcessList & process_list_, QueryDescriptors && processes_to_stop_, Block && processes_block_, + SyncKillQuerySource(ProcessList & process_list_, QueryDescriptors && processes_to_stop_, Block && processes_block_, const Block & res_sample_block_) - : process_list(process_list_), - processes_to_stop(std::move(processes_to_stop_)), - processes_block(std::move(processes_block_)), - res_sample_block(res_sample_block_) + : SourceWithProgress(res_sample_block_) + , process_list(process_list_) + , processes_to_stop(std::move(processes_to_stop_)) + , processes_block(std::move(processes_block_)) { addTotalRowsApprox(processes_to_stop.size()); } @@ -139,14 +141,12 @@ public: return "SynchronousQueryKiller"; } - Block getHeader() const override { return res_sample_block; } - - Block readImpl() override + Chunk generate() override { size_t num_result_queries = processes_to_stop.size(); if (num_processed_queries >= num_result_queries) - return Block(); + return {}; MutableColumns columns = res_sample_block.cloneEmptyColumns(); @@ -179,7 +179,8 @@ public: /// Don't produce empty block } while (columns.empty() || columns[0]->empty()); - return res_sample_block.cloneWithColumns(std::move(columns)); + size_t num_rows = columns.empty() ? columns.front()->size() : 0; + return Chunk(std::move(columns), num_rows); } ProcessList & process_list; @@ -221,12 +222,12 @@ BlockIO InterpreterKillQueryQuery::execute() insertResultRow(query_desc.source_num, code, processes_block, header, res_columns); } - res_io.in = std::make_shared(header.cloneWithColumns(std::move(res_columns))); + res_io.pipeline = QueryPipeline(std::make_shared(header.cloneWithColumns(std::move(res_columns)))); } else { - res_io.in = std::make_shared( - process_list, std::move(queries_to_stop), std::move(processes_block), header); + res_io.pipeline = QueryPipeline(std::make_shared( + process_list, std::move(queries_to_stop), std::move(processes_block), header)); } break; @@ -286,7 +287,7 @@ BlockIO InterpreterKillQueryQuery::execute() "Not allowed to kill mutation. To execute this query it's necessary to have the grant " + required_access_rights.toString(), ErrorCodes::ACCESS_DENIED); - res_io.in = std::make_shared(header.cloneWithColumns(std::move(res_columns))); + res_io.pipeline = QueryPipeline(Pipe(std::make_shared(header.cloneWithColumns(std::move(res_columns))))); break; } @@ -302,10 +303,12 @@ Block InterpreterKillQueryQuery::getSelectResult(const String & columns, const S if (where_expression) select_query += " WHERE " + queryToString(where_expression); - auto stream = executeQuery(select_query, getContext(), true).getInputStream(); - Block res = stream->read(); + auto io = executeQuery(select_query, getContext(), true); + PullingPipelineExecutor executor(io.pipeline); + Block res; + bool need_another_read = executor.pull(res); - if (res && stream->read()) + if (res && need_another_read) throw Exception("Expected one block from input stream", ErrorCodes::LOGICAL_ERROR); return res; diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 9c8dda56b44..f7d98e2e0fe 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -133,8 +133,10 @@ BlockIO InterpreterSelectIntersectExceptQuery::execute() QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - res.pipeline = std::move(*pipeline); - res.pipeline.addInterpreterContext(context); + pipeline->addInterpreterContext(context); + + res.pipeline = QueryPipelineBuilder::getPipeline(std::move(*query_plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)))); return res; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5fe9948f857..ba2750a6690 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -603,8 +603,8 @@ BlockIO InterpreterSelectQuery::execute() buildQueryPlan(query_plan); - res.pipeline = std::move(*query_plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context))); + res.pipeline = QueryPipelineBuilder::getPipeline(std::move(*query_plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)))); return res; } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index b7494a6c965..4aeaa9e4f13 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -320,13 +320,13 @@ BlockIO InterpreterSelectWithUnionQuery::execute() QueryPlan query_plan; buildQueryPlan(query_plan); - auto pipeline = query_plan.buildQueryPipeline( + auto pipeline_builder = query_plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - res.pipeline = std::move(*pipeline); - res.pipeline.addInterpreterContext(context); + pipeline_builder->addInterpreterContext(context); + res.pipeline = QueryPipelineBuilder::getPipeline(std::move(*pipeline_builder)); return res; } diff --git a/src/Interpreters/InterpreterShowAccessQuery.cpp b/src/Interpreters/InterpreterShowAccessQuery.cpp index a82ba0411b4..137da9b7c5f 100644 --- a/src/Interpreters/InterpreterShowAccessQuery.cpp +++ b/src/Interpreters/InterpreterShowAccessQuery.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include @@ -22,12 +22,12 @@ using EntityType = IAccessEntity::Type; BlockIO InterpreterShowAccessQuery::execute() { BlockIO res; - res.in = executeImpl(); + res.pipeline = executeImpl(); return res; } -BlockInputStreamPtr InterpreterShowAccessQuery::executeImpl() const +QueryPipeline InterpreterShowAccessQuery::executeImpl() const { /// Build a create query. ASTs queries = getCreateAndGrantQueries(); @@ -43,7 +43,7 @@ BlockInputStreamPtr InterpreterShowAccessQuery::executeImpl() const } String desc = "ACCESS"; - return std::make_shared(Block{{std::move(column), std::make_shared(), desc}}); + return QueryPipeline(std::make_shared(Block{{std::move(column), std::make_shared(), desc}})); } diff --git a/src/Interpreters/InterpreterShowAccessQuery.h b/src/Interpreters/InterpreterShowAccessQuery.h index d08d8962abc..d305af9487f 100644 --- a/src/Interpreters/InterpreterShowAccessQuery.h +++ b/src/Interpreters/InterpreterShowAccessQuery.h @@ -23,7 +23,7 @@ public: bool ignoreLimits() const override { return true; } private: - BlockInputStreamPtr executeImpl() const; + QueryPipeline executeImpl() const; ASTs getCreateAndGrantQueries() const; std::vector getEntities() const; diff --git a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp index 8115b4a63df..87ebd064ae5 100644 --- a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include #include #include #include @@ -241,12 +241,12 @@ InterpreterShowCreateAccessEntityQuery::InterpreterShowCreateAccessEntityQuery(c BlockIO InterpreterShowCreateAccessEntityQuery::execute() { BlockIO res; - res.in = executeImpl(); + res.pipeline = executeImpl(); return res; } -BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl() +QueryPipeline InterpreterShowCreateAccessEntityQuery::executeImpl() { /// Build a create queries. ASTs create_queries = getCreateQueries(); @@ -270,7 +270,7 @@ BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl() if (startsWith(desc, prefix)) desc = desc.substr(prefix.length()); /// `desc` always starts with "SHOW ", so we can trim this prefix. - return std::make_shared(Block{{std::move(column), std::make_shared(), desc}}); + return QueryPipeline(std::make_shared(Block{{std::move(column), std::make_shared(), desc}})); } diff --git a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h b/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h index 6d026d2b81b..0aedeb18be4 100644 --- a/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h +++ b/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h @@ -30,7 +30,7 @@ public: static ASTPtr getAttachQuery(const IAccessEntity & entity); private: - BlockInputStreamPtr executeImpl(); + QueryPipeline executeImpl(); std::vector getEntities() const; ASTs getCreateQueries() const; AccessRightsElements getRequiredAccess() const; diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 967d3e7f570..adf1aae3ff3 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -26,7 +26,7 @@ namespace ErrorCodes BlockIO InterpreterShowCreateQuery::execute() { BlockIO res; - res.in = executeImpl(); + res.pipeline = executeImpl(); return res; } @@ -40,7 +40,7 @@ Block InterpreterShowCreateQuery::getSampleBlock() } -BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() +QueryPipeline InterpreterShowCreateQuery::executeImpl() { ASTPtr create_query; ASTQueryWithTableAndOutput * show_query; @@ -100,10 +100,10 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() MutableColumnPtr column = ColumnString::create(); column->insert(res); - return std::make_shared(Block{{ + return QueryPipeline(std::make_shared(Block{{ std::move(column), std::make_shared(), - "statement"}}); + "statement"}})); } } diff --git a/src/Interpreters/InterpreterShowCreateQuery.h b/src/Interpreters/InterpreterShowCreateQuery.h index 53f587d3e7d..da622de7fc6 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.h +++ b/src/Interpreters/InterpreterShowCreateQuery.h @@ -21,7 +21,7 @@ public: private: ASTPtr query_ptr; - BlockInputStreamPtr executeImpl(); + QueryPipeline executeImpl(); }; diff --git a/src/Interpreters/InterpreterShowGrantsQuery.cpp b/src/Interpreters/InterpreterShowGrantsQuery.cpp index f3e554122e1..7302e893cdd 100644 --- a/src/Interpreters/InterpreterShowGrantsQuery.cpp +++ b/src/Interpreters/InterpreterShowGrantsQuery.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include @@ -100,12 +100,12 @@ namespace BlockIO InterpreterShowGrantsQuery::execute() { BlockIO res; - res.in = executeImpl(); + res.pipeline = executeImpl(); return res; } -BlockInputStreamPtr InterpreterShowGrantsQuery::executeImpl() +QueryPipeline InterpreterShowGrantsQuery::executeImpl() { /// Build a create query. ASTs grant_queries = getGrantQueries(); @@ -129,7 +129,7 @@ BlockInputStreamPtr InterpreterShowGrantsQuery::executeImpl() if (desc.starts_with(prefix)) desc = desc.substr(prefix.length()); /// `desc` always starts with "SHOW ", so we can trim this prefix. - return std::make_shared(Block{{std::move(column), std::make_shared(), desc}}); + return QueryPipeline(std::make_shared(Block{{std::move(column), std::make_shared(), desc}})); } diff --git a/src/Interpreters/InterpreterShowGrantsQuery.h b/src/Interpreters/InterpreterShowGrantsQuery.h index c23aa1e3b94..06bdcf169b1 100644 --- a/src/Interpreters/InterpreterShowGrantsQuery.h +++ b/src/Interpreters/InterpreterShowGrantsQuery.h @@ -27,7 +27,7 @@ public: bool ignoreLimits() const override { return true; } private: - BlockInputStreamPtr executeImpl(); + QueryPipeline executeImpl(); ASTs getGrantQueries() const; std::vector getEntities() const; diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index ee96045bbc4..c99e25eadb8 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -85,7 +85,7 @@ BlockIO InterpreterWatchQuery::execute() pipe.setQuota(getContext()->getQuota()); } - res.pipeline.init(std::move(pipe)); + res.pipeline = QueryPipeline(std::move(pipe)); return res; } diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 42e8234adc1..ae9e4e851ca 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -332,17 +332,6 @@ bool QueryStatus::streamsAreReleased() return query_streams_status == QueryStreamsStatus::Released; } -bool QueryStatus::tryGetQueryStreams(BlockInputStreamPtr & in) const -{ - std::lock_guard lock(query_streams_mutex); - - if (query_streams_status != QueryStreamsStatus::Initialized) - return false; - - in = query_stream_in; - return true; -} - CancellationCode QueryStatus::cancelQuery(bool kill) { /// Streams are destroyed, and ProcessListElement will be deleted from ProcessList soon. We need wait a little bit diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index e37309b6a6c..9f9d099ed83 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -105,11 +105,6 @@ protected: mutable std::mutex query_streams_mutex; - /// Streams with query results, point to BlockIO from executeQuery() - /// This declaration is compatible with notes about BlockIO::process_list_entry: - /// there are no cyclic dependencies: BlockIO::in,out point to objects inside ProcessListElement (not whole object) - BlockInputStreamPtr query_stream_in; - /// Array of PipelineExecutors to be cancelled when a cancelQuery is received std::vector executors; @@ -181,9 +176,6 @@ public: /// It means that ProcessListEntry still exists, but stream was already destroyed bool streamsAreReleased(); - /// Get query in/out pointers from BlockIO - bool tryGetQueryStreams(BlockInputStreamPtr & in) const; - CancellationCode cancelQuery(bool kill); bool isKilled() const { return is_killed; } diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index d577eef14c1..7212597becc 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -489,7 +489,7 @@ void SystemLog::flushImpl(const std::vector & to_flush, InterpreterInsertQuery interpreter(query_ptr, insert_context); BlockIO io = interpreter.execute(); - PushingPipelineExecutor executor(io.out); + PushingPipelineExecutor executor(io.pipeline); executor.start(); executor.push(block); diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 31491fa12f8..6c7c564dcea 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -55,6 +55,7 @@ #include #include #include +#include #include @@ -587,8 +588,7 @@ static std::tuple executeQueryImpl( res = interpreter->execute(); } - QueryPipelineBuilder & pipeline = res.pipeline; - bool use_processors = pipeline.initialized(); + QueryPipeline & pipeline = res.pipeline; if (const auto * insert_interpreter = typeid_cast(&*interpreter)) { @@ -604,54 +604,23 @@ static std::tuple executeQueryImpl( if ((*process_list_entry)->isKilled()) throw Exception("Query '" + (*process_list_entry)->getInfo().client_info.current_query_id + "' is killed in pending state", ErrorCodes::QUERY_WAS_CANCELLED); - else if (!use_processors) - (*process_list_entry)->setQueryStreams(res); } /// Hold element of process list till end of query execution. res.process_list_entry = process_list_entry; - if (use_processors) + if (pipeline.pulling()) { /// Limits on the result, the quota on the result, and also callback for progress. /// Limits apply only to the final result. pipeline.setProgressCallback(context->getProgressCallback()); pipeline.setProcessListElement(context->getProcessListElement()); - if (stage == QueryProcessingStage::Complete && !pipeline.isCompleted()) - { - pipeline.resize(1); - pipeline.addSimpleTransform([&](const Block & header) - { - auto transform = std::make_shared(header, limits); - transform->setQuota(quota); - return transform; - }); - } + if (stage == QueryProcessingStage::Complete) + pipeline.setLimitsAndQuota(limits, quota); } - else + else if (pipeline.pushing()) { - /// Limits on the result, the quota on the result, and also callback for progress. - /// Limits apply only to the final result. - if (res.in) - { - res.in->setProgressCallback(context->getProgressCallback()); - res.in->setProcessListElement(context->getProcessListElement()); - if (stage == QueryProcessingStage::Complete) - { - if (!interpreter->ignoreQuota()) - res.in->setQuota(quota); - if (!interpreter->ignoreLimits()) - res.in->setLimits(limits); - } - } - - if (!res.out.empty()) - { - if (auto * counting = dynamic_cast(&res.out.getSource())) - { - counting->setProcessListElement(context->getProcessListElement()); - } - } + pipeline.setProcessListElement(context->getProcessListElement()); } /// Everything related to query log. @@ -678,15 +647,12 @@ static std::tuple executeQueryImpl( /// Log into system table start of query execution, if need. if (log_queries) { - if (use_processors) - { - const auto & info = context->getQueryAccessInfo(); - elem.query_databases = info.databases; - elem.query_tables = info.tables; - elem.query_columns = info.columns; - elem.query_projections = info.projections; - elem.query_views = info.views; - } + const auto & info = context->getQueryAccessInfo(); + elem.query_databases = info.databases; + elem.query_tables = info.tables; + elem.query_columns = info.columns; + elem.query_projections = info.projections; + elem.query_views = info.views; interpreter->extendQueryLogElem(elem, ast, context, query_database, query_table); @@ -759,7 +725,7 @@ static std::tuple executeQueryImpl( log_queries_min_query_duration_ms = settings.log_queries_min_query_duration_ms.totalMilliseconds(), status_info_to_query_log ] - (IBlockInputStream * stream_in, QueryPipelineBuilder * query_pipeline) mutable + (QueryPipeline & query_pipeline) mutable { QueryStatus * process_list_elem = context->getProcessListElement(); @@ -787,21 +753,9 @@ static std::tuple executeQueryImpl( if (progress_callback) progress_callback(Progress(WriteProgress(info.written_rows, info.written_bytes))); - if (stream_in) + else if (query_pipeline.pulling()) { - const BlockStreamProfileInfo & stream_in_info = stream_in->getProfileInfo(); - - /// NOTE: INSERT SELECT query contains zero metrics - elem.result_rows = stream_in_info.rows; - elem.result_bytes = stream_in_info.bytes; - } - else if (query_pipeline) - { - if (const auto * output_format = query_pipeline->getOutputFormat()) - { - elem.result_rows = output_format->getResultRows(); - elem.result_bytes = output_format->getResultBytes(); - } + query_pipeline.tryGetResultRowsAndBytes(elem.result_rows, elem.result_bytes); } else /// will be used only for ordinary INSERT queries { @@ -966,12 +920,10 @@ BlockIO executeQuery( bool may_have_embedded_data, bool allow_processors) { - BlockIO res = executeQuery(query, context, internal, stage, may_have_embedded_data); + if (!allow_processors) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Flag allow_processors is deprecated for executeQuery"); - if (!allow_processors && res.pipeline.initialized()) - res.in = res.getInputStream(); - - return res; + return executeQuery(query, context, internal, stage, may_have_embedded_data); } @@ -1028,70 +980,12 @@ void executeQuery( try { - if (!streams.out.empty()) + if (pipeline.pushing()) { - auto pipe = getSourceFromFromASTInsertQuery(ast, &istr, streams.out.getInputHeader(), context, nullptr); - - pipeline.init(std::move(pipe)); - pipeline.resize(1); - pipeline.addChain(std::move(streams.out)); - pipeline.setSinks([&](const Block & header, Pipe::StreamType) - { - return std::make_shared(header); - }); - - auto executor = pipeline.execute(); - executor->execute(pipeline.getNumThreads()); + auto pipe = getSourceFromFromASTInsertQuery(ast, &istr, pipeline.getHeader(), context, nullptr); + pipeline.complete(std::move(pipe)); } - else if (streams.in) - { - const auto * ast_query_with_output = dynamic_cast(ast.get()); - - WriteBuffer * out_buf = &ostr; - std::unique_ptr compressed_buffer; - if (ast_query_with_output && ast_query_with_output->out_file) - { - if (!allow_into_outfile) - throw Exception("INTO OUTFILE is not allowed", ErrorCodes::INTO_OUTFILE_NOT_ALLOWED); - - const auto & out_file = ast_query_with_output->out_file->as().value.safeGet(); - compressed_buffer = wrapWriteBufferWithCompressionMethod( - std::make_unique(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT), - chooseCompressionMethod(out_file, ""), - /* compression level = */ 3 - ); - } - - String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr) - ? getIdentifierName(ast_query_with_output->format) - : context->getDefaultFormat(); - - auto out = FormatFactory::instance().getOutputStreamParallelIfPossible( - format_name, - compressed_buffer ? *compressed_buffer : *out_buf, - streams.in->getHeader(), - context, - {}, - output_format_settings); - - /// Save previous progress callback if any. TODO Do it more conveniently. - auto previous_progress_callback = context->getProgressCallback(); - - /// NOTE Progress callback takes shared ownership of 'out'. - streams.in->setProgressCallback([out, previous_progress_callback] (const Progress & progress) - { - if (previous_progress_callback) - previous_progress_callback(progress); - out->onProgress(progress); - }); - - if (set_result_details) - set_result_details( - context->getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::instance().getTimeZone()); - - copyData(*streams.in, *out, [](){ return false; }, [&out](const Block &) { out->flush(); }); - } - else if (pipeline.initialized()) + else if (pipeline.pushing()) { const ASTQueryWithOutput * ast_query_with_output = dynamic_cast(ast.get()); @@ -1111,55 +1005,45 @@ void executeQuery( } String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr) - ? getIdentifierName(ast_query_with_output->format) - : context->getDefaultFormat(); + ? getIdentifierName(ast_query_with_output->format) + : context->getDefaultFormat(); - if (!pipeline.isCompleted()) + auto out = FormatFactory::instance().getOutputFormatParallelIfPossible( + format_name, + compressed_buffer ? *compressed_buffer : *out_buf, + pipeline.getHeader(), + context, + {}, + output_format_settings); + + out->setAutoFlush(); + + /// Save previous progress callback if any. TODO Do it more conveniently. + auto previous_progress_callback = context->getProgressCallback(); + + /// NOTE Progress callback takes shared ownership of 'out'. + pipeline.setProgressCallback([out, previous_progress_callback] (const Progress & progress) { - pipeline.addSimpleTransform([](const Block & header) - { - return std::make_shared(header); - }); + if (previous_progress_callback) + previous_progress_callback(progress); + out->onProgress(progress); + }); - auto out = FormatFactory::instance().getOutputFormatParallelIfPossible( - format_name, - compressed_buffer ? *compressed_buffer : *out_buf, - pipeline.getHeader(), - context, - {}, - output_format_settings); + out->setBeforeFinalizeCallback(before_finalize_callback); - out->setAutoFlush(); + if (set_result_details) + set_result_details( + context->getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::instance().getTimeZone()); - /// Save previous progress callback if any. TODO Do it more conveniently. - auto previous_progress_callback = context->getProgressCallback(); - - /// NOTE Progress callback takes shared ownership of 'out'. - pipeline.setProgressCallback([out, previous_progress_callback] (const Progress & progress) - { - if (previous_progress_callback) - previous_progress_callback(progress); - out->onProgress(progress); - }); - - out->setBeforeFinalizeCallback(before_finalize_callback); - - if (set_result_details) - set_result_details( - context->getClientInfo().current_query_id, out->getContentType(), format_name, DateLUT::instance().getTimeZone()); - - pipeline.setOutputFormat(std::move(out)); - } - else - { - pipeline.setProgressCallback(context->getProgressCallback()); - } - - { - auto executor = pipeline.execute(); - executor->execute(pipeline.getNumThreads()); - } + pipeline.complete(std::move(out)); } + else + { + pipeline.setProgressCallback(context->getProgressCallback()); + } + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); } catch (...) { @@ -1174,20 +1058,15 @@ void executeTrivialBlockIO(BlockIO & streams, ContextPtr context) { try { - if (!streams.out.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Query stream requires input, but no input buffer provided, it's a bug"); - if (streams.in) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Query stream requires output, but no output buffer provided, it's a bug"); - if (!streams.pipeline.initialized()) return; - if (!streams.pipeline.isCompleted()) + if (!streams.pipeline.completed()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Query pipeline requires output, but no output buffer provided, it's a bug"); streams.pipeline.setProgressCallback(context->getProgressCallback()); - auto executor = streams.pipeline.execute(); - executor->execute(streams.pipeline.getNumThreads()); + CompletedPipelineExecutor executor(streams.pipeline); + executor.execute(); } catch (...) { diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp new file mode 100644 index 00000000000..b76173e43b1 --- /dev/null +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -0,0 +1,16 @@ +#include +#include +#include + +namespace DB +{ + +CompletedPipelineExecutor::CompletedPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) {} + +void CompletedPipelineExecutor::execute() +{ + PipelineExecutor executor(pipeline.processors); + executor.execute(pipeline.getNumThreads()); +} + +} diff --git a/src/Processors/Executors/CompletedPipelineExecutor.h b/src/Processors/Executors/CompletedPipelineExecutor.h new file mode 100644 index 00000000000..d93e2ff5920 --- /dev/null +++ b/src/Processors/Executors/CompletedPipelineExecutor.h @@ -0,0 +1,19 @@ +#pragma once + +namespace DB +{ + +class QueryPipeline; + +class CompletedPipelineExecutor +{ +public: + explicit CompletedPipelineExecutor(QueryPipeline & pipeline_); + + void execute(); + +private: + QueryPipeline & pipeline; +}; + +} diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 92dd2501245..b58139c3dcb 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -2,7 +2,8 @@ #include #include #include -#include +#include +#include #include #include @@ -10,6 +11,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + struct PullingAsyncPipelineExecutor::Data { PipelineExecutorPtr executor; @@ -36,13 +42,13 @@ struct PullingAsyncPipelineExecutor::Data } }; -PullingAsyncPipelineExecutor::PullingAsyncPipelineExecutor(QueryPipelineBuilder & pipeline_) : pipeline(pipeline_) +PullingAsyncPipelineExecutor::PullingAsyncPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) { - if (!pipeline.isCompleted()) - { - lazy_format = std::make_shared(pipeline.getHeader()); - pipeline.setOutputFormat(lazy_format); - } + if (!pipeline.pulling()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for PullingAsyncPipelineExecutor must be pulling"); + + lazy_format = std::make_shared(pipeline.output->getHeader()); + pipeline.complete(lazy_format); } PullingAsyncPipelineExecutor::~PullingAsyncPipelineExecutor() @@ -59,8 +65,7 @@ PullingAsyncPipelineExecutor::~PullingAsyncPipelineExecutor() const Block & PullingAsyncPipelineExecutor::getHeader() const { - return lazy_format ? lazy_format->getPort(IOutputFormat::PortKind::Main).getHeader() - : pipeline.getHeader(); /// Empty. + return pipeline.output->getHeader(); } static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads) @@ -99,7 +104,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) if (!data) { data = std::make_unique(); - data->executor = pipeline.execute(); + data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element); data->lazy_format = lazy_format.get(); auto func = [&, thread_group = CurrentThread::getGroup()]() diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.h b/src/Processors/Executors/PullingAsyncPipelineExecutor.h index 17cb6a6a7ae..2ce75aecab7 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.h +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.h @@ -4,7 +4,7 @@ namespace DB { -class QueryPipelineBuilder; +class QueryPipeline; class Block; class Chunk; class LazyOutputFormat; @@ -20,7 +20,7 @@ struct BlockStreamProfileInfo; class PullingAsyncPipelineExecutor { public: - explicit PullingAsyncPipelineExecutor(QueryPipelineBuilder & pipeline_); + explicit PullingAsyncPipelineExecutor(QueryPipeline & pipeline_); ~PullingAsyncPipelineExecutor(); /// Get structure of returned block or chunk. @@ -50,7 +50,7 @@ public: struct Data; private: - QueryPipelineBuilder & pipeline; + QueryPipeline & pipeline; std::shared_ptr lazy_format; std::unique_ptr data; }; diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index f13cfce3195..7ad228760fb 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -1,15 +1,25 @@ #include +#include #include -#include +#include #include +#include namespace DB { -PullingPipelineExecutor::PullingPipelineExecutor(QueryPipelineBuilder & pipeline_) : pipeline(pipeline_) +namespace ErrorCodes { - pulling_format = std::make_shared(pipeline.getHeader(), has_data_flag); - pipeline.setOutputFormat(pulling_format); + extern const int LOGICAL_ERROR; +} + +PullingPipelineExecutor::PullingPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) +{ + if (!pipeline.pulling()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for PullingPipelineExecutor must be pulling"); + + pulling_format = std::make_shared(pipeline.output->getHeader(), has_data_flag); + pipeline.complete(pulling_format); } PullingPipelineExecutor::~PullingPipelineExecutor() @@ -26,13 +36,13 @@ PullingPipelineExecutor::~PullingPipelineExecutor() const Block & PullingPipelineExecutor::getHeader() const { - return pulling_format->getPort(IOutputFormat::PortKind::Main).getHeader(); + return pipeline.output->getHeader(); } bool PullingPipelineExecutor::pull(Chunk & chunk) { if (!executor) - executor = pipeline.execute(); + executor = std::make_shared(pipeline.processors, pipeline.process_list_element); if (!executor->executeStep(&has_data_flag)) return false; diff --git a/src/Processors/Executors/PullingPipelineExecutor.h b/src/Processors/Executors/PullingPipelineExecutor.h index 3646905f9f9..878d66bd3d4 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.h +++ b/src/Processors/Executors/PullingPipelineExecutor.h @@ -7,7 +7,7 @@ namespace DB class Block; class Chunk; -class QueryPipelineBuilder; +class QueryPipeline; class PipelineExecutor; class PullingOutputFormat; struct BlockStreamProfileInfo; @@ -23,7 +23,7 @@ using PipelineExecutorPtr = std::shared_ptr; class PullingPipelineExecutor { public: - explicit PullingPipelineExecutor(QueryPipelineBuilder & pipeline_); + explicit PullingPipelineExecutor(QueryPipeline & pipeline_); ~PullingPipelineExecutor(); /// Get structure of returned block or chunk. @@ -50,7 +50,7 @@ public: private: std::atomic_bool has_data_flag = false; - QueryPipelineBuilder & pipeline; + QueryPipeline & pipeline; std::shared_ptr pulling_format; PipelineExecutorPtr executor; }; diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 938ce6f9b49..00a5892151c 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include @@ -127,19 +127,14 @@ static void threadFunction(PushingAsyncPipelineExecutor::Data & data, ThreadGrou } -PushingAsyncPipelineExecutor::PushingAsyncPipelineExecutor(Chain & chain_) : chain(chain_) +PushingAsyncPipelineExecutor::PushingAsyncPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) { - pushing_source = std::make_shared(chain.getInputHeader()); - auto sink = std::make_shared(chain.getOutputHeader()); - connect(pushing_source->getPort(), chain.getInputPort()); - connect(chain.getOutputPort(), sink->getPort()); + if (!pipeline.pushing()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for PushingPipelineExecutor must be pushing"); - processors = std::make_unique(); - processors->reserve(chain.getProcessors().size() + 2); - for (const auto & processor : chain.getProcessors()) - processors->push_back(processor); - processors->push_back(pushing_source); - processors->push_back(std::move(sink)); + pushing_source = std::make_shared(pipeline.input->getHeader()); + connect(pushing_source->getPort(), *pipeline.input); + pipeline.processors.emplace_back(pushing_source); } PushingAsyncPipelineExecutor::~PushingAsyncPipelineExecutor() @@ -168,12 +163,12 @@ void PushingAsyncPipelineExecutor::start() started = true; data = std::make_unique(); - data->executor = std::make_shared(*processors); + data->executor = std::make_shared(pipeline.processors); data->source = pushing_source.get(); auto func = [&, thread_group = CurrentThread::getGroup()]() { - threadFunction(*data, thread_group, chain.getNumThreads()); + threadFunction(*data, thread_group, pipeline.getNumThreads()); }; data->thread = ThreadFromGlobalPool(std::move(func)); diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.h b/src/Processors/Executors/PushingAsyncPipelineExecutor.h index 2dac7cc64f0..5682769fe8f 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.h +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.h @@ -7,7 +7,7 @@ namespace DB class Block; class Chunk; -class Chain; +class QueryPipeline; class PushingAsyncSource; class PipelineExecutor; @@ -28,7 +28,7 @@ using Processors = std::vector; class PushingAsyncPipelineExecutor { public: - explicit PushingAsyncPipelineExecutor(Chain & chain); + explicit PushingAsyncPipelineExecutor(QueryPipeline & pipeline_); ~PushingAsyncPipelineExecutor(); /// Get structure of returned block or chunk. @@ -47,10 +47,9 @@ public: struct Data; private: - Chain & chain; + QueryPipeline & pipeline; std::shared_ptr pushing_source; - std::unique_ptr processors; bool started = false; bool finished = false; diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index f5857b4fa8a..61aae02c068 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -1,8 +1,7 @@ #include #include #include -#include -#include +#include #include @@ -52,19 +51,14 @@ private: }; -PushingPipelineExecutor::PushingPipelineExecutor(Chain & chain_) : chain(chain_) +PushingPipelineExecutor::PushingPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) { - pushing_source = std::make_shared(chain.getInputHeader(), need_data_flag); - auto sink = std::make_shared(chain.getOutputHeader()); - connect(pushing_source->getPort(), chain.getInputPort()); - connect(chain.getOutputPort(), sink->getPort()); + if (!pipeline.pushing()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for PushingPipelineExecutor must be pushing"); - processors = std::make_unique(); - processors->reserve(chain.getProcessors().size() + 2); - for (const auto & processor : chain.getProcessors()) - processors->push_back(processor); - processors->push_back(pushing_source); - processors->push_back(std::move(sink)); + pushing_source = std::make_shared(pipeline.input->getHeader(), need_data_flag); + connect(pushing_source->getPort(), *pipeline.input); + pipeline.processors.emplace_back(pushing_source); } PushingPipelineExecutor::~PushingPipelineExecutor() @@ -91,7 +85,7 @@ void PushingPipelineExecutor::start() return; started = true; - executor = std::make_shared(*processors); + executor = std::make_shared(pipeline.processors); if (!executor->executeStep(&need_data_flag)) throw Exception(ErrorCodes::LOGICAL_ERROR, diff --git a/src/Processors/Executors/PushingPipelineExecutor.h b/src/Processors/Executors/PushingPipelineExecutor.h index f15a845a233..86c9ec02290 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.h +++ b/src/Processors/Executors/PushingPipelineExecutor.h @@ -7,7 +7,7 @@ namespace DB class Block; class Chunk; -class Chain; +class QueryPipeline; class PushingSource; class PipelineExecutor; @@ -28,7 +28,7 @@ using Processors = std::vector; class PushingPipelineExecutor { public: - explicit PushingPipelineExecutor(Chain & chain); + explicit PushingPipelineExecutor(QueryPipeline & pipeline_); ~PushingPipelineExecutor(); /// Get structure of returned block or chunk. @@ -45,11 +45,10 @@ public: void cancel(); private: - Chain & chain; + QueryPipeline & pipeline; std::atomic_bool need_data_flag = false; std::shared_ptr pushing_source; - std::unique_ptr processors; PipelineExecutorPtr executor; bool started = false; bool finished = false; diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 4d86d18f70e..16216e7f363 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -82,6 +82,8 @@ public: virtual void doWritePrefix() {} virtual void doWriteSuffix() { finalize(); } + virtual bool expectMaterializedColumns() const { return true; } + void setTotals(const Block & totals) { consumeTotals(Chunk(totals.getColumns(), totals.rows())); } void setExtremes(const Block & extremes) { consumeExtremes(Chunk(extremes.getColumns(), extremes.rows())); } diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index c6be0adb347..2c29f55c4f3 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -44,6 +44,8 @@ public: queue.emplace(Chunk()); } + bool expectMaterializedColumns() const override { return false; } + protected: void consume(Chunk chunk) override { diff --git a/src/Processors/Formats/PullingOutputFormat.h b/src/Processors/Formats/PullingOutputFormat.h index 0864b5a02ef..53b2086712f 100644 --- a/src/Processors/Formats/PullingOutputFormat.h +++ b/src/Processors/Formats/PullingOutputFormat.h @@ -24,6 +24,8 @@ public: void setRowsBeforeLimit(size_t rows_before_limit) override; + bool expectMaterializedColumns() const override { return false; } + protected: void consume(Chunk chunk) override; void consumeTotals(Chunk chunk) override { totals = std::move(chunk); } diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 66dcce9e277..a07c68f56b2 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -144,6 +144,7 @@ private: void setOutputFormat(ProcessorPtr output); friend class QueryPipelineBuilder; + friend class QueryPipeline; }; } diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp new file mode 100644 index 00000000000..cf4890d3a76 --- /dev/null +++ b/src/Processors/QueryPipeline.cpp @@ -0,0 +1,313 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +QueryPipeline::QueryPipeline() = default; +QueryPipeline::QueryPipeline(QueryPipeline &&) = default; +QueryPipeline & QueryPipeline::operator=(QueryPipeline &&) = default; +QueryPipeline::~QueryPipeline() = default; + +static void checkInput(const InputPort & input, const ProcessorPtr & processor) +{ + if (!input.isConnected()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create QueryPipeline because {} has not connected input", + processor->getName()); +} + +static void checkOutput(const OutputPort & output, const ProcessorPtr & processor) +{ + if (!output.isConnected()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create QueryPipeline because {} has not connected output", + processor->getName()); +} + +QueryPipeline::QueryPipeline( + PipelineResourcesHolder resources_, + Processors processors_) + : resources(std::move(resources_)) + , processors(std::move(processors_)) +{ + for (const auto & processor : processors) + { + for (const auto & in : processor->getInputs()) + checkInput(in, processor); + + for (const auto & out : processor->getOutputs()) + checkOutput(out, processor); + } +} + +QueryPipeline::QueryPipeline( + PipelineResourcesHolder resources_, + Processors processors_, + InputPort * input_) + : resources(std::move(resources_)) + , processors(std::move(processors_)) + , input(input_) +{ + if (!input || input->isConnected()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create pushing QueryPipeline because its input port is connected or null"); + + bool found_input = false; + for (const auto & processor : processors) + { + for (const auto & in : processor->getInputs()) + { + if (&in == input) + found_input = true; + else + checkInput(in, processor); + } + + for (const auto & out : processor->getOutputs()) + checkOutput(out, processor); + } + + if (!found_input) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create pushing QueryPipeline because its input port does not belong to any processor"); +} + +QueryPipeline::QueryPipeline( + PipelineResourcesHolder resources_, + Processors processors_, + OutputPort * output_, + OutputPort * totals_, + OutputPort * extremes_) + : resources(std::move(resources_)) + , processors(std::move(processors_)) + , output(output_) + , totals(totals_) + , extremes(extremes_) +{ + if (!output || output->isConnected()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create pulling QueryPipeline because its output port is connected or null"); + + if (totals && totals->isConnected()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create pulling QueryPipeline because its totals port is connected"); + + if (extremes || extremes->isConnected()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create pulling QueryPipeline because its extremes port is connected"); + + bool found_output = false; + bool found_totals = false; + bool found_extremes = false; + for (const auto & processor : processors) + { + for (const auto & in : processor->getInputs()) + checkInput(in, processor); + + for (const auto & out : processor->getOutputs()) + { + if (&out == output) + found_output = true; + else if (totals && &out == totals) + found_totals = true; + else if (extremes && &out == extremes) + found_extremes = true; + else + checkOutput(out, processor); + } + } + + if (!found_output) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create pulling QueryPipeline because its output port does not belong to any processor"); + if (totals && !found_totals) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create pulling QueryPipeline because its totals port does not belong to any processor"); + if (extremes && !found_extremes) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create pulling QueryPipeline because its extremes port does not belong to any processor"); +} + +QueryPipeline::QueryPipeline(std::shared_ptr source) : QueryPipeline(Pipe(std::move(source))) {} + +QueryPipeline::QueryPipeline(Pipe pipe) + : QueryPipeline(std::move(pipe.holder), std::move(pipe.processors), pipe.getOutputPort(0), pipe.getTotalsPort(), pipe.getExtremesPort()) +{ +} + +QueryPipeline::QueryPipeline(Chain chain) + : resources(chain.detachResources()) + , input(&chain.getInputPort()) + , num_threads(chain.getNumThreads()) +{ + processors.reserve(chain.getProcessors().size() + 1); + for (auto processor : chain.getProcessors()) + processors.emplace_back(std::move(processor)); + + auto sink = std::make_shared(chain.getOutputPort().getHeader()); + connect(chain.getOutputPort(), sink->getPort()); + processors.emplace_back(std::move(sink)); + + input = &chain.getInputPort(); +} + +QueryPipeline::QueryPipeline(std::shared_ptr sink) : QueryPipeline(Chain(std::move(sink))) {} + +void QueryPipeline::complete(Pipe pipe) +{ + if (!pushing()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline must be pushing to be completed with pipe"); + + pipe.resize(1); + resources = pipe.detachResources(); + pipe.dropExtremes(); + pipe.dropTotals(); + connect(*pipe.getOutputPort(0), *input); + input = nullptr; + + auto pipe_processors = Pipe::detachProcessors(std::move(pipe)); + processors.insert(processors.end(), pipe_processors.begin(), pipe_processors.end()); +} + +void QueryPipeline::complete(std::shared_ptr format) +{ + if (!pulling()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline must be pulling to be completed with output format"); + + if (format->expectMaterializedColumns()) + { + auto materializing = std::make_shared(output->getHeader()); + connect(*output, materializing->getInputPort()); + output = &materializing->getOutputPort(); + processors.emplace_back(std::move(output)); + } + + auto & format_main = format->getPort(IOutputFormat::PortKind::Main); + auto & format_totals = format->getPort(IOutputFormat::PortKind::Totals); + auto & format_extremes = format->getPort(IOutputFormat::PortKind::Extremes); + + if (!totals) + { + auto source = std::make_shared(totals->getHeader()); + totals = &source->getPort(); + processors.emplace_back(std::move(source)); + } + + if (!extremes) + { + auto source = std::make_shared(extremes->getHeader()); + extremes = &source->getPort(); + processors.emplace_back(std::move(source)); + } + + processors.emplace_back(std::move(format)); + + connect(*output, format_main); + connect(*totals, format_totals); + connect(*extremes, format_extremes); + + output = nullptr; + totals = nullptr; + extremes = nullptr; +} + +Block QueryPipeline::getHeader() const +{ + if (input) + return input->getHeader(); + else if (output) + return output->getHeader(); + else + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Header is available only for pushing or pulling QueryPipeline"); +} + +void QueryPipeline::setProgressCallback(const ProgressCallback & callback) +{ + for (auto & processor : processors) + { + if (auto * source = dynamic_cast(processor.get())) + source->setProgressCallback(callback); + } +} + +void QueryPipeline::setProcessListElement(QueryStatus * elem) +{ + process_list_element = elem; + + if (pulling()) + { + for (auto & processor : processors) + { + if (auto * source = dynamic_cast(processor.get())) + source->setProcessListElement(elem); + } + } + else if (pushing()) + { + if (auto * counting = dynamic_cast(&input->getOutputPort().getProcessor())) + { + counting->setProcessListElement(elem); + } + } +} + + +void QueryPipeline::setLimitsAndQuota(const StreamLocalLimits & limits, std::shared_ptr quota) +{ + if (!pulling()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "It is possible to set limits and quota only to pullint QueryPipeline"); + + auto transform = std::make_shared(output->getHeader(), limits); + transform->setQuota(quota); + connect(*output, transform->getInputPort()); + output = &transform->getOutputPort(); + processors.emplace_back(std::move(transform)); +} + + +bool QueryPipeline::tryGetResultRowsAndBytes(size_t & result_rows, size_t & result_bytes) const +{ + if (!output || !output->isConnected()) + return false; + + const auto * format = typeid_cast(&output->getInputPort().getProcessor()); + if (!format) + return false; + + result_rows = format->getResultRows(); + result_bytes = format->getResultBytes(); + return true; +} + +void QueryPipeline::reset() +{ + QueryPipeline to_remove = std::move(*this); + *this = QueryPipeline(); +} + +} diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h new file mode 100644 index 00000000000..e14f42dd07a --- /dev/null +++ b/src/Processors/QueryPipeline.h @@ -0,0 +1,117 @@ +#pragma once +#include + +namespace DB +{ + +class InputPort; +class OutputPort; + +class IProcessor; +using ProcessorPtr = std::shared_ptr; +using Processors = std::vector; + +class QueryStatus; + +struct Progress; +using ProgressCallback = std::function; + +struct StreamLocalLimits; +class EnabledQuota; + +class Block; +class Pipe; +class Chain; +class IOutputFormat; +class SinkToStorage; +class ISource; + +class QueryPipeline +{ +public: + QueryPipeline(); + QueryPipeline(QueryPipeline &&); + QueryPipeline(const QueryPipeline &) = delete; + + QueryPipeline & operator=(QueryPipeline &&); + QueryPipeline & operator=(const QueryPipeline &) = delete; + + ~QueryPipeline(); + + /// pulling + explicit QueryPipeline(Pipe pipe); + explicit QueryPipeline(std::shared_ptr source); + /// pushing + explicit QueryPipeline(Chain chain); + explicit QueryPipeline(std::shared_ptr sink); + + /// completed + QueryPipeline( + PipelineResourcesHolder resources_, + Processors processors_); + + /// pushing + QueryPipeline( + PipelineResourcesHolder resources_, + Processors processors_, + InputPort * input_); + + /// pulling + QueryPipeline( + PipelineResourcesHolder resources_, + Processors processors_, + OutputPort * output_, + OutputPort * totals_ = nullptr, + OutputPort * extremes_ = nullptr); + + /// Exactly one of the following is true. + bool initialized() const { return !processors.empty(); } + /// Use PullingPipelineExecutor or PullingAsyncPipelineExecutor. + bool pulling() const { return output != nullptr; } + /// Use PushingPipelineExecutor or PushingAsyncPipelineExecutor. + bool pushing() const { return input != nullptr; } + /// Use PipelineExecutor. Call execute() to build one. + bool completed() const { return !pulling() && !pushing(); } + + /// Only for pushing. + void complete(Pipe pipe); + /// Only for pulling. + void complete(std::shared_ptr format); + + /// Only for pushing and pulling. + Block getHeader() const; + + size_t getNumThreads() const { return num_threads; } + void setNumThreads(size_t num_threads_) { num_threads = num_threads_; } + + void setProcessListElement(QueryStatus * elem); + void setProgressCallback(const ProgressCallback & callback); + void setLimitsAndQuota(const StreamLocalLimits & limits, std::shared_ptr quota); + bool tryGetResultRowsAndBytes(size_t & result_rows, size_t & result_bytes) const; + + const Processors & getProcessors() const { return processors; } + + void reset(); + +private: + PipelineResourcesHolder resources; + Processors processors; + + InputPort * input = nullptr; + + OutputPort * output = nullptr; + OutputPort * totals = nullptr; + OutputPort * extremes = nullptr; + + QueryStatus * process_list_element = nullptr; + + size_t num_threads = 0; + + friend class PushingPipelineExecutor; + friend class PullingPipelineExecutor; + friend class PushingAsyncPipelineExecutor; + friend class PullingAsyncPipelineExecutor; + friend class CompletedPipelineExecutor; +}; + +} diff --git a/src/Processors/QueryPipelineBuilder.cpp b/src/Processors/QueryPipelineBuilder.cpp index b3f7290b4d3..ab4b7501870 100644 --- a/src/Processors/QueryPipelineBuilder.cpp +++ b/src/Processors/QueryPipelineBuilder.cpp @@ -538,6 +538,13 @@ PipelineExecutorPtr QueryPipelineBuilder::execute() return std::make_shared(pipe.processors, process_list_element); } +QueryPipeline QueryPipelineBuilder::getPipeline(QueryPipelineBuilder builder) +{ + QueryPipeline res(std::move(builder.pipe)); + res.setNumThreads(builder.getNumThreads()); + return res; +} + void QueryPipelineBuilder::setCollectedProcessors(Processors * processors) { pipe.collected_processors = processors; diff --git a/src/Processors/QueryPipelineBuilder.h b/src/Processors/QueryPipelineBuilder.h index fd44aea2d33..5ca65398bcc 100644 --- a/src/Processors/QueryPipelineBuilder.h +++ b/src/Processors/QueryPipelineBuilder.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -155,6 +156,7 @@ public: /// Convert query pipeline to pipe. static Pipe getPipe(QueryPipelineBuilder pipeline) { return std::move(pipeline.pipe); } + static QueryPipeline getPipeline(QueryPipelineBuilder builder); private: diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b3dece3192e..b2d3961ba1b 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include "Core/Protocol.h" @@ -295,7 +296,7 @@ void TCPHandler::runImpl() after_check_cancelled.restart(); after_send_progress.restart(); - if (!state.io.out.empty()) + if (state.io.pipeline.pushing()) { state.need_receive_data_for_insert = true; processInsertQuery(); @@ -303,13 +304,13 @@ void TCPHandler::runImpl() else if (state.need_receive_data_for_input) // It implies pipeline execution { /// It is special case for input(), all works for reading data from client will be done in callbacks. - auto executor = state.io.pipeline.execute(); - executor->execute(state.io.pipeline.getNumThreads()); + CompletedPipelineExecutor executor(state.io.pipeline); + executor.execute(); } - else if (state.io.pipeline.initialized()) + else if (state.io.pipeline.pulling()) processOrdinaryQueryWithProcessors(); - else if (state.io.in) - processOrdinaryQuery(); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected QueryPipeline state."); state.io.onFinish(); @@ -544,7 +545,7 @@ void TCPHandler::skipData() void TCPHandler::processInsertQuery() { - size_t num_threads = state.io.out.getNumThreads(); + size_t num_threads = state.io.pipeline.getNumThreads(); auto send_table_columns = [&]() { @@ -565,7 +566,7 @@ void TCPHandler::processInsertQuery() if (num_threads > 1) { - PushingAsyncPipelineExecutor executor(state.io.out); + PushingAsyncPipelineExecutor executor(state.io.pipeline); /** Made above the rest of the lines, so that in case of `writePrefix` function throws an exception, * client receive exception before sending data. */ @@ -585,7 +586,7 @@ void TCPHandler::processInsertQuery() } else { - PushingPipelineExecutor executor(state.io.out); + PushingPipelineExecutor executor(state.io.pipeline); executor.start(); send_table_columns(); @@ -602,77 +603,6 @@ void TCPHandler::processInsertQuery() } -void TCPHandler::processOrdinaryQuery() -{ - OpenTelemetrySpanHolder span(__PRETTY_FUNCTION__); - - /// Pull query execution result, if exists, and send it to network. - if (state.io.in) - { - - if (query_context->getSettingsRef().allow_experimental_query_deduplication) - sendPartUUIDs(); - - /// This allows the client to prepare output format - if (Block header = state.io.in->getHeader()) - sendData(header); - - /// Use of async mode here enables reporting progress and monitoring client cancelling the query - AsynchronousBlockInputStream async_in(state.io.in); - - async_in.readPrefix(); - while (true) - { - if (isQueryCancelled()) - { - async_in.cancel(false); - break; - } - - if (after_send_progress.elapsed() / 1000 >= interactive_delay) - { - /// Some time passed. - after_send_progress.restart(); - sendProgress(); - } - - sendLogs(); - - if (async_in.poll(interactive_delay / 1000)) - { - const auto block = async_in.read(); - if (!block) - break; - - if (!state.io.null_format) - sendData(block); - } - } - async_in.readSuffix(); - - /** When the data has run out, we send the profiling data and totals up to the terminating empty block, - * so that this information can be used in the suffix output of stream. - * If the request has been interrupted, then sendTotals and other methods should not be called, - * because we have not read all the data. - */ - if (!isQueryCancelled()) - { - sendTotals(state.io.in->getTotals()); - sendExtremes(state.io.in->getExtremes()); - sendProfileInfo(state.io.in->getProfileInfo()); - sendProgress(); - } - - if (state.is_connection_closed) - return; - - sendData({}); - } - - sendProgress(); -} - - void TCPHandler::processOrdinaryQueryWithProcessors() { auto & pipeline = state.io.pipeline; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 50411c826f7..4c9542abc94 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -617,7 +617,7 @@ bool StorageKafka::streamToViews() streams.reserve(stream_count); for (size_t i = 0; i < stream_count; ++i) { - auto stream = std::make_shared(*this, metadata_snapshot, kafka_context, block_io.out.getInputHeader().getNames(), log, block_size, false); + auto stream = std::make_shared(*this, metadata_snapshot, kafka_context, block_io.pipeline.getHeader().getNames(), log, block_size, false); streams.emplace_back(stream); // Limit read batch to maximum block size to allow DDL @@ -642,7 +642,7 @@ bool StorageKafka::streamToViews() // It will be cancelled on underlying layer (kafka buffer) size_t rows = 0; - PushingPipelineExecutor executor(block_io.out); + PushingPipelineExecutor executor(block_io.pipeline); in->readPrefix(); executor.start(); diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 83835a048c3..ee5551a1e84 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB { @@ -485,21 +486,15 @@ void MaterializedPostgreSQLConsumer::syncTables() insert->columns = buffer.columnsAST; InterpreterInsertQuery interpreter(insert, insert_context, true); - auto block_io = interpreter.execute(); + auto io = interpreter.execute(); auto input = std::make_shared( result_rows.cloneEmpty(), Chunk(result_rows.getColumns(), result_rows.rows())); - assertBlocksHaveEqualStructure(input->getPort().getHeader(), block_io.out.getInputHeader(), "postgresql replica table sync"); - QueryPipelineBuilder pipeline; - pipeline.init(Pipe(std::move(input))); - pipeline.addChain(std::move(block_io.out)); - pipeline.setSinks([&](const Block & header, Pipe::StreamType) - { - return std::make_shared(header); - }); + assertBlocksHaveEqualStructure(input->getPort().getHeader(), io.pipeline.getHeader(), "postgresql replica table sync"); + io.pipeline.complete(Pipe(std::move(input))); - auto executor = pipeline.execute(); - executor->execute(1); + CompletedPipelineExecutor executor(io.pipeline); + executor.execute(); buffer.columns = buffer.description.sample_block.cloneEmptyColumns(); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 8490d39c917..7c4fbabc473 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,8 +1,8 @@ #include "PostgreSQLReplicationHandler.h" #include -#include #include +#include #include #include #include @@ -239,29 +239,21 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(String & snapshot_name materialized_storage->createNestedIfNeeded(fetchTableStructure(*tx, table_name)); auto nested_storage = materialized_storage->getNested(); - auto insert = std::make_shared(); - insert->table_id = nested_storage->getStorageID(); - auto insert_context = materialized_storage->getNestedTableContext(); - InterpreterInsertQuery interpreter(insert, insert_context); - auto block_io = interpreter.execute(); + InterpreterInsertQuery interpreter(nullptr, insert_context); + auto chain = interpreter.buildChain(nested_storage, nested_storage->getInMemoryMetadataPtr(), {}); const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); auto input = std::make_unique>(tx, query_str, sample_block, DEFAULT_BLOCK_SIZE); - QueryPipelineBuilder pipeline; - pipeline.init(Pipe(std::move(input))); - assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out.getInputHeader(), "postgresql replica load from snapshot"); - pipeline.addChain(std::move(block_io.out)); - pipeline.setSinks([&](const Block & header, Pipe::StreamType) - { - return std::make_shared(header); - }); + assertBlocksHaveEqualStructure(input->getPort().getHeader(), chain.getInputHeader(), "postgresql replica load from snapshot"); + QueryPipeline pipeline(std::move(chain)); + pipeline.complete(Pipe(std::move(input))); - auto executor = pipeline.execute(); - executor->execute(1); + CompletedPipelineExecutor executor(pipeline); + executor.execute(); nested_storage = materialized_storage->prepare(); auto nested_table_id = nested_storage->getStorageID(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index aceae5ec123..ca398f9c4e3 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -919,11 +919,11 @@ bool StorageRabbitMQ::streamToViews() insert->table_id = table_id; // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(insert, rabbitmq_context, false, true, true); - auto block_io = interpreter.execute(); + InterpreterInsertQuery interpreter(nullptr, rabbitmq_context, false, true, true); + auto chain = interpreter.buildChain(table, table->getInMemoryMetadataPtr(), {}); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto column_names = block_io.out.getInputHeader().getNames(); + auto column_names = chain.getInputHeader().getNames(); auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); auto block_size = getMaxBlockSize(); @@ -963,7 +963,8 @@ bool StorageRabbitMQ::streamToViews() looping_task->activateAndSchedule(); } - PushingPipelineExecutor executor(block_io.out); + QueryPipeline pipeline(std::move(chain)); + PushingPipelineExecutor executor(pipeline); executor.start(); in->readPrefix(); while (auto block = in->read()) From 2249ed858eba2dc562a1c636eaef110251755cde Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 16 Sep 2021 15:40:44 +0300 Subject: [PATCH 055/142] Add anchors to all the headings --- .../example-datasets/opensky.md | 18 +++++----- .../example-datasets/uk-price-paid.md | 34 +++++++++---------- .../example-datasets/opensky.md | 18 +++++----- .../example-datasets/uk-price-paid.md | 34 +++++++++---------- 4 files changed, 52 insertions(+), 52 deletions(-) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md index 119b5cc4b41..261c101fab8 100644 --- a/docs/en/getting-started/example-datasets/opensky.md +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -3,7 +3,7 @@ toc_priority: 20 toc_title: OpenSky --- -# Crowdsourced air traffic data from The OpenSky Network 2020 +# Crowdsourced air traffic data from The OpenSky Network 2020 {#opensky} "The data in this dataset is derived and cleaned from the full OpenSky dataset to illustrate the development of air traffic during the COVID-19 pandemic. It spans all flights seen by the network's more than 2500 members since 1 January 2019. More data will be periodically included in the dataset until the end of the COVID-19 pandemic". @@ -14,7 +14,7 @@ Martin Strohmeier, Xavier Olive, Jannis Lübbe, Matthias Schäfer, and Vincent L Earth System Science Data 13(2), 2021 https://doi.org/10.5194/essd-13-357-2021 -## Download the Dataset +## Download the Dataset {#download-dataset} Run the command: @@ -24,7 +24,7 @@ wget -O- https://zenodo.org/record/5092942 | grep -oP 'https://zenodo.org/record Download will take about 2 minutes with good internet connection. There are 30 files with total size of 4.3 GB. -## Create the Table +## Create the Table {#create-table} ```sql CREATE TABLE opensky @@ -48,7 +48,7 @@ CREATE TABLE opensky ) ENGINE = MergeTree ORDER BY (origin, destination, callsign); ``` -## Import Data +## Import Data {#import-data} Upload data into ClickHouse in parallel: @@ -72,7 +72,7 @@ If you don't like parallel upload, here is sequential variant: for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"; done ``` -## Validate the Data +## Validate the Data {#validate-data} Query: @@ -104,7 +104,7 @@ Result: └─────────────────────────────────┘ ``` -## Run Some Queries +## Run Some Queries {#run-queries} Total distance travelled is 68 billion kilometers. @@ -138,7 +138,7 @@ Result: └────────────────────────────────────────────────────────────────────┘ ``` -### Most busy origin airports and the average distance seen +### Most busy origin airports and the average distance seen {#busy-airports-average-distance} Query: @@ -262,7 +262,7 @@ Result: └────────┴─────────┴──────────┴────────────────────────────────────────┘ ``` -### Number of flights from three major Moscow airports, weekly +### Number of flights from three major Moscow airports, weekly {#flights-from-moscow} Query: @@ -415,6 +415,6 @@ Result: └────────────┴──────┴──────────────────────────────────────────────────────────────────────────────┘ ``` -### Online Playground +### Online Playground {#playground} You can test other queries to this data set using the interactive resource [Online Playground](https://gh-api.clickhouse.tech/play?user=play). For example, [like this](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICBvcmlnaW4sCiAgICBjb3VudCgpLAogICAgcm91bmQoYXZnKGdlb0Rpc3RhbmNlKGxvbmdpdHVkZV8xLCBsYXRpdHVkZV8xLCBsb25naXR1ZGVfMiwgbGF0aXR1ZGVfMikpKSBBUyBkaXN0YW5jZSwKICAgIGJhcihkaXN0YW5jZSwgMCwgMTAwMDAwMDAsIDEwMCkgQVMgYmFyCkZST00gb3BlbnNreQpXSEVSRSBvcmlnaW4gIT0gJycKR1JPVVAgQlkgb3JpZ2luCk9SREVSIEJZIGNvdW50KCkgREVTQwpMSU1JVCAxMDA=). However, please note that you cannot create temporary tables here. diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 3868c324b32..74fa7dddb3b 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -3,7 +3,7 @@ toc_priority: 20 toc_title: UK Property Price Paid --- -# UK Property Price Paid +# UK Property Price Paid {#uk-property-price-paid} The dataset contains data about prices paid for real-estate property in England and Wales. The data is available since year 1995. The size of the dataset in uncompressed form is about 4 GiB and it will take about 278 MiB in ClickHouse. @@ -13,7 +13,7 @@ Description of the fields: https://www.gov.uk/guidance/about-the-price-paid-data Contains HM Land Registry data © Crown copyright and database right 2021. This data is licensed under the Open Government Licence v3.0. -## Download the Dataset +## Download the Dataset {#download-dataset} Run the command: @@ -23,7 +23,7 @@ wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.c Download will take about 2 minutes with good internet connection. -## Create the Table +## Create the Table {#create-table} ```sql CREATE TABLE uk_price_paid @@ -46,7 +46,7 @@ CREATE TABLE uk_price_paid ) ENGINE = MergeTree ORDER BY (postcode1, postcode2, addr1, addr2); ``` -## Preprocess and Import Data +## Preprocess and Import Data {#preprocess-import-data} We will use `clickhouse-local` tool for data preprocessing and `clickhouse-client` to upload it. @@ -102,7 +102,7 @@ clickhouse-local --input-format CSV --structure ' It will take about 40 seconds. -## Validate the Data +## Validate the Data {#validate-data} Query: @@ -134,9 +134,9 @@ Result: └─────────────────────────────────┘ ``` -## Run Some Queries +## Run Some Queries {#run-queries} -### Query 1. Average Price Per Year +### Query 1. Average Price Per Year {#average-price} Query: @@ -178,7 +178,7 @@ Result: └──────┴────────┴────────────────────────────────────────┘ ``` -### Query 2. Average Price per Year in London +### Query 2. Average Price per Year in London {#average-price-london} Query: @@ -222,7 +222,7 @@ Result: Something happened in 2013. I don't have a clue. Maybe you have a clue what happened in 2020? -### Query 3. The Most Expensive Neighborhoods +### Query 3. The Most Expensive Neighborhoods {#most-expensive-neighborhoods} Query: @@ -351,11 +351,11 @@ Result: └──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ ``` -## Let's Speed Up Queries Using Projections +## Let's Speed Up Queries Using Projections {#speedup-with-projections} [Projections](../../sql-reference/statements/alter/projection.md) allow to improve queries speed by storing pre-aggregated data. -### Build a Projection +### Build a Projection {#build-projection} Create an aggregate projection by dimensions `toYear(date)`, `district`, `town`: @@ -385,7 +385,7 @@ ALTER TABLE uk_price_paid SETTINGS mutations_sync = 1; ``` -## Test Performance +## Test Performance {#test-performance} Let's run the same 3 queries. @@ -395,7 +395,7 @@ Enable projections for selects: SET allow_experimental_projection_optimization = 1; ``` -### Query 1. Average Price Per Year +### Query 1. Average Price Per Year {#average-price-projections} Query: @@ -443,7 +443,7 @@ Result: └──────┴────────┴────────────────────────────────────────┘ ``` -### Query 2. Average Price Per Year in London +### Query 2. Average Price Per Year in London {#average-price-london-projections} Query: @@ -492,7 +492,7 @@ Result: └──────┴─────────┴───────────────────────────────────────────────────────┘ ``` -### Query 3. The Most Expensive Neighborhoods +### Query 3. The Most Expensive Neighborhoods {#most-expensive-neighborhoods-projections} The condition (date >= '2020-01-01') needs to be modified to match projection dimension (toYear(date) >= 2020). @@ -622,7 +622,7 @@ Result: └──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ ``` -### Summary +### Summary {#summary} All 3 queries work much faster and read fewer rows. @@ -644,6 +644,6 @@ no projection: 100 rows in set. Elapsed: 0.069 sec. Processed 26.32 million rows projection: 100 rows in set. Elapsed: 0.029 sec. Processed 8.08 thousand rows, 511.08 KB (276.06 thousand rows/s., 17.47 MB/s.) ``` -### Test It in Playground +### Test It in Playground {#playground} The dataset is also available in the [Online Playground](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index 8b51e737f87..59d366da3c3 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -3,7 +3,7 @@ toc_priority: 20 toc_title: Набор данных о воздушном движении OpenSky Network 2020 --- -# Набор данных о воздушном движении OpenSky Network 2020 +# Набор данных о воздушном движении OpenSky Network 2020 {#opensky} "Данные в этом наборе получены и отфильтрованы из полного набора данных OpenSky, чтобы проиллюстрировать развитие воздушного движения во время пандемии COVID-19. Набор включает в себя все рейсы, которые видели более 2500 участников сети с 1 января 2019 года. Дополнительные данные будут периодически включаться в набор данных до окончания пандемии COVID-19". @@ -14,7 +14,7 @@ Martin Strohmeier, Xavier Olive, Jannis Lübbe, Matthias Schäfer, and Vincent L Earth System Science Data 13(2), 2021 https://doi.org/10.5194/essd-13-357-2021 -## Загрузите набор данных +## Загрузите набор данных {#download-dataset} Выполните команду: @@ -24,7 +24,7 @@ wget -O- https://zenodo.org/record/5092942 | grep -oP 'https://zenodo.org/record Загрузка займет около 2 минут при хорошем подключении к интернету. Будет загружено 30 файлов общим размером 4,3 ГБ. -## Создайте таблицу +## Создайте таблицу {#create-table} ```sql CREATE TABLE opensky @@ -48,7 +48,7 @@ CREATE TABLE opensky ) ENGINE = MergeTree ORDER BY (origin, destination, callsign); ``` -## Импортируйте данные в ClickHouse +## Импортируйте данные в ClickHouse {#import-data} Загрузите данные в ClickHouse параллельными потоками: @@ -73,7 +73,7 @@ ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c 'gzip -c -d "{}" | clickhou for file in flightlist_*.csv.gz; do gzip -c -d "$file" | clickhouse-client --date_time_input_format best_effort --query "INSERT INTO opensky FORMAT CSVWithNames"; done ``` -## Проверьте импортированные данные +## Проверьте импортированные данные {#validate-data} Запрос: @@ -105,7 +105,7 @@ SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'opensky' └─────────────────────────────────┘ ``` -## Примеры +## Примеры {#run-queries} Общее пройденное расстояние составляет 68 миллиардов километров. @@ -139,7 +139,7 @@ SELECT avg(geoDistance(longitude_1, latitude_1, longitude_2, latitude_2)) FROM o └────────────────────────────────────────────────────────────────────┘ ``` -### Наиболее загруженные аэропорты в указанных координатах и среднее пройденное расстояние +### Наиболее загруженные аэропорты в указанных координатах и среднее пройденное расстояние {#busy-airports-average-distance} Запрос: @@ -263,7 +263,7 @@ LIMIT 100; └────────┴─────────┴──────────┴────────────────────────────────────────┘ ``` -### Номера рейсов из трех крупных аэропортов Москвы, еженедельно +### Номера рейсов из трех крупных аэропортов Москвы, еженедельно {#flights-from-moscow} Запрос: @@ -416,6 +416,6 @@ ORDER BY k ASC; └────────────┴──────┴──────────────────────────────────────────────────────────────────────────────┘ ``` -### Online Playground +### Online Playground {#playground} Вы можете протестировать другие запросы к этому набору данным с помощью интерактивного ресурса [Online Playground](https://gh-api.clickhouse.tech/play?user=play). Например, [вот так](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICBvcmlnaW4sCiAgICBjb3VudCgpLAogICAgcm91bmQoYXZnKGdlb0Rpc3RhbmNlKGxvbmdpdHVkZV8xLCBsYXRpdHVkZV8xLCBsb25naXR1ZGVfMiwgbGF0aXR1ZGVfMikpKSBBUyBkaXN0YW5jZSwKICAgIGJhcihkaXN0YW5jZSwgMCwgMTAwMDAwMDAsIDEwMCkgQVMgYmFyCkZST00gb3BlbnNreQpXSEVSRSBvcmlnaW4gIT0gJycKR1JPVVAgQlkgb3JpZ2luCk9SREVSIEJZIGNvdW50KCkgREVTQwpMSU1JVCAxMDA=). Однако обратите внимание, что здесь нельзя создавать временные таблицы. diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index 44c33973f93..c3eae1cdff4 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -3,7 +3,7 @@ toc_priority: 20 toc_title: Набор данных о стоимости недвижимости в Великобритании --- -# Набор данных о стоимости недвижимости в Великобритании +# Набор данных о стоимости недвижимости в Великобритании {#uk-property-price-paid} Набор содержит данные о стоимости недвижимости в Англии и Уэльсе. Данные доступны с 1995 года. Размер набора данных в несжатом виде составляет около 4 GiB, а в ClickHouse он займет около 278 MiB. @@ -13,7 +13,7 @@ toc_title: Набор данных о стоимости недвижимост Набор содержит данные HM Land Registry data © Crown copyright and database right 2021. Эти данные лицензированы в соответствии с Open Government Licence v3.0. -## Загрузите набор данных +## Загрузите набор данных {#download-dataset} Выполните команду: @@ -23,7 +23,7 @@ wget http://prod.publicdata.landregistry.gov.uk.s3-website-eu-west-1.amazonaws.c Загрузка займет около 2 минут при хорошем подключении к интернету. -## Создайте таблицу +## Создайте таблицу {#create-table} ```sql CREATE TABLE uk_price_paid @@ -46,7 +46,7 @@ CREATE TABLE uk_price_paid ) ENGINE = MergeTree ORDER BY (postcode1, postcode2, addr1, addr2); ``` -## Обработайте и импортируйте данные +## Обработайте и импортируйте данные {#preprocess-import-data} В этом примере используется `clickhouse-local` для предварительной обработки данных и `clickhouse-client` для импорта данных. @@ -102,7 +102,7 @@ clickhouse-local --input-format CSV --structure ' Выполнение запроса займет около 40 секунд. -## Проверьте импортированные данные +## Проверьте импортированные данные {#validate-data} Запрос: @@ -134,9 +134,9 @@ SELECT formatReadableSize(total_bytes) FROM system.tables WHERE name = 'uk_price └─────────────────────────────────┘ ``` -## Примеры запросов +## Примеры запросов {#run-queries} -### Запрос 1. Средняя цена за год +### Запрос 1. Средняя цена за год {#average-price} Запрос: @@ -178,7 +178,7 @@ SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 1000000, └──────┴────────┴────────────────────────────────────────┘ ``` -### Запрос 2. Средняя цена за год в Лондоне +### Запрос 2. Средняя цена за год в Лондоне {#average-price-london} Запрос: @@ -222,7 +222,7 @@ SELECT toYear(date) AS year, round(avg(price)) AS price, bar(price, 0, 2000000, Что-то случилось в 2013 году. Я понятия не имею. Может быть, вы имеете представление о том, что произошло в 2020 году? -### Запрос 3. Самые дорогие районы +### Запрос 3. Самые дорогие районы {#most-expensive-neighborhoods} Запрос: @@ -351,11 +351,11 @@ LIMIT 100; └──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ ``` -## Ускорьте запросы с помощью проекций +## Ускорьте запросы с помощью проекций {#speedup-with-projections} [Проекции](../../sql-reference/statements/alter/projection.md) позволяют повысить скорость запросов за счет хранения предварительно агрегированных данных. -### Создайте проекцию +### Создайте проекцию {#build-projection} Создайте агрегирующую проекцию по параметрам `toYear(date)`, `district`, `town`: @@ -385,7 +385,7 @@ ALTER TABLE uk_price_paid SETTINGS mutations_sync = 1; ``` -## Проверьте производительность +## Проверьте производительность {#test-performance} Давайте выполним те же 3 запроса. @@ -395,7 +395,7 @@ SETTINGS mutations_sync = 1; SET allow_experimental_projection_optimization = 1; ``` -### Запрос 1. Средняя цена за год +### Запрос 1. Средняя цена за год {#average-price-projections} Запрос: @@ -443,7 +443,7 @@ ORDER BY year ASC; └──────┴────────┴────────────────────────────────────────┘ ``` -### Запрос 2. Средняя цена за год в Лондоне +### Запрос 2. Средняя цена за год в Лондоне {#average-price-london-projections} Запрос: @@ -492,7 +492,7 @@ ORDER BY year ASC; └──────┴─────────┴───────────────────────────────────────────────────────┘ ``` -### Запрос 3. Самые дорогие районы +### Запрос 3. Самые дорогие районы {#most-expensive-neighborhoods-projections} Условие (date >= '2020-01-01') необходимо изменить, чтобы оно соответствовало проекции (toYear(date) >= 2020). @@ -622,7 +622,7 @@ LIMIT 100; └──────────────────────┴────────────────────────┴──────┴─────────┴────────────────────────────────────────────────────────────────────┘ ``` -### Резюме +### Резюме {#summary} Все три запроса работают намного быстрее и читают меньшее количество строк. @@ -644,6 +644,6 @@ no projection: 100 rows in set. Elapsed: 0.069 sec. Processed 26.32 million rows projection: 100 rows in set. Elapsed: 0.029 sec. Processed 8.08 thousand rows, 511.08 KB (276.06 thousand rows/s., 17.47 MB/s.) ``` -### Online Playground +### Online Playground {#playground} Этот набор данных доступен в [Online Playground](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). From 09d7d34d503558d57374fb7eef4c4f83cbf78e01 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 16 Sep 2021 16:25:48 +0300 Subject: [PATCH 056/142] Fix PR comments. --- docs/en/getting-started/example-datasets/opensky.md | 2 +- docs/en/getting-started/example-datasets/uk-price-paid.md | 8 ++++---- docs/ru/getting-started/example-datasets/opensky.md | 4 ++-- docs/ru/getting-started/example-datasets/uk-price-paid.md | 8 ++++---- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md index 261c101fab8..05e10ee50bd 100644 --- a/docs/en/getting-started/example-datasets/opensky.md +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -60,7 +60,7 @@ ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c 'gzip -c -d "{}" | clickhou `xargs -P100` specifies to use up to 100 parallel workers but as we only have 30 files, the number of workers will be only 30. - For every file, `xargs` will run a script with `bash -c`. The script has substitution in form of `{}` and the `xargs` command will substitute the filename to it (we have asked it for `xargs` with `-I{}`). - The script will decompress the file (`gzip -c -d "{}"`) to standard output (`-c` parameter) and the output is redirected to `clickhouse-client`. -- We also asked to parse `DateTime` fields with extended parser (`--date_time_input_format best_effort`) to recognize ISO-8601 format with timezone offsets. +- We also asked to parse [DateTime](../../sql-reference/data-types/datetime.md) fields with extended parser ([--date_time_input_format best_effort](../../operations/settings/settings.md#settings-date_time_input_format)) to recognize ISO-8601 format with timezone offsets. Finally, `clickhouse-client` will do insertion. It will read input data in [CSVWithNames](../../interfaces/formats.md#csvwithnames) format. diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 74fa7dddb3b..61f31a0cfb1 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -55,9 +55,9 @@ In this example, we define the structure of source data from the CSV file and sp The preprocessing is: - splitting the postcode to two different columns `postcode1` and `postcode2` that is better for storage and queries; - coverting the `time` field to date as it only contains 00:00 time; -- ignoring the `uuid` field because we don't need it for analysis; -- transforming `type` and `duration` to more readable Enum fields with function `transform`; -- transforming `is_new` and `category` fields from single-character string (`Y`/`N` and `A`/`B`) to UInt8 field with 0 and 1. +- ignoring the [UUid](../../sql-reference/data-types/uuid.md) field because we don't need it for analysis; +- transforming `type` and `duration` to more readable Enum fields with function [transform](../../sql-reference/functions/other-functions.md#transform); +- transforming `is_new` and `category` fields from single-character string (`Y`/`N` and `A`/`B`) to [UInt8](../../sql-reference/data-types/int-uint.md##uint8-uint16-uint32-uint64-uint256-int8-int16-int32-int64-int128-int256) field with 0 and 1. Preprocessed data is piped directly to `clickhouse-client` to be inserted into ClickHouse table in streaming fashion. @@ -389,7 +389,7 @@ SETTINGS mutations_sync = 1; Let's run the same 3 queries. -Enable projections for selects: +[Enable](../../operations/settings/settings.md#allow-experimental-projection-optimization) projections for selects: ```sql SET allow_experimental_projection_optimization = 1; diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index 59d366da3c3..4000173aa79 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -60,7 +60,7 @@ ls -1 flightlist_*.csv.gz | xargs -P100 -I{} bash -c 'gzip -c -d "{}" | clickhou - `xargs -P100` указывает на возможность использования до 100 параллельных обработчиков, но поскольку у нас всего 30 файлов, то количество обработчиков будет всего 30. - Для каждого файла `xargs` будет запускать скрипт с `bash -c`. Сценарий имеет подстановку в виде `{}`, а команда `xargs` заменяет имя файла на указанные в подстановке символы (мы указали это для `xargs` с помощью `-I{}`). - Скрипт распакует файл (`gzip -c -d "{}"`) в стандартный вывод (параметр `-c`) и перенаправит его в `clickhouse-client`. -- Чтобы распознать формат ISO-8601 со смещениями часовых поясов в полях типа `DateTime`, указывается параметр парсера `--date_time_input_format best_effort`. +- Чтобы распознать формат ISO-8601 со смещениями часовых поясов в полях типа [DateTime](../../sql-reference/data-types/datetime.md), указывается параметр парсера [--date_time_input_format best_effort](../../operations/settings/settings.md#settings-date_time_input_format). В итоге: клиент clickhouse добавит данные в таблицу `opensky`. Входные данные импортируются в формате [CSVWithNames](../../interfaces/formats.md#csvwithnames). @@ -89,7 +89,7 @@ SELECT count() FROM opensky; └──────────┘ ``` -Убедитесь, что размер набора данных в ClickHouse составляет всего 2,66 ГБ. +Убедитесь, что размер набора данных в ClickHouse составляет всего 2,66 GiB. Запрос: diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index c3eae1cdff4..a08dc35fa2a 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -55,9 +55,9 @@ CREATE TABLE uk_price_paid Предварительная обработка включает: - разделение почтового индекса на два разных столбца `postcode1` и `postcode2`, что лучше подходит для хранения данных и выполнения запросов к ним; - преобразование поля `time` в дату, поскольку оно содержит только время 00:00; -- поле `uuid` игнорируется, потому что оно не будет использовано для анализа; -- преобразование полей `type` и `duration` в более читаемые поля типа `Enum` с помощью функции `transform`; -- преобразование полей `is_new` и `category` из односимвольной строки (`Y`/`N` и `A`/`B`) в поле `UInt8` со значениями 0 и 1 соответственно. +- поле [UUid](../../sql-reference/data-types/uuid.md) игнорируется, потому что оно не будет использовано для анализа; +- преобразование полей `type` и `duration` в более читаемые поля типа `Enum` с помощью функции [transform](../../sql-reference/functions/other-functions.md#transform); +- преобразование полей `is_new` и `category` из односимвольной строки (`Y`/`N` и `A`/`B`) в поле [UInt8](../../sql-reference/data-types/int-uint.md##uint8-uint16-uint32-uint64-uint256-int8-int16-int32-int64-int128-int256) со значениями 0 и 1 соответственно. Обработанные данные передаются в `clickhouse-client` и импортируются в таблицу ClickHouse потоковым способом. @@ -389,7 +389,7 @@ SETTINGS mutations_sync = 1; Давайте выполним те же 3 запроса. -Включите поддержку проекций: +[Включите](../../operations/settings/settings.md#allow-experimental-projection-optimization) поддержку проекций: ```sql SET allow_experimental_projection_optimization = 1; From f9d6e6a7f52ef1344016d6f8fc993e45a8d50a94 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 16 Sep 2021 16:40:17 +0300 Subject: [PATCH 057/142] Fix index pages. --- .../en/getting-started/example-datasets/index.md | 10 +++++----- .../ru/getting-started/example-datasets/index.md | 16 ++++++++-------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/en/getting-started/example-datasets/index.md b/docs/en/getting-started/example-datasets/index.md index e61c76970a7..3ed74a20ce9 100644 --- a/docs/en/getting-started/example-datasets/index.md +++ b/docs/en/getting-started/example-datasets/index.md @@ -13,16 +13,16 @@ The list of documented datasets: - [GitHub Events](../../getting-started/example-datasets/github-events.md) - [Anonymized Yandex.Metrica Dataset](../../getting-started/example-datasets/metrica.md) - [Recipes](../../getting-started/example-datasets/recipes.md) -- [OnTime](../../getting-started/example-datasets/ontime.md) -- [OpenSky](../../getting-started/example-datasets/opensky.md) -- [New York Taxi Data](../../getting-started/example-datasets/nyc-taxi.md) -- [UK Property Price Paid](../../getting-started/example-datasets/uk-price-paid.md) -- [What's on the Menu?](../../getting-started/example-datasets/menus.md) - [Star Schema Benchmark](../../getting-started/example-datasets/star-schema.md) - [WikiStat](../../getting-started/example-datasets/wikistat.md) - [Terabyte of Click Logs from Criteo](../../getting-started/example-datasets/criteo.md) - [AMPLab Big Data Benchmark](../../getting-started/example-datasets/amplab-benchmark.md) - [Brown University Benchmark](../../getting-started/example-datasets/brown-benchmark.md) +- [New York Taxi Data](../../getting-started/example-datasets/nyc-taxi.md) +- [OpenSky](../../getting-started/example-datasets/opensky.md) +- [UK Property Price Paid](../../getting-started/example-datasets/uk-price-paid.md) - [Cell Towers](../../getting-started/example-datasets/cell-towers.md) +- [What's on the Menu?](../../getting-started/example-datasets/menus.md) +- [OnTime](../../getting-started/example-datasets/ontime.md) [Original article](https://clickhouse.tech/docs/en/getting_started/example_datasets) diff --git a/docs/ru/getting-started/example-datasets/index.md b/docs/ru/getting-started/example-datasets/index.md index 00d14a6cb5a..2049ddd5d86 100644 --- a/docs/ru/getting-started/example-datasets/index.md +++ b/docs/ru/getting-started/example-datasets/index.md @@ -9,16 +9,16 @@ toc_title: "Введение" Этот раздел описывает как получить тестовые массивы данных и загрузить их в ClickHouse. Для некоторых тестовых массивов данных также доступны тестовые запросы. -- [Анонимизированные данные Яндекс.Метрики](metrica.md) -- [Star Schema Benchmark](star-schema.md) -- [Набор данных кулинарных рецептов](recipes.md) -- [WikiStat](wikistat.md) -- [Терабайт логов кликов от Criteo](criteo.md) -- [AMPLab Big Data Benchmark](amplab-benchmark.md) -- [Данные о такси в Нью-Йорке](nyc-taxi.md) +- [Анонимизированные данные Яндекс.Метрики](../../getting-started/example-datasets/metrica.md) +- [Star Schema Benchmark](../../getting-started/example-datasets/star-schema.md) +- [Набор данных кулинарных рецептов](../../getting-started/example-datasets/recipes.md) +- [WikiStat](../../getting-started/example-datasets/wikistat.md) +- [Терабайт логов кликов от Criteo](../../getting-started/example-datasets/criteo.md) +- [AMPLab Big Data Benchmark](../../getting-started/example-datasets/amplab-benchmark.md) +- [Данные о такси в Нью-Йорке](../../getting-started/example-datasets/nyc-taxi.md) - [Набор данных о воздушном движении OpenSky Network 2020](../../getting-started/example-datasets/opensky.md) - [Данные о стоимости недвижимости в Великобритании](../../getting-started/example-datasets/uk-price-paid.md) -- [OnTime](ontime.md) +- [OnTime](../../getting-started/example-datasets/ontime.md) - [Вышки сотовой связи](../../getting-started/example-datasets/cell-towers.md) [Оригинальная статья](https://clickhouse.tech/docs/ru/getting_started/example_datasets) From b1f18eff945e2eeb3b8f3a0dcd3f9094c07ef900 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 16 Sep 2021 18:35:25 +0300 Subject: [PATCH 058/142] Try fix doc check. --- docs/ru/getting-started/example-datasets/uk-price-paid.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index a08dc35fa2a..1e5acde6ee3 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -647,3 +647,4 @@ no projection: 100 rows in set. Elapsed: 0.069 sec. Processed 26.32 million rows ### Online Playground {#playground} Этот набор данных доступен в [Online Playground](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). + From 341553febd94c252681354d8b4831c9b35cfb034 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 16 Sep 2021 20:40:42 +0300 Subject: [PATCH 059/142] Fix build. --- programs/copier/ClusterCopier.cpp | 36 ++++--- programs/copier/Internals.cpp | 18 +++- programs/copier/Internals.h | 5 +- src/DataStreams/BlockIO.cpp | 2 - src/DataStreams/ShellCommandSource.h | 4 +- src/DataStreams/TemporaryFileStream.h | 3 +- src/Databases/MySQL/DatabaseMySQL.cpp | 3 +- .../MySQL/FetchTablesColumnsList.cpp | 3 +- src/Databases/MySQL/MaterializeMetadata.cpp | 18 ++-- .../MySQL/MaterializedMySQLSyncThread.cpp | 48 ++++------ src/Dictionaries/CacheDictionary.cpp | 6 +- .../ClickHouseDictionarySource.cpp | 22 ++--- src/Dictionaries/DictionaryHelpers.h | 3 +- src/Dictionaries/DirectDictionary.cpp | 7 +- src/Dictionaries/FlatDictionary.cpp | 6 +- src/Dictionaries/HashedDictionary.cpp | 9 +- src/Dictionaries/IPAddressDictionary.cpp | 3 +- src/Dictionaries/MySQLDictionarySource.cpp | 3 +- src/Dictionaries/PolygonDictionary.cpp | 3 +- .../PostgreSQLDictionarySource.cpp | 3 +- src/Dictionaries/RangeHashedDictionary.cpp | 3 +- src/Dictionaries/XDBCDictionarySource.cpp | 3 +- src/Dictionaries/readInvalidateQuery.cpp | 5 +- src/Dictionaries/readInvalidateQuery.h | 4 +- src/Interpreters/GlobalSubqueriesVisitor.h | 13 +-- .../IInterpreterUnionOrSelectQuery.cpp | 2 +- .../InterpreterCreateFunctionQuery.cpp | 1 + src/Interpreters/InterpreterDescribeQuery.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Interpreters/InterpreterWatchQuery.cpp | 15 ++- src/Interpreters/InterpreterWatchQuery.h | 1 + src/Interpreters/MergeJoin.cpp | 9 +- src/Interpreters/MutationsInterpreter.cpp | 18 ++-- src/Interpreters/ProcessList.cpp | 54 +---------- src/Interpreters/ProcessList.h | 11 +-- src/Interpreters/SortedBlocksWriter.cpp | 16 ++-- src/Interpreters/UserDefinedObjectsLoader.cpp | 1 + src/Interpreters/executeDDLQueryOnCluster.cpp | 7 +- src/Interpreters/executeQuery.h | 3 +- .../PipelineExecutingBlockInputStream.cpp | 4 +- .../PipelineExecutingBlockInputStream.h | 6 +- src/Processors/QueryPipeline.cpp | 83 ++++++++++++---- src/Processors/QueryPipeline.h | 3 + src/Processors/QueryPipelineBuilder.cpp | 16 ++++ src/Processors/QueryPipelineBuilder.h | 1 + .../Transforms/CreatingSetsTransform.cpp | 6 +- .../Transforms/CreatingSetsTransform.h | 3 +- src/Server/GRPCServer.cpp | 96 ++----------------- src/Server/PostgreSQLHandler.cpp | 1 + src/Server/TCPHandler.cpp | 17 ++-- src/Storages/Distributed/DirectoryMonitor.cpp | 2 +- src/Storages/Distributed/DirectoryMonitor.h | 4 +- src/Storages/Distributed/DistributedSink.cpp | 10 +- src/Storages/Distributed/DistributedSink.h | 4 +- src/Storages/LiveView/StorageLiveView.cpp | 27 +++--- .../MergeTree/MergeTreeDataMergerMutator.cpp | 13 ++- src/Storages/ProjectionsDescription.cpp | 21 ++-- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageDistributed.cpp | 3 +- src/Storages/StorageExecutable.cpp | 65 ++++++++----- src/Storages/StorageFile.cpp | 13 +-- src/Storages/StorageMerge.cpp | 4 +- src/Storages/StorageS3.cpp | 7 +- src/Storages/StorageS3.h | 2 +- src/Storages/StorageURL.cpp | 9 +- .../System/StorageSystemZooKeeper.cpp | 11 ++- src/TableFunctions/ITableFunctionFileLike.cpp | 6 +- 67 files changed, 396 insertions(+), 418 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index bdca1e097c7..64a6c5cea20 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include namespace DB @@ -1449,7 +1450,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( local_context->setSettings(task_cluster->settings_pull); local_context->setSetting("skip_unavailable_shards", true); - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().getInputStream()); + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().pipeline); count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0; } @@ -1527,27 +1528,30 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( context_insert->setSettings(task_cluster->settings_push); /// Custom INSERT SELECT implementation - QueryPipelineBuilder input; - Chain output; + QueryPipeline input; + QueryPipeline output; { BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); - output = std::move(io_insert.out); + output = std::move(io_insert.pipeline); /// Add converting actions to make it possible to copy blocks with slightly different schema const auto & select_block = io_select.pipeline.getHeader(); - const auto & insert_block = output.getInputHeader(); + const auto & insert_block = output.getHeader(); auto actions_dag = ActionsDAG::makeConvertingActions( select_block.getColumnsWithTypeAndName(), insert_block.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Position); auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext())); - input.addSimpleTransform([&](const Block & header) + QueryPipelineBuilder builder; + builder.init(std::move(io_select.pipeline)); + builder.addSimpleTransform([&](const Block & header) { return std::make_shared(header, actions); }); + input = QueryPipelineBuilder::getPipeline(std::move(builder)); } /// Fail-fast optimization to abort copying when the current clean state expires @@ -1735,7 +1739,8 @@ String ClusterCopier::getRemoteCreateTable( String query = "SHOW CREATE TABLE " + getQuotedTable(table); Block block = getBlockWithAllStreamData( - std::make_shared(connection, query, InterpreterShowCreateQuery::getSampleBlock(), remote_context)); + QueryPipeline(std::make_shared( + std::make_shared(connection, query, InterpreterShowCreateQuery::getSampleBlock(), remote_context), false, false))); return typeid_cast(*block.safeGetByPosition(0).column).getDataAt(0).toString(); } @@ -1848,7 +1853,7 @@ std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti auto local_context = Context::createCopy(context); local_context->setSettings(task_cluster->settings_pull); - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()); + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().pipeline); if (block) { @@ -1893,7 +1898,11 @@ bool ClusterCopier::checkShardHasPartition(const ConnectionTimeouts & timeouts, auto local_context = Context::createCopy(context); local_context->setSettings(task_cluster->settings_pull); - return InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()->read().rows() != 0; + auto pipeline = InterpreterFactory::get(query_ast, local_context)->execute().pipeline; + PullingPipelineExecutor executor(pipeline); + Block block; + executor.pull(block); + return block.rows() != 0; } bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, @@ -1932,12 +1941,15 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi auto local_context = Context::createCopy(context); local_context->setSettings(task_cluster->settings_pull); - auto result = InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()->read().rows(); - if (result != 0) + auto pipeline = InterpreterFactory::get(query_ast, local_context)->execute().pipeline; + PullingPipelineExecutor executor(pipeline); + Block result; + executor.pull(result); + if (result.rows() != 0) LOG_INFO(log, "Partition {} piece number {} is PRESENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription()); else LOG_INFO(log, "Partition {} piece number {} is ABSENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription()); - return result != 0; + return result.rows() != 0; } diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index bec612a8226..c5e702cd1dc 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -1,6 +1,8 @@ #include "Internals.h" #include #include +#include +#include namespace DB { @@ -63,9 +65,21 @@ BlockInputStreamPtr squashStreamIntoOneBlock(const BlockInputStreamPtr & stream) std::numeric_limits::max()); } -Block getBlockWithAllStreamData(const BlockInputStreamPtr & stream) +Block getBlockWithAllStreamData(QueryPipeline pipeline) { - return squashStreamIntoOneBlock(stream)->read(); + QueryPipelineBuilder builder; + builder.init(std::move(pipeline)); + builder.addTransform(std::make_shared( + builder.getHeader(), + std::numeric_limits::max(), + std::numeric_limits::max())); + + auto cur_pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); + Block block; + PullingPipelineExecutor executor(cur_pipeline); + executor.pull(block); + + return block; } diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h index 9e40d7ebd7b..8a3e676baf5 100644 --- a/programs/copier/Internals.h +++ b/programs/copier/Internals.h @@ -165,10 +165,7 @@ std::shared_ptr createASTStorageDistributed( const String & cluster_name, const String & database, const String & table, const ASTPtr & sharding_key_ast = nullptr); - -BlockInputStreamPtr squashStreamIntoOneBlock(const BlockInputStreamPtr & stream); - -Block getBlockWithAllStreamData(const BlockInputStreamPtr & stream); +Block getBlockWithAllStreamData(QueryPipeline pipeline); bool isExtendedDefinitionStorage(const ASTPtr & storage_ast); diff --git a/src/DataStreams/BlockIO.cpp b/src/DataStreams/BlockIO.cpp index 0c3ad92ff18..8a6d0f19cc5 100644 --- a/src/DataStreams/BlockIO.cpp +++ b/src/DataStreams/BlockIO.cpp @@ -22,8 +22,6 @@ void BlockIO::reset() */ /// TODO simplify it all - if (process_list_entry) - process_list_entry->get().releaseQueryStreams(); pipeline.reset(); process_list_entry.reset(); diff --git a/src/DataStreams/ShellCommandSource.h b/src/DataStreams/ShellCommandSource.h index 752d3209fb2..3f8f648f18a 100644 --- a/src/DataStreams/ShellCommandSource.h +++ b/src/DataStreams/ShellCommandSource.h @@ -97,7 +97,7 @@ public: max_block_size = configuration.number_of_rows_to_read; } - pipeline.init(Pipe(FormatFactory::instance().getInput(format, command->out, sample_block, context, max_block_size))); + pipeline = QueryPipeline(Pipe(FormatFactory::instance().getInput(format, command->out, sample_block, context, max_block_size))); executor = std::make_unique(pipeline); } @@ -175,7 +175,7 @@ private: std::shared_ptr process_pool; - QueryPipelineBuilder pipeline; + QueryPipeline pipeline; std::unique_ptr executor; std::vector send_data_threads; diff --git a/src/DataStreams/TemporaryFileStream.h b/src/DataStreams/TemporaryFileStream.h index 74e00477789..ee7660db717 100644 --- a/src/DataStreams/TemporaryFileStream.h +++ b/src/DataStreams/TemporaryFileStream.h @@ -35,12 +35,13 @@ struct TemporaryFileStream {} /// Flush data from input stream into file for future reading - static void write(const std::string & path, const Block & header, QueryPipelineBuilder pipeline, const std::string & codec) + static void write(const std::string & path, const Block & header, QueryPipelineBuilder builder, const std::string & codec) { WriteBufferFromFile file_buf(path); CompressedWriteBuffer compressed_buf(file_buf, CompressionCodecFactory::instance().get(codec, {})); NativeBlockOutputStream output(compressed_buf, 0, header); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); output.writePrefix(); diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index c02d4acc7ae..72c505bde94 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -284,8 +284,7 @@ std::map DatabaseMySQL::fetchTablesWithModificationTime(ContextP std::map tables_with_modification_time; StreamSettings mysql_input_stream_settings(local_context->getSettingsRef()); auto result = std::make_unique(mysql_pool.get(), query.str(), tables_status_sample_block, mysql_input_stream_settings); - QueryPipelineBuilder pipeline; - pipeline.init(Pipe(std::move(result))); + QueryPipeline pipeline(std::move(result)); Block block; PullingPipelineExecutor executor(pipeline); diff --git a/src/Databases/MySQL/FetchTablesColumnsList.cpp b/src/Databases/MySQL/FetchTablesColumnsList.cpp index 8c338dbcbcb..618f6bf6d34 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.cpp +++ b/src/Databases/MySQL/FetchTablesColumnsList.cpp @@ -88,8 +88,7 @@ std::map fetchTablesColumnsList( StreamSettings mysql_input_stream_settings(settings); auto result = std::make_unique(pool.get(), query.str(), tables_columns_sample_block, mysql_input_stream_settings); - QueryPipelineBuilder pipeline; - pipeline.init(Pipe(std::move(result))); + QueryPipeline pipeline(std::move(result)); Block block; PullingPipelineExecutor executor(pipeline); diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index ed7a0fa4622..f55cddfab59 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -44,8 +44,7 @@ static std::unordered_map fetchTablesCreateQuery( connection, "SHOW CREATE TABLE " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(fetch_table_name), show_create_table_header, mysql_input_stream_settings); - QueryPipelineBuilder pipeline; - pipeline.init(Pipe(std::move(show_create_table))); + QueryPipeline pipeline(std::move(show_create_table)); Block create_query_block; PullingPipelineExecutor executor(pipeline); @@ -69,8 +68,7 @@ static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entr StreamSettings mysql_input_stream_settings(global_settings); auto input = std::make_unique(connection, query, header, mysql_input_stream_settings); - QueryPipelineBuilder pipeline; - pipeline.init(Pipe(std::move(input))); + QueryPipeline pipeline(std::move(input)); Block block; PullingPipelineExecutor executor(pipeline); @@ -97,8 +95,7 @@ void MaterializeMetadata::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & c StreamSettings mysql_input_stream_settings(settings, false, true); auto input = std::make_unique(connection, "SHOW MASTER STATUS;", header, mysql_input_stream_settings); - QueryPipelineBuilder pipeline; - pipeline.init(Pipe(std::move(input))); + QueryPipeline pipeline(std::move(input)); Block master_status; PullingPipelineExecutor executor(pipeline); @@ -125,8 +122,7 @@ void MaterializeMetadata::fetchMasterVariablesValue(const mysqlxx::PoolWithFailo const String & fetch_query = "SHOW VARIABLES WHERE Variable_name = 'binlog_checksum'"; StreamSettings mysql_input_stream_settings(settings, false, true); auto variables_input = std::make_unique(connection, fetch_query, variables_header, mysql_input_stream_settings); - QueryPipelineBuilder pipeline; - pipeline.init(Pipe(std::move(variables_input))); + QueryPipeline pipeline(std::move(variables_input)); Block variables_block; PullingPipelineExecutor executor(pipeline); @@ -153,8 +149,7 @@ static bool checkSyncUserPrivImpl(const mysqlxx::PoolWithFailover::Entry & conne String grants_query, sub_privs; StreamSettings mysql_input_stream_settings(global_settings); auto input = std::make_unique(connection, "SHOW GRANTS FOR CURRENT_USER();", sync_user_privs_header, mysql_input_stream_settings); - QueryPipelineBuilder pipeline; - pipeline.init(Pipe(std::move(input))); + QueryPipeline pipeline(std::move(input)); Block block; PullingPipelineExecutor executor(pipeline); @@ -204,8 +199,7 @@ bool MaterializeMetadata::checkBinlogFileExists(const mysqlxx::PoolWithFailover: StreamSettings mysql_input_stream_settings(settings, false, true); auto input = std::make_unique(connection, "SHOW MASTER LOGS", logs_header, mysql_input_stream_settings); - QueryPipelineBuilder pipeline; - pipeline.init(Pipe(std::move(input))); + QueryPipeline pipeline(std::move(input)); Block block; PullingPipelineExecutor executor(pipeline); diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index dc4bb3862ff..ecb212240ed 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -11,6 +11,7 @@ # include # include # include +# include # include # include # include @@ -114,8 +115,7 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S {"log_bin_use_v1_row_events", "OFF"} }; - QueryPipelineBuilder pipeline; - pipeline.init(Pipe(std::move(variables_input))); + QueryPipeline pipeline(std::move(variables_input)); PullingPipelineExecutor executor(pipeline); Block variables_block; @@ -290,7 +290,7 @@ static inline void cleanOutdatedTables(const String & database_name, ContextPtr } } -static inline Chain +static inline QueryPipeline getTableOutput(const String & database_name, const String & table_name, ContextMutablePtr query_context, bool insert_materialized = false) { const StoragePtr & storage = DatabaseCatalog::instance().getTable(StorageID(database_name, table_name), query_context); @@ -314,10 +314,7 @@ getTableOutput(const String & database_name, const String & table_name, ContextM BlockIO res = tryToExecuteQuery("INSERT INTO " + backQuoteIfNeed(table_name) + "(" + insert_columns_str.str() + ")" + " VALUES", query_context, database_name, comment); - if (res.out.empty()) - throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR); - - return std::move(res.out); + return std::move(res.pipeline); } static inline void dumpDataForTables( @@ -335,26 +332,19 @@ static inline void dumpDataForTables( String comment = "Materialize MySQL step 1: execute MySQL DDL for dump data"; tryToExecuteQuery(query_prefix + " " + iterator->second, query_context, database_name, comment); /// create table. - auto chain = getTableOutput(database_name, table_name, query_context); - auto counting = std::make_shared(chain.getInputHeader()); - chain.addSource(counting); + auto pipeline = getTableOutput(database_name, table_name, query_context); StreamSettings mysql_input_stream_settings(context->getSettingsRef()); auto input = std::make_unique( connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), - chain.getInputHeader(), mysql_input_stream_settings); - - QueryPipelineBuilder pipeline; - pipeline.init(Pipe(std::move(input))); - pipeline.addChain(std::move(chain)); - pipeline.setSinks([&](const Block & header, Pipe::StreamType) - { - return std::make_shared(header); - }); - - auto executor = pipeline.execute(); + pipeline.getHeader(), mysql_input_stream_settings); + auto counting = std::make_shared(pipeline.getHeader()); + Pipe pipe(std::move(input)); + pipe.addTransform(std::move(counting)); + pipeline.complete(std::move(pipe)); Stopwatch watch; - executor->execute(1); + CompletedPipelineExecutor executor(pipeline); + executor.execute(); const Progress & progress = counting->getProgress(); LOG_INFO(&Poco::Logger::get("MaterializedMySQLSyncThread(" + database_name + ")"), @@ -807,17 +797,11 @@ void MaterializedMySQLSyncThread::Buffers::commit(ContextPtr context) { auto query_context = createQueryContext(context); auto input = std::make_shared(table_name_and_buffer.second->first); - auto out = getTableOutput(database, table_name_and_buffer.first, query_context, true); - QueryPipelineBuilder pipeline; - pipeline.init(Pipe(std::move(input))); - pipeline.addChain(std::move(out)); - pipeline.setSinks([&](const Block & header, Pipe::StreamType) - { - return std::make_shared(header); - }); + auto pipeline = getTableOutput(database, table_name_and_buffer.first, query_context, true); + pipeline.complete(Pipe(std::move(input))); - auto executor = pipeline.execute(); - executor->execute(1); + CompletedPipelineExecutor executor(pipeline); + executor.execute(); } data.clear(); diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 4dbd8461efa..bbf596ba55a 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -570,12 +570,12 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtrloadIds(requested_keys_vector)); + pipeline = QueryPipeline(current_source_ptr->loadIds(requested_keys_vector)); else - pipeline.init(current_source_ptr->loadKeys(update_unit_ptr->key_columns, requested_complex_key_rows)); + pipeline = QueryPipeline(current_source_ptr->loadKeys(update_unit_ptr->key_columns, requested_complex_key_rows)); size_t skip_keys_size_offset = dict_struct.getKeysSize(); PaddedPODArray found_keys_in_source; diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 22ed68cea1f..806511248a3 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -162,39 +163,39 @@ std::string ClickHouseDictionarySource::toString() const Pipe ClickHouseDictionarySource::createStreamForQuery(const String & query, std::atomic * result_size_hint) { - QueryPipelineBuilder pipeline; + QueryPipelineBuilder builder; /// Sample block should not contain first row default values auto empty_sample_block = sample_block.cloneEmpty(); if (configuration.is_local) { - pipeline = executeQuery(query, context, true).pipeline; + builder.init(executeQuery(query, context, true).pipeline); auto converting = ActionsDAG::makeConvertingActions( - pipeline.getHeader().getColumnsWithTypeAndName(), + builder.getHeader().getColumnsWithTypeAndName(), empty_sample_block.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Position); - pipeline.addSimpleTransform([&](const Block & header) + builder.addSimpleTransform([&](const Block & header) { return std::make_shared(header, std::make_shared(converting)); }); } else { - pipeline.init(Pipe(std::make_shared( + builder.init(Pipe(std::make_shared( std::make_shared(pool, query, empty_sample_block, context), false, false))); } if (result_size_hint) { - pipeline.setProgressCallback([result_size_hint](const Progress & progress) + builder.setProgressCallback([result_size_hint](const Progress & progress) { *result_size_hint += progress.total_rows_to_read; }); } - return QueryPipelineBuilder::getPipe(std::move(pipeline)); + return QueryPipelineBuilder::getPipe(std::move(builder)); } std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & request) const @@ -203,16 +204,15 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re if (configuration.is_local) { auto query_context = Context::createCopy(context); - auto pipe = QueryPipelineBuilder::getPipe(executeQuery(request, query_context, true).pipeline); - return readInvalidateQuery(std::move(pipe)); + return readInvalidateQuery(executeQuery(request, query_context, true).pipeline); } else { /// We pass empty block to RemoteBlockInputStream, because we don't know the structure of the result. Block invalidate_sample_block; - Pipe pipe(std::make_shared( + QueryPipeline pipeline(std::make_shared( std::make_shared(pool, request, invalidate_sample_block, context), false, false)); - return readInvalidateQuery(std::move(pipe)); + return readInvalidateQuery(std::move(pipeline)); } } diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 1609c1d6080..6266bd2cf4f 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -567,8 +567,7 @@ void mergeBlockWithPipe( auto result_fetched_columns = block_to_update.cloneEmptyColumns(); - QueryPipelineBuilder pipeline; - pipeline.init(std::move(pipe)); + QueryPipeline pipeline(std::move(pipe)); PullingPipelineExecutor executor(pipeline); Block block; diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 2630146fa3e..03d3b579ec3 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -68,8 +68,7 @@ Columns DirectDictionary::getColumns( size_t dictionary_keys_size = dict_struct.getKeysNames().size(); block_key_columns.reserve(dictionary_keys_size); - QueryPipelineBuilder pipeline; - pipeline.init(getSourceBlockInputStream(key_columns, requested_keys)); + QueryPipeline pipeline(getSourceBlockInputStream(key_columns, requested_keys)); PullingPipelineExecutor executor(pipeline); @@ -185,9 +184,7 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys( size_t dictionary_keys_size = dict_struct.getKeysNames().size(); block_key_columns.reserve(dictionary_keys_size); - QueryPipelineBuilder pipeline; - pipeline.init(getSourceBlockInputStream(key_columns, requested_keys)); - + QueryPipeline pipeline(getSourceBlockInputStream(key_columns, requested_keys)); PullingPipelineExecutor executor(pipeline); size_t keys_found = 0; diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 48022a0b239..e14ee5d30d1 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -322,8 +322,7 @@ void FlatDictionary::updateData() { if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) { - QueryPipelineBuilder pipeline; - pipeline.init(source_ptr->loadUpdatedAll()); + QueryPipeline pipeline(source_ptr->loadUpdatedAll()); PullingPipelineExecutor executor(pipeline); Block block; @@ -358,8 +357,7 @@ void FlatDictionary::loadData() { if (!source_ptr->hasUpdateField()) { - QueryPipelineBuilder pipeline; - pipeline.init(source_ptr->loadAll()); + QueryPipeline pipeline(source_ptr->loadAll()); PullingPipelineExecutor executor(pipeline); Block block; diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 712333fb38d..1f3821096da 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -367,8 +367,7 @@ void HashedDictionary::updateData() if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) { - QueryPipelineBuilder pipeline; - pipeline.init(source_ptr->loadUpdatedAll()); + QueryPipeline pipeline(source_ptr->loadUpdatedAll()); PullingPipelineExecutor executor(pipeline); Block block; @@ -561,11 +560,11 @@ void HashedDictionary::loadData() { std::atomic new_size = 0; - QueryPipelineBuilder pipeline; + QueryPipeline pipeline; if (configuration.preallocate) - pipeline.init(source_ptr->loadAllWithSizeHint(&new_size)); + pipeline = QueryPipeline(source_ptr->loadAllWithSizeHint(&new_size)); else - pipeline.init(source_ptr->loadAll()); + pipeline = QueryPipeline(source_ptr->loadAll()); PullingPipelineExecutor executor(pipeline); Block block; diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 25c1eba0a7a..6514ecd9e3e 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -352,8 +352,7 @@ void IPAddressDictionary::createAttributes() void IPAddressDictionary::loadData() { - QueryPipelineBuilder pipeline; - pipeline.init(source_ptr->loadAll()); + QueryPipeline pipeline(source_ptr->loadAll()); std::vector ip_records; diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index bd53c1e60a7..e564d3a5b6c 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB { @@ -294,7 +295,7 @@ std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request Block invalidate_sample_block; ColumnPtr column(ColumnString::create()); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - return readInvalidateQuery(Pipe(std::make_unique(pool->get(), request, invalidate_sample_block, settings))); + return readInvalidateQuery(QueryPipeline(std::make_unique(pool->get(), request, invalidate_sample_block, settings))); } } diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index 504319ce688..06a705a351e 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -165,8 +165,7 @@ void IPolygonDictionary::blockToAttributes(const DB::Block & block) void IPolygonDictionary::loadData() { - QueryPipelineBuilder pipeline; - pipeline.init(source_ptr->loadAll()); + QueryPipeline pipeline(source_ptr->loadAll()); PullingPipelineExecutor executor(pipeline); Block block; diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 3fe9e899cd9..ee6f9d5c460 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -10,6 +10,7 @@ #include #include "readInvalidateQuery.h" #include +#include #endif @@ -129,7 +130,7 @@ std::string PostgreSQLDictionarySource::doInvalidateQuery(const std::string & re Block invalidate_sample_block; ColumnPtr column(ColumnString::create()); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - return readInvalidateQuery(Pipe(std::make_unique>(pool->get(), request, invalidate_sample_block, 1))); + return readInvalidateQuery(QueryPipeline(std::make_unique>(pool->get(), request, invalidate_sample_block, 1))); } diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index 5ef2397fc5a..86dfaa615ed 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -303,8 +303,7 @@ void RangeHashedDictionary::createAttributes() template void RangeHashedDictionary::loadData() { - QueryPipelineBuilder pipeline; - pipeline.init(source_ptr->loadAll()); + QueryPipeline pipeline(source_ptr->loadAll()); PullingPipelineExecutor executor(pipeline); Block block; diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 9fc7e92634b..0835aaa2f2e 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -18,6 +18,7 @@ #include "readInvalidateQuery.h" #include "registerDictionaries.h" #include +#include namespace DB @@ -206,7 +207,7 @@ std::string XDBCDictionarySource::doInvalidateQuery(const std::string & request) for (const auto & [name, value] : url_params) invalidate_url.addQueryParameter(name, value); - return readInvalidateQuery(loadFromQuery(invalidate_url, invalidate_sample_block, request)); + return readInvalidateQuery(QueryPipeline(loadFromQuery(invalidate_url, invalidate_sample_block, request))); } diff --git a/src/Dictionaries/readInvalidateQuery.cpp b/src/Dictionaries/readInvalidateQuery.cpp index 4623c43c5fa..c2c42eece58 100644 --- a/src/Dictionaries/readInvalidateQuery.cpp +++ b/src/Dictionaries/readInvalidateQuery.cpp @@ -15,11 +15,8 @@ namespace ErrorCodes extern const int RECEIVED_EMPTY_DATA; } -std::string readInvalidateQuery(Pipe pipe) +std::string readInvalidateQuery(QueryPipeline pipeline) { - QueryPipelineBuilder pipeline; - pipeline.init(std::move(pipe)); - PullingPipelineExecutor executor(pipeline); Block block; diff --git a/src/Dictionaries/readInvalidateQuery.h b/src/Dictionaries/readInvalidateQuery.h index 61d5b29dc89..52cd9f54b2b 100644 --- a/src/Dictionaries/readInvalidateQuery.h +++ b/src/Dictionaries/readInvalidateQuery.h @@ -5,9 +5,9 @@ namespace DB { -class Pipe; +class QueryPipeline; /// Using in MySQLDictionarySource and XDBCDictionarySource after processing invalidate_query. -std::string readInvalidateQuery(Pipe pipe); +std::string readInvalidateQuery(QueryPipeline pipeline); } diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index b5edaa0d53f..769f35ab17c 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -15,7 +16,7 @@ #include #include #include -#include +#include #include #include #include @@ -159,13 +160,9 @@ public: auto external_table = external_storage_holder->getTable(); auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext()); auto io = interpreter->execute(); - io.pipeline.addChain(Chain(std::move(table_out))); - io.pipeline.setSinks([&](const Block & header, Pipe::StreamType) -> ProcessorPtr - { - return std::make_shared(header); - }); - auto executor = io.pipeline.execute(); - executor->execute(io.pipeline.getNumStreams()); + io.pipeline.complete(std::move(table_out)); + CompletedPipelineExecutor executor(io.pipeline); + executor.execute(); } else { diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index 62a7f0bdf7e..55c007e2713 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -20,7 +20,7 @@ QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline() buildQueryPlan(query_plan); return std::move(*query_plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)))); + QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context))); } } diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index 01fc60060b3..75f2c147404 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index beaa870f1a2..e43eda11be6 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -23,7 +23,7 @@ namespace DB BlockIO InterpreterDescribeQuery::execute() { BlockIO res; - res.in = executeImpl(); + res.pipeline = executeImpl(); return res; } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 003d29dcc42..e918e373f24 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -356,7 +356,7 @@ BlockIO InterpreterInsertQuery::execute() else if (query.watch) { InterpreterWatchQuery interpreter_watch{ query.watch, getContext() }; - pipeline = interpreter_watch.execute(); + pipeline = interpreter_watch.buildQueryPipeline(); } for (size_t i = 0; i < out_streams_size; i++) diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index c99e25eadb8..bc0aeda56bd 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -31,11 +31,17 @@ namespace ErrorCodes BlockIO InterpreterWatchQuery::execute() +{ + BlockIO res; + res.pipeline = QueryPipelineBuilder::getPipeline(buildQueryPipeline()); + return res; +} + +QueryPipelineBuilder InterpreterWatchQuery::buildQueryPipeline() { if (!getContext()->getSettingsRef().allow_experimental_live_view) throw Exception("Experimental LIVE VIEW feature is not enabled (the setting 'allow_experimental_live_view')", ErrorCodes::SUPPORT_IS_DISABLED); - BlockIO res; const ASTWatchQuery & query = typeid_cast(*query_ptr); auto table_id = getContext()->resolveStorageID(query, Context::ResolveOrdinary); @@ -85,10 +91,9 @@ BlockIO InterpreterWatchQuery::execute() pipe.setQuota(getContext()->getQuota()); } - res.pipeline = QueryPipeline(std::move(pipe)); - - return res; + QueryPipelineBuilder pipeline; + pipeline.init(std::move(pipe)); + return pipeline; } - } diff --git a/src/Interpreters/InterpreterWatchQuery.h b/src/Interpreters/InterpreterWatchQuery.h index 51eb4a00556..2bc7236582a 100644 --- a/src/Interpreters/InterpreterWatchQuery.h +++ b/src/Interpreters/InterpreterWatchQuery.h @@ -31,6 +31,7 @@ public: InterpreterWatchQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {} BlockIO execute() override; + QueryPipelineBuilder buildQueryPipeline(); private: ASTPtr query_ptr; diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index ba4d23f5491..35e0ba66eb9 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -580,13 +580,14 @@ void MergeJoin::mergeInMemoryRightBlocks() Pipe source(std::make_shared(std::move(right_blocks.blocks))); right_blocks.clear(); - QueryPipelineBuilder pipeline; - pipeline.init(std::move(source)); + QueryPipelineBuilder builder; + builder.init(std::move(source)); /// TODO: there should be no split keys by blocks for RIGHT|FULL JOIN - pipeline.addTransform(std::make_shared( - pipeline.getHeader(), right_sort_description, max_rows_in_right_block, 0, 0, 0, 0, nullptr, 0)); + builder.addTransform(std::make_shared( + builder.getHeader(), right_sort_description, max_rows_in_right_block, 0, 0, 0, 0, nullptr, 0)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); auto sorted_input = PipelineExecutingBlockInputStream(std::move(pipeline)); while (Block block = sorted_input.read()) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index d008fec0a85..890c0dff0d0 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -216,12 +217,14 @@ bool isStorageTouchedByMutations( /// For some reason it may copy context and and give it into ExpressionBlockInputStream /// after that we will use context from destroyed stack frame in our stream. InterpreterSelectQuery interpreter(select_query, context_copy, storage, metadata_snapshot, SelectQueryOptions().ignoreLimits()); - BlockInputStreamPtr in = interpreter.execute().getInputStream(); + auto io = interpreter.execute(); + PullingPipelineExecutor executor(io.pipeline); - Block block = in->read(); + Block block; + auto should_continue = executor.pull(block); if (!block.rows()) return false; - else if (block.rows() != 1) + else if (block.rows() != 1 || should_continue) throw Exception("count() expression returned " + toString(block.rows()) + " rows, not 1", ErrorCodes::LOGICAL_ERROR); @@ -936,19 +939,20 @@ BlockInputStreamPtr MutationsInterpreter::execute() QueryPlan plan; select_interpreter->buildQueryPlan(plan); - auto pipeline = addStreamsForLaterStages(stages, plan); + auto builder = addStreamsForLaterStages(stages, plan); /// Sometimes we update just part of columns (for example UPDATE mutation) /// in this case we don't read sorting key, so just we don't check anything. - if (auto sort_desc = getStorageSortDescriptionIfPossible(pipeline->getHeader())) + if (auto sort_desc = getStorageSortDescriptionIfPossible(builder->getHeader())) { - pipeline->addSimpleTransform([&](const Block & header) + builder->addSimpleTransform([&](const Block & header) { return std::make_shared(header, *sort_desc); }); } - BlockInputStreamPtr result_stream = std::make_shared(std::move(*pipeline)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + BlockInputStreamPtr result_stream = std::make_shared(std::move(pipeline)); if (!updated_header) updated_header = std::make_unique(result_stream->getHeader()); diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index ae9e4e851ca..27ecc534763 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -238,9 +238,6 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as ProcessListEntry::~ProcessListEntry() { - /// Destroy all streams to avoid long lock of ProcessList - it->releaseQueryStreams(); - std::lock_guard lock(parent.mutex); String user = it->getClientInfo().current_user; @@ -303,72 +300,31 @@ QueryStatus::~QueryStatus() assert(executors.empty()); } -void QueryStatus::setQueryStreams(const BlockIO & io) +CancellationCode QueryStatus::cancelQuery(bool) { - std::lock_guard lock(query_streams_mutex); - - query_stream_in = io.in; - query_streams_status = QueryStreamsStatus::Initialized; -} - -void QueryStatus::releaseQueryStreams() -{ - BlockInputStreamPtr in; - - { - std::lock_guard lock(query_streams_mutex); - - query_streams_status = QueryStreamsStatus::Released; - in = std::move(query_stream_in); - } - - /// Destroy streams outside the mutex lock -} - -bool QueryStatus::streamsAreReleased() -{ - std::lock_guard lock(query_streams_mutex); - - return query_streams_status == QueryStreamsStatus::Released; -} - -CancellationCode QueryStatus::cancelQuery(bool kill) -{ - /// Streams are destroyed, and ProcessListElement will be deleted from ProcessList soon. We need wait a little bit - if (streamsAreReleased()) + if (is_killed.load()) return CancellationCode::CancelSent; - BlockInputStreamPtr input_stream; SCOPE_EXIT({ - std::lock_guard lock(query_streams_mutex); + std::lock_guard lock(executors_mutex); for (auto * e : executors) e->cancel(); }); - if (tryGetQueryStreams(input_stream)) - { - if (input_stream) - { - input_stream->cancel(kill); - return CancellationCode::CancelSent; - } - return CancellationCode::CancelCannotBeSent; - } - /// Query is not even started is_killed.store(true); return CancellationCode::CancelSent; } void QueryStatus::addPipelineExecutor(PipelineExecutor * e) { - std::lock_guard lock(query_streams_mutex); + std::lock_guard lock(executors_mutex); assert(std::find(executors.begin(), executors.end(), e) == executors.end()); executors.push_back(e); } void QueryStatus::removePipelineExecutor(PipelineExecutor * e) { - std::lock_guard lock(query_streams_mutex); + std::lock_guard lock(executors_mutex); assert(std::find(executors.begin(), executors.end(), e) != executors.end()); std::erase_if(executors, [e](PipelineExecutor * x) { return x == e; }); } diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index 9f9d099ed83..2e300472647 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -103,7 +103,7 @@ protected: /// Be careful using it. For example, queries field of ProcessListForUser could be modified concurrently. const ProcessListForUser * getUserProcessList() const { return user_process_list; } - mutable std::mutex query_streams_mutex; + mutable std::mutex executors_mutex; /// Array of PipelineExecutors to be cancelled when a cancelQuery is received std::vector executors; @@ -167,15 +167,6 @@ public: QueryStatusInfo getInfo(bool get_thread_list = false, bool get_profile_events = false, bool get_settings = false) const; - /// Copies pointers to in/out streams - void setQueryStreams(const BlockIO & io); - - /// Frees in/out streams - void releaseQueryStreams(); - - /// It means that ProcessListEntry still exists, but stream was already destroyed - bool streamsAreReleased(); - CancellationCode cancelQuery(bool kill); bool isKilled() const { return is_killed; } diff --git a/src/Interpreters/SortedBlocksWriter.cpp b/src/Interpreters/SortedBlocksWriter.cpp index c38402f7b72..ebe4aba71ab 100644 --- a/src/Interpreters/SortedBlocksWriter.cpp +++ b/src/Interpreters/SortedBlocksWriter.cpp @@ -24,10 +24,11 @@ std::unique_ptr flushToFile(const String & tmp_path, const Block return tmp_file; } -SortedBlocksWriter::SortedFiles flushToManyFiles(const String & tmp_path, const Block & header, QueryPipelineBuilder pipeline, +SortedBlocksWriter::SortedFiles flushToManyFiles(const String & tmp_path, const Block & header, QueryPipelineBuilder builder, const String & codec, std::function callback = [](const Block &){}) { std::vector> files; + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); Block block; @@ -293,20 +294,21 @@ Block SortedBlocksBuffer::mergeBlocks(Blocks && blocks) const Blocks tmp_blocks; - QueryPipelineBuilder pipeline; - pipeline.init(Pipe::unitePipes(std::move(pipes))); + QueryPipelineBuilder builder; + builder.init(Pipe::unitePipes(std::move(pipes))); - if (pipeline.getNumStreams() > 1) + if (builder.getNumStreams() > 1) { auto transform = std::make_shared( - pipeline.getHeader(), - pipeline.getNumStreams(), + builder.getHeader(), + builder.getNumStreams(), sort_description, num_rows); - pipeline.addTransform(std::move(transform)); + builder.addTransform(std::move(transform)); } + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); Block block; while (executor.pull(block)) diff --git a/src/Interpreters/UserDefinedObjectsLoader.cpp b/src/Interpreters/UserDefinedObjectsLoader.cpp index 5237de594f4..964dd6794f7 100644 --- a/src/Interpreters/UserDefinedObjectsLoader.cpp +++ b/src/Interpreters/UserDefinedObjectsLoader.cpp @@ -21,6 +21,7 @@ #include #include +#include namespace DB diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index 255d10d14cf..f32bf301070 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include @@ -212,11 +213,11 @@ BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & en if (context->getSettingsRef().distributed_ddl_task_timeout == 0) return io; - ProcessorPtr processor = std::make_shared(node_path, entry, context, hosts_to_wait); - io.pipeline.init(Pipe{processor}); + auto source = std::make_shared(node_path, entry, context, hosts_to_wait); + io.pipeline = QueryPipeline(std::move(source)); if (context->getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::NONE) - io.pipeline.setSinks([](const Block & header, QueryPipelineBuilder::StreamType){ return std::make_shared(header); }); + io.pipeline.complete(Pipe(std::make_shared(io.pipeline.getHeader()))); return io; } diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index 64261d044c7..bfd76ae6d6a 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -2,6 +2,8 @@ #include #include +#include +#include namespace DB { @@ -9,7 +11,6 @@ namespace DB class ReadBuffer; class WriteBuffer; - /// Parse and execute a query. void executeQuery( ReadBuffer & istr, /// Where to read query from (and data for INSERT, if present). diff --git a/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp b/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp index 9881ce1806f..bdfbbc2874e 100644 --- a/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp +++ b/src/Processors/Executors/PipelineExecutingBlockInputStream.cpp @@ -11,8 +11,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -PipelineExecutingBlockInputStream::PipelineExecutingBlockInputStream(QueryPipelineBuilder pipeline_) - : pipeline(std::make_unique(std::move(pipeline_))) +PipelineExecutingBlockInputStream::PipelineExecutingBlockInputStream(QueryPipeline pipeline_) + : pipeline(std::make_unique(std::move(pipeline_))) { } diff --git a/src/Processors/Executors/PipelineExecutingBlockInputStream.h b/src/Processors/Executors/PipelineExecutingBlockInputStream.h index ca9614c7ee9..68497938ad4 100644 --- a/src/Processors/Executors/PipelineExecutingBlockInputStream.h +++ b/src/Processors/Executors/PipelineExecutingBlockInputStream.h @@ -4,7 +4,7 @@ namespace DB { -class QueryPipelineBuilder; +class QueryPipeline; class PullingAsyncPipelineExecutor; class PullingPipelineExecutor; @@ -13,7 +13,7 @@ class PullingPipelineExecutor; class PipelineExecutingBlockInputStream : public IBlockInputStream { public: - explicit PipelineExecutingBlockInputStream(QueryPipelineBuilder pipeline_); + explicit PipelineExecutingBlockInputStream(QueryPipeline pipeline_); ~PipelineExecutingBlockInputStream() override; String getName() const override { return "PipelineExecuting"; } @@ -32,7 +32,7 @@ protected: Block readImpl() override; private: - std::unique_ptr pipeline; + std::unique_ptr pipeline; /// One of executors is used. std::unique_ptr executor; /// for single thread. std::unique_ptr async_executor; /// for many threads. diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index cf4890d3a76..93fa75f165d 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -87,17 +88,11 @@ QueryPipeline::QueryPipeline( "Cannot create pushing QueryPipeline because its input port does not belong to any processor"); } -QueryPipeline::QueryPipeline( - PipelineResourcesHolder resources_, - Processors processors_, - OutputPort * output_, - OutputPort * totals_, - OutputPort * extremes_) - : resources(std::move(resources_)) - , processors(std::move(processors_)) - , output(output_) - , totals(totals_) - , extremes(extremes_) +static void checkPulling( + Processors & processors, + OutputPort * output, + OutputPort * totals, + OutputPort * extremes) { if (!output || output->isConnected()) throw Exception( @@ -109,7 +104,7 @@ QueryPipeline::QueryPipeline( ErrorCodes::LOGICAL_ERROR, "Cannot create pulling QueryPipeline because its totals port is connected"); - if (extremes || extremes->isConnected()) + if (extremes && extremes->isConnected()) throw Exception( ErrorCodes::LOGICAL_ERROR, "Cannot create pulling QueryPipeline because its extremes port is connected"); @@ -151,9 +146,30 @@ QueryPipeline::QueryPipeline( QueryPipeline::QueryPipeline(std::shared_ptr source) : QueryPipeline(Pipe(std::move(source))) {} -QueryPipeline::QueryPipeline(Pipe pipe) - : QueryPipeline(std::move(pipe.holder), std::move(pipe.processors), pipe.getOutputPort(0), pipe.getTotalsPort(), pipe.getExtremesPort()) +QueryPipeline::QueryPipeline( + PipelineResourcesHolder resources_, + Processors processors_, + OutputPort * output_, + OutputPort * totals_, + OutputPort * extremes_) + : resources(std::move(resources_)) + , processors(std::move(processors_)) + , output(output_) + , totals(totals_) + , extremes(extremes_) { + checkPulling(processors, output, totals, extremes); +} + +QueryPipeline::QueryPipeline(Pipe pipe) +{ + pipe.resize(1); + resources = std::move(pipe.holder); + output = pipe.getOutputPort(0); + totals = pipe.getTotalsPort(); + extremes = pipe.getExtremesPort(); + processors = std::move(pipe.processors); + checkPulling(processors, output, totals, extremes); } QueryPipeline::QueryPipeline(Chain chain) @@ -172,8 +188,41 @@ QueryPipeline::QueryPipeline(Chain chain) input = &chain.getInputPort(); } +static void drop(OutputPort *& port, Processors & processors) +{ + auto null_sink = std::make_shared(port->getHeader()); + connect(*port, null_sink->getPort()); + + processors.emplace_back(std::move(null_sink)); + port = nullptr; +} + QueryPipeline::QueryPipeline(std::shared_ptr sink) : QueryPipeline(Chain(std::move(sink))) {} +void QueryPipeline::complete(Chain chain) +{ + if (!pulling()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline must be pulling to be completed with chain"); + + drop(totals, processors); + drop(extremes, processors); + + processors.reserve(processors.size() + chain.getProcessors().size() + 1); + for (auto processor : chain.getProcessors()) + processors.emplace_back(std::move(processor)); + + auto sink = std::make_shared(chain.getOutputPort().getHeader()); + connect(*output, chain.getInputPort()); + connect(chain.getOutputPort(), sink->getPort()); + processors.emplace_back(std::move(sink)); + input = nullptr; +} + +void QueryPipeline::complete(std::shared_ptr sink) +{ + complete(Chain(std::move(sink))); +} + void QueryPipeline::complete(Pipe pipe) { if (!pushing()) @@ -200,7 +249,7 @@ void QueryPipeline::complete(std::shared_ptr format) auto materializing = std::make_shared(output->getHeader()); connect(*output, materializing->getInputPort()); output = &materializing->getOutputPort(); - processors.emplace_back(std::move(output)); + processors.emplace_back(std::move(materializing)); } auto & format_main = format->getPort(IOutputFormat::PortKind::Main); @@ -209,14 +258,14 @@ void QueryPipeline::complete(std::shared_ptr format) if (!totals) { - auto source = std::make_shared(totals->getHeader()); + auto source = std::make_shared(format_totals.getHeader()); totals = &source->getPort(); processors.emplace_back(std::move(source)); } if (!extremes) { - auto source = std::make_shared(extremes->getHeader()); + auto source = std::make_shared(format_extremes.getHeader()); extremes = &source->getPort(); processors.emplace_back(std::move(source)); } diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index e14f42dd07a..f8d9e24b76f 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -77,6 +77,8 @@ public: void complete(Pipe pipe); /// Only for pulling. void complete(std::shared_ptr format); + void complete(Chain chain); + void complete(std::shared_ptr sink); /// Only for pushing and pulling. Block getHeader() const; @@ -112,6 +114,7 @@ private: friend class PushingAsyncPipelineExecutor; friend class PullingAsyncPipelineExecutor; friend class CompletedPipelineExecutor; + friend class QueryPipelineBuilder; }; } diff --git a/src/Processors/QueryPipelineBuilder.cpp b/src/Processors/QueryPipelineBuilder.cpp index ab4b7501870..437199c5226 100644 --- a/src/Processors/QueryPipelineBuilder.cpp +++ b/src/Processors/QueryPipelineBuilder.cpp @@ -79,6 +79,22 @@ void QueryPipelineBuilder::init(Pipe pipe_) pipe = std::move(pipe_); } +void QueryPipelineBuilder::init(QueryPipeline pipeline) +{ + if (initialized()) + throw Exception("Pipeline has already been initialized.", ErrorCodes::LOGICAL_ERROR); + + if (!pipeline.pulling()) + throw Exception("Can't initialize not pulling pipeline.", ErrorCodes::LOGICAL_ERROR); + + pipe.holder = std::move(pipeline.resources); + pipe.processors = std::move(pipeline.processors); + pipe.output_ports = {pipeline.output}; + pipe.totals_port = pipeline.totals; + pipe.extremes_port = pipeline.extremes; + pipe.max_parallel_streams = pipeline.num_threads; +} + void QueryPipelineBuilder::reset() { Pipe pipe_to_destroy(std::move(pipe)); diff --git a/src/Processors/QueryPipelineBuilder.h b/src/Processors/QueryPipelineBuilder.h index 5ca65398bcc..78ae5dd41be 100644 --- a/src/Processors/QueryPipelineBuilder.h +++ b/src/Processors/QueryPipelineBuilder.h @@ -42,6 +42,7 @@ public: /// All pipes must have same header. void init(Pipe pipe); + void init(QueryPipeline pipeline); /// Clear and release all resources. void reset(); diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 1aeaa6e5166..49bab960e26 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -52,7 +52,7 @@ void CreatingSetsTransform::startSubquery() if (subquery.table) /// TODO: make via port - table_out = Chain(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext())); + table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext())); done_with_set = !subquery.set; done_with_table = !subquery.table; @@ -60,7 +60,7 @@ void CreatingSetsTransform::startSubquery() if (done_with_set /*&& done_with_join*/ && done_with_table) throw Exception("Logical error: nothing to do with subquery", ErrorCodes::LOGICAL_ERROR); - if (!table_out.empty()) + if (!table_out.initialized()) { executor = std::make_unique(table_out); executor->start(); @@ -128,7 +128,7 @@ Chunk CreatingSetsTransform::generate() if (subquery.set) subquery.set->finishInsert(); - if (!table_out.empty()) + if (!table_out.initialized()) { executor->finish(); executor.reset(); diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index 23eeca4f7c5..eca12c33f54 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -45,7 +46,7 @@ private: SubqueryForSet subquery; std::unique_ptr executor; - Chain table_out; + QueryPipeline table_out; UInt64 read_rows = 0; Stopwatch watch; diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 6631da60385..f0f901394ae 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -537,7 +537,6 @@ namespace void createExternalTables(); void generateOutput(); - void generateOutputWithProcessors(); void finishQuery(); void onException(const Exception & exception); @@ -589,7 +588,7 @@ namespace std::optional read_buffer; std::optional write_buffer; - std::unique_ptr pipeline; + std::unique_ptr pipeline; std::unique_ptr pipeline_executor; BlockOutputStreamPtr block_output_stream; bool need_input_data_from_insert_query = true; @@ -806,7 +805,7 @@ namespace void Call::processInput() { - if (io.out.empty()) + if (!io.pipeline.pushing()) return; bool has_data_to_insert = (insert_query && insert_query->data) @@ -821,9 +820,9 @@ namespace /// This is significant, because parallel parsing may be used. /// So we mustn't touch the input stream from other thread. - initializeBlockInputStream(io.out.getInputHeader()); + initializeBlockInputStream(io.pipeline.getHeader()); - PushingPipelineExecutor executor(io.out); + PushingPipelineExecutor executor(io.pipeline); executor.start(); Block block; @@ -896,10 +895,10 @@ namespace }); assert(!pipeline); - pipeline = std::make_unique(); auto source = FormatFactory::instance().getInput( input_format, *read_buffer, header, query_context, query_context->getSettings().max_insert_block_size); - pipeline->init(Pipe(source)); + QueryPipelineBuilder builder; + builder.init(Pipe(source)); /// Add default values if necessary. if (ast) @@ -913,7 +912,7 @@ namespace const auto & columns = storage->getInMemoryMetadataPtr()->getColumns(); if (!columns.empty()) { - pipeline->addSimpleTransform([&](const Block & cur_header) + builder.addSimpleTransform([&](const Block & cur_header) { return std::make_shared(cur_header, columns, *source, query_context); }); @@ -922,6 +921,7 @@ namespace } } + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); pipeline_executor = std::make_unique(*pipeline); } @@ -1030,85 +1030,7 @@ namespace void Call::generateOutput() { - if (io.pipeline.initialized()) - { - generateOutputWithProcessors(); - return; - } - - if (!io.in) - return; - - AsynchronousBlockInputStream async_in(io.in); - write_buffer.emplace(*result.mutable_output()); - block_output_stream = query_context->getOutputStream(output_format, *write_buffer, async_in.getHeader()); - Stopwatch after_send_progress; - - /// Unless the input() function is used we are not going to receive input data anymore. - if (!input_function_is_used) - check_query_info_contains_cancel_only = true; - - auto check_for_cancel = [&] - { - if (isQueryCancelled()) - { - async_in.cancel(false); - return false; - } - return true; - }; - - async_in.readPrefix(); - block_output_stream->writePrefix(); - - while (check_for_cancel()) - { - Block block; - if (async_in.poll(interactive_delay / 1000)) - { - block = async_in.read(); - if (!block) - break; - } - - throwIfFailedToSendResult(); - if (!check_for_cancel()) - break; - - if (block && !io.null_format) - block_output_stream->write(block); - - if (after_send_progress.elapsedMicroseconds() >= interactive_delay) - { - addProgressToResult(); - after_send_progress.restart(); - } - - addLogsToResult(); - - bool has_output = write_buffer->offset(); - if (has_output || result.has_progress() || result.logs_size()) - sendResult(); - - throwIfFailedToSendResult(); - if (!check_for_cancel()) - break; - } - - async_in.readSuffix(); - block_output_stream->writeSuffix(); - - if (!isQueryCancelled()) - { - addTotalsToResult(io.in->getTotals()); - addExtremesToResult(io.in->getExtremes()); - addProfileInfoToResult(io.in->getProfileInfo()); - } - } - - void Call::generateOutputWithProcessors() - { - if (!io.pipeline.initialized()) + if (!io.pipeline.pulling()) return; auto executor = std::make_shared(io.pipeline); diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index 0716d828520..226f1fea324 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -7,6 +7,7 @@ #include "PostgreSQLHandler.h" #include #include +#include #include #if !defined(ARCADIA_BUILD) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b2d3961ba1b..3c999afb846 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -301,16 +301,15 @@ void TCPHandler::runImpl() state.need_receive_data_for_insert = true; processInsertQuery(); } - else if (state.need_receive_data_for_input) // It implies pipeline execution + else if (state.io.pipeline.pulling()) + { + processOrdinaryQueryWithProcessors(); + } + else { - /// It is special case for input(), all works for reading data from client will be done in callbacks. CompletedPipelineExecutor executor(state.io.pipeline); executor.execute(); } - else if (state.io.pipeline.pulling()) - processOrdinaryQueryWithProcessors(); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected QueryPipeline state."); state.io.onFinish(); @@ -1299,7 +1298,7 @@ bool TCPHandler::receiveData(bool scalar) } auto metadata_snapshot = storage->getInMemoryMetadataPtr(); /// The data will be written directly to the table. - Chain temporary_table_out(storage->write(ASTPtr(), metadata_snapshot, query_context)); + QueryPipeline temporary_table_out(storage->write(ASTPtr(), metadata_snapshot, query_context)); PushingPipelineExecutor executor(temporary_table_out); executor.start(); executor.push(block); @@ -1355,8 +1354,8 @@ void TCPHandler::initBlockInput() state.maybe_compressed_in = in; Block header; - if (!state.io.out.empty()) - header = state.io.out.getInputHeader(); + if (state.io.pipeline.pushing()) + header = state.io.pipeline.getHeader(); else if (state.need_receive_data_for_input) header = state.input_header; diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index f20cfc85b8b..935db1c912f 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -969,7 +969,7 @@ private: Data data; }; -ProcessorPtr StorageDistributedDirectoryMonitor::createSourceFromFile(const String & file_name) +std::shared_ptr StorageDistributedDirectoryMonitor::createSourceFromFile(const String & file_name) { return std::make_shared(file_name); } diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index cd1d25179f3..307b57a5668 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -24,6 +24,8 @@ class BackgroundSchedulePool; class IProcessor; using ProcessorPtr = std::shared_ptr; +class ISource; + /** Details of StorageDistributed. * This type is not designed for standalone use. */ @@ -48,7 +50,7 @@ public: void shutdownAndDropAllData(); - static ProcessorPtr createSourceFromFile(const String & file_name); + static std::shared_ptr createSourceFromFile(const String & file_name); /// For scheduling via DistributedBlockOutputStream bool addAndSchedule(size_t file_size, size_t ms); diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index be1a64be926..6596598476d 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -355,9 +355,9 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si if (throttler) job.connection_entry->setThrottler(throttler); - job.chain.addSource(std::make_shared( + job.pipeline = QueryPipeline(std::make_shared( *job.connection_entry, timeouts, query_string, settings, context->getClientInfo())); - job.executor = std::make_unique(job.chain); + job.executor = std::make_unique(job.pipeline); job.executor->start(); } @@ -384,8 +384,8 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si InterpreterInsertQuery interp(copy_query_ast, job.local_context, allow_materialized); auto block_io = interp.execute(); - job.chain = std::move(block_io.out); - job.executor = std::make_unique(job.chain); + job.pipeline = std::move(block_io.pipeline); + job.executor = std::make_unique(job.pipeline); job.executor->start(); } @@ -621,7 +621,7 @@ void DistributedSink::writeToLocal(const Block & block, size_t repeats) InterpreterInsertQuery interp(query_ast, context, allow_materialized); auto block_io = interp.execute(); - PushingPipelineExecutor executor(block_io.out); + PushingPipelineExecutor executor(block_io.pipeline); executor.start(); writeBlockConvert(executor, block, repeats, log); diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index 14498c57364..1fdf5c0291f 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -121,7 +121,7 @@ private: ConnectionPool::Entry connection_entry; ContextPtr local_context; - Chain chain; + QueryPipeline pipeline; std::unique_ptr executor; UInt64 blocks_written = 0; diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index ebfb01f0934..dd8b10ff1fd 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -112,15 +112,16 @@ MergeableBlocksPtr StorageLiveView::collectMergeableBlocks(ContextPtr local_cont InterpreterSelectQuery interpreter(mergeable_query->clone(), local_context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); - auto io = interpreter.execute(); - io.pipeline.addSimpleTransform([&](const Block & cur_header) + auto builder = interpreter.buildQueryPipeline(); + builder.addSimpleTransform([&](const Block & cur_header) { return std::make_shared(cur_header); }); - new_mergeable_blocks->sample_block = io.pipeline.getHeader(); + new_mergeable_blocks->sample_block = builder.getHeader(); - PullingPipelineExecutor executor(io.pipeline); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); + PullingPipelineExecutor executor(pipeline); Block this_block; while (executor.pull(this_block)) @@ -158,8 +159,8 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes) }; block_context->addExternalTable(getBlocksTableName(), TemporaryTableHolder(getContext(), creator)); InterpreterSelectQuery select(getInnerBlocksQuery(), block_context, StoragePtr(), nullptr, SelectQueryOptions(QueryProcessingStage::Complete)); - auto io = select.execute(); - io.pipeline.addSimpleTransform([&](const Block & cur_header) + auto builder = select.buildQueryPipeline(); + builder.addSimpleTransform([&](const Block & cur_header) { return std::make_shared(cur_header); }); @@ -167,7 +168,7 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes) /// 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). - io.pipeline.addSimpleTransform([&](const Block & cur_header) + builder.addSimpleTransform([&](const Block & cur_header) { return std::make_shared( cur_header, @@ -175,7 +176,7 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes) getContext()->getSettingsRef().min_insert_block_size_bytes); }); - return std::move(io.pipeline); + return builder; } void StorageLiveView::writeIntoLiveView( @@ -237,13 +238,14 @@ void StorageLiveView::writeIntoLiveView( InterpreterSelectQuery select_block(mergeable_query, local_context, blocks_storage.getTable(), blocks_storage.getTable()->getInMemoryMetadataPtr(), QueryProcessingStage::WithMergeableState); - auto io = select_block.execute(); - io.pipeline.addSimpleTransform([&](const Block & cur_header) + auto builder = select_block.buildQueryPipeline(); + builder.addSimpleTransform([&](const Block & cur_header) { return std::make_shared(cur_header); }); - PullingPipelineExecutor executor(io.pipeline); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); + PullingPipelineExecutor executor(pipeline); Block this_block; while (executor.pull(this_block)) @@ -381,7 +383,8 @@ bool StorageLiveView::getNewBlocks() /// inserted data to be duplicated auto new_mergeable_blocks = collectMergeableBlocks(live_view_context); Pipes from = blocksToPipes(new_mergeable_blocks->blocks, new_mergeable_blocks->sample_block); - auto pipeline = completeQuery(std::move(from)); + auto builder = completeQuery(std::move(from)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); Block block; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 56383ae7d5d..d9d50a896ea 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -924,10 +924,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor break; } - QueryPipelineBuilder pipeline; - pipeline.init(Pipe::unitePipes(std::move(pipes))); - pipeline.addTransform(std::move(merged_transform)); - pipeline.setMaxThreads(1); + QueryPipelineBuilder builder; + builder.init(Pipe::unitePipes(std::move(pipes))); + builder.addTransform(std::move(merged_transform)); + builder.setMaxThreads(1); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); BlockInputStreamPtr merged_stream = std::make_shared(std::move(pipeline)); if (deduplicate) @@ -1053,8 +1054,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor column_part_pipeline.init(Pipe(std::move(column_part_source))); column_part_pipeline.setMaxThreads(1); + auto cur_pipeline = QueryPipelineBuilder::getPipeline(std::move(column_part_pipeline)); + column_part_streams[part_num] = - std::make_shared(std::move(column_part_pipeline)); + std::make_shared(std::move(cur_pipeline)); } rows_sources_read_buf.seek(0, 0); diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index d1b72d71b83..9d38a70fdf2 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -13,8 +13,9 @@ #include #include #include +#include +#include -#include namespace DB { @@ -235,21 +236,23 @@ void ProjectionDescription::recalculateWithNewColumns(const ColumnsDescription & Block ProjectionDescription::calculate(const Block & block, ContextPtr context) const { - auto in = InterpreterSelectQuery( + auto builder = InterpreterSelectQuery( query_ast, context, Pipe(std::make_shared(block, Chunk(block.getColumns(), block.rows()))), SelectQueryOptions{ type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns : QueryProcessingStage::WithMergeableState}) - .execute() - .getInputStream(); - in = std::make_shared(in, block.rows(), 0); - in->readPrefix(); - auto ret = in->read(); - if (in->read()) + .buildQueryPipeline(); + builder.resize(1); + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); + + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); + PullingPipelineExecutor executor(pipeline); + Block ret; + executor.pull(ret); + if (executor.pull(ret)) throw Exception("Projection cannot increase the number of rows in a block", ErrorCodes::LOGICAL_ERROR); - in->readSuffix(); return ret; } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 9c029d46867..95e6fbaf306 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -962,7 +962,7 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl InterpreterInsertQuery interpreter{insert, insert_context, allow_materialized}; auto block_io = interpreter.execute(); - PushingPipelineExecutor executor(block_io.out); + PushingPipelineExecutor executor(block_io.pipeline); executor.start(); executor.push(std::move(block_to_write)); executor.finish(); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 60459900688..dd304065a14 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -749,7 +749,8 @@ QueryPipelineBuilderPtr StorageDistributed::distributedWrite(const ASTInsertQuer if (shard_info.isLocal()) { InterpreterInsertQuery interpreter(new_query, local_context); - pipelines.emplace_back(std::make_unique(interpreter.execute().pipeline)); + pipelines.emplace_back(std::make_unique()); + pipelines.back()->init(interpreter.execute().pipeline); } else { diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 56b410a3ea4..bbd96f89b82 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -12,6 +12,10 @@ #include #include +#include +#include +#include +#include #include #include #include @@ -75,6 +79,29 @@ StorageExecutable::StorageExecutable( setInMemoryMetadata(storage_metadata); } +class SendingChunkHeaderTransform final : public ISimpleTransform +{ +public: + SendingChunkHeaderTransform(const Block & header, WriteBuffer & buffer_) + : ISimpleTransform(header, header, false) + , buffer(buffer_) + { + } + + String getName() const override { return "SendingChunkHeaderTransform"; } + +protected: + + void transform(Chunk & chunk) override + { + writeText(chunk.getNumRows(), buffer); + writeChar('\n', buffer); + } + +private: + WriteBuffer & buffer; +}; + Pipe StorageExecutable::read( const Names & /*column_names*/, const StorageMetadataPtr & metadata_snapshot, @@ -92,14 +119,13 @@ Pipe StorageExecutable::read( script_name, user_scripts_path); - std::vector inputs; + std::vector inputs; inputs.reserve(input_queries.size()); for (auto & input_query : input_queries) { InterpreterSelectWithUnionQuery interpreter(input_query, context, {}); - auto input = interpreter.execute().getInputStream(); - inputs.emplace_back(std::move(input)); + inputs.emplace_back(interpreter.buildQueryPipeline()); } ShellCommand::Config config(script_path); @@ -134,11 +160,8 @@ Pipe StorageExecutable::read( for (size_t i = 0; i < inputs.size(); ++i) { - BlockInputStreamPtr input_stream = inputs[i]; WriteBufferFromFile * write_buffer = nullptr; - bool send_chunk_header = settings.send_chunk_header; - if (i == 0) { write_buffer = &process->in; @@ -153,27 +176,23 @@ Pipe StorageExecutable::read( write_buffer = &it->second; } - ShellCommandSource::SendDataTask task = [input_stream, write_buffer, context, is_executable_pool, send_chunk_header, this]() + inputs[i].resize(1); + if (settings.send_chunk_header) { - auto output_stream = context->getOutputStream(format, *write_buffer, input_stream->getHeader().cloneEmpty()); - input_stream->readPrefix(); - output_stream->writePrefix(); + auto transform = std::make_shared(inputs[i].getHeader(), *write_buffer); + inputs[i].addTransform(std::move(transform)); + } - while (auto block = input_stream->read()) - { - if (send_chunk_header) - { - writeText(block.rows(), *write_buffer); - writeChar('\n', *write_buffer); - } + auto out = FormatFactory::instance().getOutputFormat(format, *write_buffer, inputs[i].getHeader(), context); + out->setAutoFlush(); + inputs[i].setOutputFormat(std::move(out)); - output_stream->write(block); - } + auto pipeline = std::make_shared(QueryPipelineBuilder::getPipeline(std::move(inputs[i]))); - input_stream->readSuffix(); - output_stream->writeSuffix(); - - output_stream->flush(); + ShellCommandSource::SendDataTask task = [pipeline, write_buffer, is_executable_pool]() + { + CompletedPipelineExecutor executor(*pipeline); + executor.execute(); if (!is_executable_pool) write_buffer->close(); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 997f52e5d70..c6c40453874 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -336,8 +336,7 @@ public: /// Special case for distributed format. Defaults are not needed here. if (storage->format_name == "Distributed") { - pipeline = std::make_unique(); - pipeline->init(Pipe(StorageDistributedDirectoryMonitor::createSourceFromFile(current_path))); + pipeline = std::make_unique(StorageDistributedDirectoryMonitor::createSourceFromFile(current_path)); reader = std::make_unique(*pipeline); continue; } @@ -394,17 +393,19 @@ public: auto format = FormatFactory::instance().getInput( storage->format_name, *read_buf, get_block_for_format(), context, max_block_size, storage->format_settings); - pipeline = std::make_unique(); - pipeline->init(Pipe(format)); + QueryPipelineBuilder builder; + builder.init(Pipe(format)); if (columns_description.hasDefaults()) { - pipeline->addSimpleTransform([&](const Block & header) + builder.addSimpleTransform([&](const Block & header) { return std::make_shared(header, columns_description, *format, context); }); } + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + reader = std::make_unique(*pipeline); } @@ -454,7 +455,7 @@ private: String current_path; Block sample_block; std::unique_ptr read_buf; - std::unique_ptr pipeline; + std::unique_ptr pipeline; std::unique_ptr reader; ColumnsDescription columns_description; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 3e8c2738eb8..b18f26a006c 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -383,7 +383,7 @@ Pipe StorageMerge::createSources( pipe = QueryPipelineBuilder::getPipe(InterpreterSelectQuery( modified_query_info.query, modified_context, std::make_shared(header), - SelectQueryOptions(processed_stage).analyze()).execute().pipeline); + SelectQueryOptions(processed_stage).analyze()).buildQueryPipeline()); pipe.addInterpreterContext(modified_context); return pipe; @@ -424,7 +424,7 @@ Pipe StorageMerge::createSources( InterpreterSelectQuery interpreter{modified_query_info.query, modified_context, SelectQueryOptions(processed_stage)}; - pipe = QueryPipelineBuilder::getPipe(interpreter.execute().pipeline); + pipe = QueryPipelineBuilder::getPipe(interpreter.buildQueryPipeline()); /** Materialization is needed, since from distributed storage the constants come materialized. * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 5061d7a58b9..244957db0df 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -233,17 +233,18 @@ bool StorageS3Source::initialize() std::make_unique(client, bucket, current_key, max_single_read_retries, DBMS_DEFAULT_BUFFER_SIZE), chooseCompressionMethod(current_key, compression_hint)); auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, getContext(), max_block_size, format_settings); - pipeline = std::make_unique(); - pipeline->init(Pipe(input_format)); + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); if (columns_desc.hasDefaults()) { - pipeline->addSimpleTransform([&](const Block & header) + builder.addSimpleTransform([&](const Block & header) { return std::make_shared(header, columns_desc, *input_format, getContext()); }); } + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); reader = std::make_unique(*pipeline); initialized = false; diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index f3387301663..cfd7e496928 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -82,7 +82,7 @@ private: std::unique_ptr read_buf; - std::unique_ptr pipeline; + std::unique_ptr pipeline; std::unique_ptr reader; bool initialized = false; bool with_file_column = false; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 16b20ac78e0..49957814b7c 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -105,14 +105,15 @@ namespace compression_method); auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size, format_settings); - pipeline = std::make_unique(); - pipeline->init(Pipe(input_format)); + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); - pipeline->addSimpleTransform([&](const Block & cur_header) + builder.addSimpleTransform([&](const Block & cur_header) { return std::make_shared(cur_header, columns, *input_format, context); }); + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); reader = std::make_unique(*pipeline); } @@ -139,7 +140,7 @@ namespace private: String name; std::unique_ptr read_buf; - std::unique_ptr pipeline; + std::unique_ptr pipeline; std::unique_ptr reader; }; } diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index cba10548852..9aedee66b5f 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -94,18 +95,18 @@ static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context) if (value->as()) { auto interpreter_subquery = interpretSubquery(value, context, {}, {}); - auto stream = interpreter_subquery->execute().getInputStream(); + auto pipeline = interpreter_subquery->execute().pipeline; SizeLimits limites(context->getSettingsRef().max_rows_in_set, context->getSettingsRef().max_bytes_in_set, OverflowMode::THROW); Set set(limites, true, context->getSettingsRef().transform_null_in); - set.setHeader(stream->getHeader().getColumnsWithTypeAndName()); + set.setHeader(pipeline.getHeader().getColumnsWithTypeAndName()); - stream->readPrefix(); - while (Block block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { set.insertFromBlock(block.getColumnsWithTypeAndName()); } set.finishInsert(); - stream->readSuffix(); set.checkColumnsNumber(1); const auto & set_column = *set.getSetElements()[0]; diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 1b96a0fe713..afd81638da4 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -14,6 +14,8 @@ #include #include +#include + #include @@ -85,8 +87,8 @@ ColumnsDescription ITableFunctionFileLike::getActualTableStructure(ContextPtr co Strings paths = StorageFile::getPathsList(filename, context->getUserFilesPath(), context, total_bytes_to_read); if (paths.empty()) throw Exception("Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME); - auto read_stream = StorageDistributedDirectoryMonitor::createSourceFromFile(paths[0]); - return ColumnsDescription{read_stream->getOutputs().front().getHeader().getNamesAndTypesList()}; + auto source = StorageDistributedDirectoryMonitor::createSourceFromFile(paths[0]); + return ColumnsDescription{source->getOutputs().front().getHeader().getNamesAndTypesList()}; } return parseColumnsListFromString(structure, context); } From 58bb5fe462b0644f0afcc776988067f3416bf908 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Sep 2021 14:40:03 +0300 Subject: [PATCH 060/142] Fix some tests. --- src/Interpreters/executeQuery.cpp | 9 +- .../Executors/CompletedPipelineExecutor.cpp | 11 +- src/Processors/QueryPipeline.cpp | 240 +++++++++++++----- src/Processors/QueryPipeline.h | 4 +- .../Transforms/CreatingSetsTransform.cpp | 5 +- src/Server/TCPHandler.cpp | 2 +- 6 files changed, 202 insertions(+), 69 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 6c7c564dcea..493b1a65752 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -985,7 +985,7 @@ void executeQuery( auto pipe = getSourceFromFromASTInsertQuery(ast, &istr, pipeline.getHeader(), context, nullptr); pipeline.complete(std::move(pipe)); } - else if (pipeline.pushing()) + else if (pipeline.pulling()) { const ASTQueryWithOutput * ast_query_with_output = dynamic_cast(ast.get()); @@ -1042,8 +1042,11 @@ void executeQuery( pipeline.setProgressCallback(context->getProgressCallback()); } - CompletedPipelineExecutor executor(pipeline); - executor.execute(); + if (pipeline.initialized()) + { + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + } } catch (...) { diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index b76173e43b1..e8f55bea936 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -5,7 +5,16 @@ namespace DB { -CompletedPipelineExecutor::CompletedPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) {} +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +CompletedPipelineExecutor::CompletedPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) +{ + if (!pipeline.completed()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for CompletedPipelineExecutor must be completed"); +} void CompletedPipelineExecutor::execute() { diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 93fa75f165d..bbba300c572 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include #include @@ -11,6 +13,9 @@ #include #include #include +#include +#include +#include namespace DB { @@ -38,56 +43,6 @@ static void checkOutput(const OutputPort & output, const ProcessorPtr & processo processor->getName()); } -QueryPipeline::QueryPipeline( - PipelineResourcesHolder resources_, - Processors processors_) - : resources(std::move(resources_)) - , processors(std::move(processors_)) -{ - for (const auto & processor : processors) - { - for (const auto & in : processor->getInputs()) - checkInput(in, processor); - - for (const auto & out : processor->getOutputs()) - checkOutput(out, processor); - } -} - -QueryPipeline::QueryPipeline( - PipelineResourcesHolder resources_, - Processors processors_, - InputPort * input_) - : resources(std::move(resources_)) - , processors(std::move(processors_)) - , input(input_) -{ - if (!input || input->isConnected()) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot create pushing QueryPipeline because its input port is connected or null"); - - bool found_input = false; - for (const auto & processor : processors) - { - for (const auto & in : processor->getInputs()) - { - if (&in == input) - found_input = true; - else - checkInput(in, processor); - } - - for (const auto & out : processor->getOutputs()) - checkOutput(out, processor); - } - - if (!found_input) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Cannot create pushing QueryPipeline because its input port does not belong to any processor"); -} - static void checkPulling( Processors & processors, OutputPort * output, @@ -144,6 +99,156 @@ static void checkPulling( "Cannot create pulling QueryPipeline because its extremes port does not belong to any processor"); } +static void checkCompleted(Processors & processors) +{ + for (const auto & processor : processors) + { + for (const auto & in : processor->getInputs()) + checkInput(in, processor); + + for (const auto & out : processor->getOutputs()) + checkOutput(out, processor); + } +} + +static void initRowsBeforeLimit(IOutputFormat * output_format) +{ + RowsBeforeLimitCounterPtr rows_before_limit_at_least; + + /// TODO: add setRowsBeforeLimitCounter as virtual method to IProcessor. + std::vector limits; + std::vector sources; + std::vector remote_sources; + + std::unordered_set visited; + + struct QueuedEntry + { + IProcessor * processor; + bool visited_limit; + }; + + std::queue queue; + + queue.push({ output_format, false }); + visited.emplace(output_format); + + while (!queue.empty()) + { + auto * processor = queue.front().processor; + auto visited_limit = queue.front().visited_limit; + queue.pop(); + + if (!visited_limit) + { + if (auto * limit = typeid_cast(processor)) + { + visited_limit = true; + limits.emplace_back(limit); + } + + if (auto * source = typeid_cast(processor)) + sources.emplace_back(source); + + if (auto * source = typeid_cast(processor)) + remote_sources.emplace_back(source); + } + else if (auto * sorting = typeid_cast(processor)) + { + if (!rows_before_limit_at_least) + rows_before_limit_at_least = std::make_shared(); + + sorting->setRowsBeforeLimitCounter(rows_before_limit_at_least); + + /// Don't go to children. Take rows_before_limit from last PartialSortingTransform. + continue; + } + + /// Skip totals and extremes port for output format. + if (auto * format = dynamic_cast(processor)) + { + auto * child_processor = &format->getPort(IOutputFormat::PortKind::Main).getOutputPort().getProcessor(); + if (visited.emplace(child_processor).second) + queue.push({ child_processor, visited_limit }); + + continue; + } + + for (auto & child_port : processor->getInputs()) + { + auto * child_processor = &child_port.getOutputPort().getProcessor(); + if (visited.emplace(child_processor).second) + queue.push({ child_processor, visited_limit }); + } + } + + if (!rows_before_limit_at_least && (!limits.empty() || !sources.empty() || !remote_sources.empty())) + { + rows_before_limit_at_least = std::make_shared(); + + for (auto & limit : limits) + limit->setRowsBeforeLimitCounter(rows_before_limit_at_least); + + for (auto & source : sources) + source->setRowsBeforeLimitCounter(rows_before_limit_at_least); + + for (auto & source : remote_sources) + source->setRowsBeforeLimitCounter(rows_before_limit_at_least); + } + + /// If there is a limit, then enable rows_before_limit_at_least + /// It is needed when zero rows is read, but we still want rows_before_limit_at_least in result. + if (!limits.empty()) + rows_before_limit_at_least->add(0); + + if (rows_before_limit_at_least) + output_format->setRowsBeforeLimitCounter(rows_before_limit_at_least); +} + + +QueryPipeline::QueryPipeline( + PipelineResourcesHolder resources_, + Processors processors_) + : resources(std::move(resources_)) + , processors(std::move(processors_)) +{ + checkCompleted(processors); +} + +QueryPipeline::QueryPipeline( + PipelineResourcesHolder resources_, + Processors processors_, + InputPort * input_) + : resources(std::move(resources_)) + , processors(std::move(processors_)) + , input(input_) +{ + if (!input || input->isConnected()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create pushing QueryPipeline because its input port is connected or null"); + + bool found_input = false; + for (const auto & processor : processors) + { + for (const auto & in : processor->getInputs()) + { + if (&in == input) + found_input = true; + else + checkInput(in, processor); + } + + for (const auto & out : processor->getOutputs()) + checkOutput(out, processor); + } + + if (!found_input) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot create pushing QueryPipeline because its input port does not belong to any processor"); +} + QueryPipeline::QueryPipeline(std::shared_ptr source) : QueryPipeline(Pipe(std::move(source))) {} QueryPipeline::QueryPipeline( @@ -163,13 +268,23 @@ QueryPipeline::QueryPipeline( QueryPipeline::QueryPipeline(Pipe pipe) { - pipe.resize(1); resources = std::move(pipe.holder); - output = pipe.getOutputPort(0); - totals = pipe.getTotalsPort(); - extremes = pipe.getExtremesPort(); - processors = std::move(pipe.processors); - checkPulling(processors, output, totals, extremes); + + if (pipe.numOutputPorts() > 0) + { + pipe.resize(1); + output = pipe.getOutputPort(0); + totals = pipe.getTotalsPort(); + extremes = pipe.getExtremesPort(); + + processors = std::move(pipe.processors); + checkPulling(processors, output, totals, extremes); + } + else + { + processors = std::move(pipe.processors); + checkCompleted(processors); + } } QueryPipeline::QueryPipeline(Chain chain) @@ -190,6 +305,9 @@ QueryPipeline::QueryPipeline(Chain chain) static void drop(OutputPort *& port, Processors & processors) { + if (!port) + return; + auto null_sink = std::make_shared(port->getHeader()); connect(*port, null_sink->getPort()); @@ -215,7 +333,7 @@ void QueryPipeline::complete(Chain chain) connect(*output, chain.getInputPort()); connect(chain.getOutputPort(), sink->getPort()); processors.emplace_back(std::move(sink)); - input = nullptr; + output = nullptr; } void QueryPipeline::complete(std::shared_ptr sink) @@ -270,8 +388,6 @@ void QueryPipeline::complete(std::shared_ptr format) processors.emplace_back(std::move(source)); } - processors.emplace_back(std::move(format)); - connect(*output, format_main); connect(*totals, format_totals); connect(*extremes, format_extremes); @@ -279,6 +395,10 @@ void QueryPipeline::complete(std::shared_ptr format) output = nullptr; totals = nullptr; extremes = nullptr; + + initRowsBeforeLimit(format.get()); + + processors.emplace_back(std::move(format)); } Block QueryPipeline::getHeader() const @@ -316,7 +436,7 @@ void QueryPipeline::setProcessListElement(QueryStatus * elem) } else if (pushing()) { - if (auto * counting = dynamic_cast(&input->getOutputPort().getProcessor())) + if (auto * counting = dynamic_cast(&input->getProcessor())) { counting->setProcessListElement(elem); } diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index f8d9e24b76f..0167d3edeeb 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -64,14 +64,14 @@ public: OutputPort * totals_ = nullptr, OutputPort * extremes_ = nullptr); - /// Exactly one of the following is true. bool initialized() const { return !processors.empty(); } + /// When initialized, exactly one of the following is true. /// Use PullingPipelineExecutor or PullingAsyncPipelineExecutor. bool pulling() const { return output != nullptr; } /// Use PushingPipelineExecutor or PushingAsyncPipelineExecutor. bool pushing() const { return input != nullptr; } /// Use PipelineExecutor. Call execute() to build one. - bool completed() const { return !pulling() && !pushing(); } + bool completed() const { return initialized() && !pulling() && !pushing(); } /// Only for pushing. void complete(Pipe pipe); diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 49bab960e26..6776caae9bf 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -60,7 +60,7 @@ void CreatingSetsTransform::startSubquery() if (done_with_set /*&& done_with_join*/ && done_with_table) throw Exception("Logical error: nothing to do with subquery", ErrorCodes::LOGICAL_ERROR); - if (!table_out.initialized()) + if (table_out.initialized()) { executor = std::make_unique(table_out); executor->start(); @@ -128,10 +128,11 @@ Chunk CreatingSetsTransform::generate() if (subquery.set) subquery.set->finishInsert(); - if (!table_out.initialized()) + if (table_out.initialized()) { executor->finish(); executor.reset(); + table_out.reset(); } finishSubquery(); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 3c999afb846..82f1a48bbcb 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -305,7 +305,7 @@ void TCPHandler::runImpl() { processOrdinaryQueryWithProcessors(); } - else + else if (state.io.pipeline.completed()) { CompletedPipelineExecutor executor(state.io.pipeline); executor.execute(); From 6e0ac024a2718d70c1d0bff0937a6e6865a15728 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Sep 2021 15:05:54 +0300 Subject: [PATCH 061/142] Maybe fix some build. --- src/DataStreams/BlockIO.cpp | 4 ---- .../gtest_blocks_size_merging_streams.cpp | 10 ++++------ .../tests/gtest_check_sorted_stream.cpp | 19 +++++++------------ src/Interpreters/InterpreterInsertQuery.cpp | 16 ++++++++-------- .../InterpreterKillQueryQuery.cpp | 2 +- src/Interpreters/executeQuery.cpp | 1 + src/Processors/QueryPipeline.cpp | 5 +++++ src/Storages/tests/gtest_storage_log.cpp | 8 +++----- 8 files changed, 29 insertions(+), 36 deletions(-) diff --git a/src/DataStreams/BlockIO.cpp b/src/DataStreams/BlockIO.cpp index 8a6d0f19cc5..5f1abdaf806 100644 --- a/src/DataStreams/BlockIO.cpp +++ b/src/DataStreams/BlockIO.cpp @@ -5,10 +5,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} void BlockIO::reset() { diff --git a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp b/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp index aa4c717a28b..71cf41fcbab 100644 --- a/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/DataStreams/tests/gtest_blocks_size_merging_streams.cpp @@ -87,9 +87,8 @@ TEST(MergingSortedTest, SimpleBlockSizeTest) pipe.addTransform(std::move(transform)); - QueryPipeline pipeline; - pipeline.init(std::move(pipe)); - pipeline.setMaxThreads(1); + QueryPipeline pipeline(std::move(pipe)); + pipeline.setNumThreads(1); auto stream = std::make_shared(std::move(pipeline)); size_t total_rows = 0; @@ -132,9 +131,8 @@ TEST(MergingSortedTest, MoreInterestingBlockSizes) pipe.addTransform(std::move(transform)); - QueryPipeline pipeline; - pipeline.init(std::move(pipe)); - pipeline.setMaxThreads(1); + QueryPipeline pipeline(std::move(pipe)); + pipeline.setNumThreads(1); auto stream = std::make_shared(std::move(pipeline)); auto block1 = stream->read(); diff --git a/src/DataStreams/tests/gtest_check_sorted_stream.cpp b/src/DataStreams/tests/gtest_check_sorted_stream.cpp index 2788c44389b..04ccc64fd7c 100644 --- a/src/DataStreams/tests/gtest_check_sorted_stream.cpp +++ b/src/DataStreams/tests/gtest_check_sorted_stream.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -97,8 +98,7 @@ TEST(CheckSortedBlockInputStream, CheckGoodCase) return std::make_shared(header, sort_description); }); - QueryPipeline pipeline; - pipeline.init(std::move(pipe)); + QueryPipeline pipeline(std::move(pipe)); PullingPipelineExecutor executor(pipeline); @@ -125,8 +125,7 @@ TEST(CheckSortedBlockInputStream, CheckBadLastRow) return std::make_shared(header, sort_description); }); - QueryPipeline pipeline; - pipeline.init(std::move(pipe)); + QueryPipeline pipeline(std::move(pipe)); PullingPipelineExecutor executor(pipeline); @@ -150,8 +149,7 @@ TEST(CheckSortedBlockInputStream, CheckUnsortedBlock1) return std::make_shared(header, sort_description); }); - QueryPipeline pipeline; - pipeline.init(std::move(pipe)); + QueryPipeline pipeline(std::move(pipe)); PullingPipelineExecutor executor(pipeline); @@ -172,8 +170,7 @@ TEST(CheckSortedBlockInputStream, CheckUnsortedBlock2) return std::make_shared(header, sort_description); }); - QueryPipeline pipeline; - pipeline.init(std::move(pipe)); + QueryPipeline pipeline(std::move(pipe)); PullingPipelineExecutor executor(pipeline); @@ -194,8 +191,7 @@ TEST(CheckSortedBlockInputStream, CheckUnsortedBlock3) return std::make_shared(header, sort_description); }); - QueryPipeline pipeline; - pipeline.init(std::move(pipe)); + QueryPipeline pipeline(std::move(pipe)); PullingPipelineExecutor executor(pipeline); @@ -218,8 +214,7 @@ TEST(CheckSortedBlockInputStream, CheckEqualBlock) return std::make_shared(header, sort_description); }); - QueryPipeline pipeline; - pipeline.init(std::move(pipe)); + QueryPipeline pipeline(std::move(pipe)); PullingPipelineExecutor executor(pipeline); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index e918e373f24..62b0e5a4992 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -173,13 +173,13 @@ Chain InterpreterInsertQuery::buildChainImpl( const Block & query_sample_block, ExceptionKeepingTransformRuntimeDataPtr runtime_data) { - auto context = getContext(); + auto context_ptr = getContext(); const ASTInsertQuery * query = nullptr; if (query_ptr) query = query_ptr->as(); - const Settings & settings = context->getSettingsRef(); - bool null_as_default = query && query->select && context->getSettingsRef().insert_null_as_default; + const Settings & settings = context_ptr->getSettingsRef(); + bool null_as_default = query && query->select && context_ptr->getSettingsRef().insert_null_as_default; /// We create a pipeline of several streams, into which we will write data. Chain out; @@ -188,13 +188,13 @@ Chain InterpreterInsertQuery::buildChainImpl( /// Otherwise we'll get duplicates when MV reads same rows again from Kafka. if (table->noPushingToViews() && !no_destination) { - auto sink = table->write(query_ptr, metadata_snapshot, context); + auto sink = table->write(query_ptr, metadata_snapshot, context_ptr); sink->setRuntimeData(runtime_data); out.addSource(std::move(sink)); } else { - out = buildPushingToViewsDrain(table, metadata_snapshot, context, query_ptr, no_destination, runtime_data); + out = buildPushingToViewsDrain(table, metadata_snapshot, context_ptr, query_ptr, no_destination, runtime_data); } /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. @@ -202,13 +202,13 @@ Chain InterpreterInsertQuery::buildChainImpl( /// Checking constraints. It must be done after calculation of all defaults, so we can check them on calculated columns. if (const auto & constraints = metadata_snapshot->getConstraints(); !constraints.empty()) out.addSource(std::make_shared( - table->getStorageID(), out.getInputHeader(), metadata_snapshot->getConstraints(), context)); + table->getStorageID(), out.getInputHeader(), metadata_snapshot->getConstraints(), context_ptr)); auto adding_missing_defaults_dag = addMissingDefaults( query_sample_block, out.getInputHeader().getNamesAndTypesList(), metadata_snapshot->getColumns(), - context, + context_ptr, null_as_default); auto adding_missing_defaults_actions = std::make_shared(adding_missing_defaults_dag); @@ -233,7 +233,7 @@ Chain InterpreterInsertQuery::buildChainImpl( } auto counting = std::make_shared(out.getInputHeader(), runtime_data ? runtime_data->thread_status : nullptr); - counting->setProcessListElement(context->getProcessListElement()); + counting->setProcessListElement(context_ptr->getProcessListElement()); out.addSource(std::move(counting)); return out; diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 7aad817488b..89fc69e1a3c 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -179,7 +179,7 @@ public: /// Don't produce empty block } while (columns.empty() || columns[0]->empty()); - size_t num_rows = columns.empty() ? columns.front()->size() : 0; + size_t num_rows = columns.empty() ? 0 : columns.front()->size(); return Chunk(std::move(columns), num_rows); } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 493b1a65752..efac66fb458 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -79,6 +79,7 @@ namespace ErrorCodes extern const int INTO_OUTFILE_NOT_ALLOWED; extern const int QUERY_WAS_CANCELLED; extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; } diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index bbba300c572..17319f927bf 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -20,6 +20,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + QueryPipeline::QueryPipeline() = default; QueryPipeline::QueryPipeline(QueryPipeline &&) = default; QueryPipeline & QueryPipeline::operator=(QueryPipeline &&) = default; diff --git a/src/Storages/tests/gtest_storage_log.cpp b/src/Storages/tests/gtest_storage_log.cpp index 27c97aa7eb5..6cd03cad6d0 100644 --- a/src/Storages/tests/gtest_storage_log.cpp +++ b/src/Storages/tests/gtest_storage_log.cpp @@ -102,10 +102,9 @@ std::string writeData(int rows, DB::StoragePtr & table, const DB::ContextPtr con block.insert(column); } - Chain chain; - chain.addSource(table->write({}, metadata_snapshot, context)); + QueryPipeline pipeline(table->write({}, metadata_snapshot, context)); - PushingPipelineExecutor executor(chain); + PushingPipelineExecutor executor(pipeline); executor.push(block); executor.finish(); @@ -125,8 +124,7 @@ std::string readData(DB::StoragePtr & table, const DB::ContextPtr context) QueryProcessingStage::Enum stage = table->getQueryProcessingStage( context, QueryProcessingStage::Complete, metadata_snapshot, query_info); - QueryPipeline pipeline; - pipeline.init(table->read(column_names, metadata_snapshot, query_info, context, stage, 8192, 1)); + QueryPipeline pipeline(table->read(column_names, metadata_snapshot, query_info, context, stage, 8192, 1)); PullingPipelineExecutor executor(pipeline); Block sample; From 618d4d863edc9ee50547791a9bce6d1d124ed069 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Sep 2021 15:59:40 +0300 Subject: [PATCH 062/142] Restore InterpreterInsertQuery --- src/Interpreters/InterpreterInsertQuery.cpp | 72 ++++++--------------- src/Interpreters/InterpreterInsertQuery.h | 13 +--- 2 files changed, 22 insertions(+), 63 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 5b58798693f..d1f1abb6871 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -54,6 +54,7 @@ InterpreterInsertQuery::InterpreterInsertQuery( checkStackSize(); } + StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query) { if (query.table_function) @@ -238,20 +239,26 @@ Chain InterpreterInsertQuery::buildChainImpl( return out; } -std::pair InterpreterInsertQuery::executeImpl( - const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot, Block & sample_block) +BlockIO InterpreterInsertQuery::execute() { - const auto & settings = getContext()->getSettingsRef(); - const auto & query = query_ptr->as(); + const Settings & settings = getContext()->getSettingsRef(); + auto & query = query_ptr->as(); + QueryPipelineBuilder pipeline; + + StoragePtr table = getTable(query); if (query.partition_by && !table->supportsPartitionBy()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "PARTITION BY clause is not supported by storage"); - BlockIO res; - QueryPipelineBuilder pipeline; - std::vector out_chains; + auto table_lock = table->lockForShare(getContext()->getInitialQueryId(), settings.lock_acquire_timeout); + auto metadata_snapshot = table->getInMemoryMetadataPtr(); + + auto query_sample_block = getSampleBlock(query, table, metadata_snapshot); + if (!query.table_function) + getContext()->checkAccess(AccessType::INSERT, query.table_id, query_sample_block.getNames()); bool is_distributed_insert_select = false; + if (query.select && table->isRemote() && settings.parallel_distributed_insert_select) { // Distributed INSERT SELECT @@ -262,6 +269,7 @@ std::pair InterpreterInsertQuery::executeImpl( } } + std::vector out_chains; if (!is_distributed_insert_select || query.watch) { size_t out_streams_size = 1; @@ -330,7 +338,7 @@ std::pair InterpreterInsertQuery::executeImpl( if (getContext()->getSettingsRef().insert_null_as_default) { const auto & input_columns = pipeline.getHeader().getColumnsWithTypeAndName(); - const auto & query_columns = sample_block.getColumnsWithTypeAndName(); + const auto & query_columns = query_sample_block.getColumnsWithTypeAndName(); const auto & output_columns = metadata_snapshot->getColumns(); if (input_columns.size() == query_columns.size()) @@ -340,7 +348,7 @@ std::pair InterpreterInsertQuery::executeImpl( /// Change query sample block columns to Nullable to allow inserting nullable columns, where NULL values will be substituted with /// default column values (in AddingDefaultBlockOutputStream), so all values will be cast correctly. if (input_columns[col_idx].type->isNullable() && !query_columns[col_idx].type->isNullable() && output_columns.hasDefault(query_columns[col_idx].name)) - sample_block.setColumn(col_idx, ColumnWithTypeAndName(makeNullable(query_columns[col_idx].column), makeNullable(query_columns[col_idx].type), query_columns[col_idx].name)); + query_sample_block.setColumn(col_idx, ColumnWithTypeAndName(makeNullable(query_columns[col_idx].column), makeNullable(query_columns[col_idx].type), query_columns[col_idx].name)); } } } @@ -367,28 +375,8 @@ std::pair InterpreterInsertQuery::executeImpl( BlockIO res; - return {std::move(res), std::move(out_streams)}; -} - -BlockIO InterpreterInsertQuery::execute() -{ - const auto & settings = getContext()->getSettingsRef(); - auto & query = query_ptr->as(); - - auto table = getTable(query); - auto table_lock = table->lockForShare(getContext()->getInitialQueryId(), settings.lock_acquire_timeout); - auto metadata_snapshot = table->getInMemoryMetadataPtr(); - auto sample_block = getSampleBlock(query, table, metadata_snapshot); - - if (!query.table_function) - getContext()->checkAccess(AccessType::INSERT, query.table_id, sample_block.getNames()); - - BlockIO res; - BlockOutputStreams out_streams; - std::tie(res, out_streams) = executeImpl(table, metadata_snapshot, sample_block); - /// What type of query: INSERT or INSERT SELECT or INSERT WATCH? - if (out_streams.empty()) + if (is_distributed_insert_select) { res.pipeline = QueryPipelineBuilder::getPipeline(std::move(pipeline)); } @@ -436,7 +424,7 @@ BlockIO InterpreterInsertQuery::execute() if (query.hasInlinedData()) { /// can execute without additional data - auto pipe = getSourceFromFromASTInsertQuery(query_ptr, nullptr, query_sample_block, getContext(), nullptr); + auto pipe = getSourceFromASTInsertQuery(query_ptr, false, query_sample_block, getContext(), nullptr); res.pipeline.complete(std::move(pipe)); } } @@ -444,28 +432,6 @@ BlockIO InterpreterInsertQuery::execute() return res; } -Processors InterpreterInsertQuery::getSinks() -{ - const auto & settings = getContext()->getSettingsRef(); - auto & query = query_ptr->as(); - - auto table = getTable(query); - auto table_lock = table->lockForShare(getContext()->getInitialQueryId(), settings.lock_acquire_timeout); - auto metadata_snapshot = table->getInMemoryMetadataPtr(); - auto sample_block = getSampleBlock(query, table, metadata_snapshot); - - if (!query.table_function) - getContext()->checkAccess(AccessType::INSERT, query.table_id, sample_block.getNames()); - - auto out_streams = executeImpl(table, metadata_snapshot, sample_block).second; - - Processors sinks; - sinks.reserve(out_streams.size()); - for (const auto & out : out_streams) - sinks.emplace_back(std::make_shared(out)); - - return sinks; -} StorageID InterpreterInsertQuery::getDatabaseTable() const { diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 38c15fb4898..378e80a46fe 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB { @@ -33,10 +32,6 @@ public: * Or nothing if the request INSERT SELECT (self-sufficient query - does not accept the input data, does not return the result). */ BlockIO execute() override; - Chain buildChain(); - - /// Returns only sinks, without input sources. - Processors getSinks(); StorageID getDatabaseTable() const; @@ -47,13 +42,11 @@ public: ExceptionKeepingTransformRuntimeDataPtr runtime_data = nullptr); void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const override; - Block getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const; - Block getSampleBlock(const Names & names, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const; - StoragePtr getTable(ASTInsertQuery & query); private: - std::pair executeImpl( - const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot, Block & sample_block); + StoragePtr getTable(ASTInsertQuery & query); + Block getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const; + Block getSampleBlock(const Names & names, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const; ASTPtr query_ptr; const bool allow_materialized; From a8443bef4de90fbd06d650d2f7bb43419643a9a3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Sep 2021 20:52:26 +0300 Subject: [PATCH 063/142] Fix build. --- src/Interpreters/AsynchronousInsertQueue.cpp | 19 ++++++++----------- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.h | 3 ++- src/Interpreters/executeQuery.cpp | 6 +++--- src/Storages/MergeTree/MergeTask.cpp | 13 ++++++------- 5 files changed, 20 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 5b9521f334e..32df55406c2 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -7,7 +7,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -363,10 +365,10 @@ try insert_context->setSettings(key.settings); InterpreterInsertQuery interpreter(key.query, insert_context, key.settings.insert_allow_materialized_columns); - auto sinks = interpreter.getSinks(); - assert(sinks.size() == 1); + auto pipeline = interpreter.execute().pipeline; + assert(pipeline.pushing()); - auto header = sinks.at(0)->getInputs().front().getHeader(); + auto header = pipeline.getHeader(); auto format = getInputFormatFromASTInsertQuery(key.query, false, header, insert_context, nullptr); size_t total_rows = 0; @@ -408,15 +410,10 @@ try size_t total_bytes = chunk.bytes(); auto source = std::make_shared(header, std::move(chunk)); - Pipe pipe(source); + pipeline.complete(Pipe(std::move(source))); - QueryPipeline out_pipeline; - out_pipeline.init(std::move(pipe)); - out_pipeline.resize(1); - out_pipeline.setSinks([&](const Block &, Pipe::StreamType) { return sinks.at(0); }); - - auto out_executor = out_pipeline.execute(); - out_executor->execute(out_pipeline.getNumThreads()); + CompletedPipelineExecutor completed_executor(pipeline); + completed_executor.execute(); LOG_INFO(log, "Flushed {} rows, {} bytes for query '{}'", total_rows, total_bytes, queryToString(key.query)); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index d1f1abb6871..82cc1a68bdd 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -424,7 +424,7 @@ BlockIO InterpreterInsertQuery::execute() if (query.hasInlinedData()) { /// can execute without additional data - auto pipe = getSourceFromASTInsertQuery(query_ptr, false, query_sample_block, getContext(), nullptr); + auto pipe = getSourceFromASTInsertQuery(query_ptr, true, query_sample_block, getContext(), nullptr); res.pipeline.complete(std::move(pipe)); } } diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 378e80a46fe..4afb3218bd6 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -43,9 +43,10 @@ public: void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const override; -private: StoragePtr getTable(ASTInsertQuery & query); Block getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const; + +private: Block getSampleBlock(const Names & names, const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot) const; ASTPtr query_ptr; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 19650157198..88c158652a8 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -52,13 +52,13 @@ #include #include "IO/CompressionMethod.h" +#include "Processors/printPipeline.h" #include #include #include #include #include -#include #include #include @@ -597,7 +597,7 @@ static std::tuple executeQueryImpl( auto timeout = settings.wait_for_async_insert_timeout.totalMilliseconds(); auto query_id = context->getCurrentQueryId(); auto source = std::make_shared(query_id, timeout, *queue); - io.pipeline.init(Pipe(source)); + io.pipeline = QueryPipeline(Pipe(std::move(source))); } return std::make_tuple(ast, std::move(io)); @@ -1020,7 +1020,7 @@ void executeQuery( { if (pipeline.pushing()) { - auto pipe = getSourceFromFromASTInsertQuery(ast, true, pipeline.getHeader(), context, nullptr); + auto pipe = getSourceFromASTInsertQuery(ast, true, pipeline.getHeader(), context, nullptr); pipeline.complete(std::move(pipe)); } else if (pipeline.pulling()) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 855198f697e..a925917cd9a 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -398,9 +398,8 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const column_part_source->setProgressCallback( MergeProgressCallback(*global_ctx->merge_entry, global_ctx->watch_prev_elapsed, *global_ctx->column_progress)); - QueryPipeline column_part_pipeline; - column_part_pipeline.init(Pipe(std::move(column_part_source))); - column_part_pipeline.setMaxThreads(1); + QueryPipeline column_part_pipeline(Pipe(std::move(column_part_source))); + column_part_pipeline.setNumThreads(1); ctx->column_part_streams[part_num] = std::make_shared(std::move(column_part_pipeline)); @@ -795,10 +794,10 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() break; } - QueryPipeline pipeline; - pipeline.init(Pipe::unitePipes(std::move(pipes))); - pipeline.addTransform(std::move(merged_transform)); - pipeline.setMaxThreads(1); + auto res_pipe = Pipe::unitePipes(std::move(pipes)); + res_pipe.addTransform(std::move(merged_transform)); + QueryPipeline pipeline(std::move(res_pipe)); + pipeline.setNumThreads(1); global_ctx->merged_stream = std::make_shared(std::move(pipeline)); From 8f6cb9bd71fba83365970b83042c12abeb12ac2d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 19 Sep 2021 00:10:18 +0300 Subject: [PATCH 064/142] test_send_crash_reports: tiny pylint fixes --- tests/integration/test_send_crash_reports/test.py | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_send_crash_reports/test.py b/tests/integration/test_send_crash_reports/test.py index ab52879c7c3..8cf1c19fb2c 100644 --- a/tests/integration/test_send_crash_reports/test.py +++ b/tests/integration/test_send_crash_reports/test.py @@ -1,9 +1,13 @@ +# pylint: disable=redefined-outer-name +# pylint: disable=unused-argument +# pylint: disable=line-too-long + import os import time +import pytest import helpers.cluster import helpers.test_tools -import pytest from . import fake_sentry_server @@ -30,7 +34,7 @@ def test_send_segfault(started_node): started_node.copy_file_to_container(os.path.join(SCRIPT_DIR, "fake_sentry_server.py"), "/fake_sentry_server.py") started_node.exec_in_container(["bash", "-c", "python3 /fake_sentry_server.py > /fake_sentry_server.log 2>&1"], detach=True, user="root") time.sleep(1) - started_node.exec_in_container(["bash", "-c", "pkill -11 clickhouse"], user="root") + started_node.exec_in_container(["bash", "-c", "pkill -SEGV clickhouse"], user="root") result = None for attempt in range(1, 6): @@ -38,9 +42,9 @@ def test_send_segfault(started_node): result = started_node.exec_in_container(['cat', fake_sentry_server.RESULT_PATH], user='root') if result == 'OK': break - elif result == 'INITIAL_STATE': + if result == 'INITIAL_STATE': continue - elif result: + if result: assert False, 'Unexpected state: ' + result assert result == 'OK', 'Crash report not sent' From f242b5e48c8aec09ee151cd111b44fa5d94aeb6d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 19 Sep 2021 00:13:06 +0300 Subject: [PATCH 065/142] test_send_crash_reports: suppress Fatal messages check (due to explicit SIGSEGV) CI: https://clickhouse-test-reports.s3.yandex.net/0/20d8523a2ebe22c93266e56fc097e273e4c8c68b/integration_tests_(asan).html#fail1 --- tests/integration/test_send_crash_reports/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_send_crash_reports/test.py b/tests/integration/test_send_crash_reports/test.py index 8cf1c19fb2c..55c63c3fe12 100644 --- a/tests/integration/test_send_crash_reports/test.py +++ b/tests/integration/test_send_crash_reports/test.py @@ -1,6 +1,7 @@ # pylint: disable=redefined-outer-name # pylint: disable=unused-argument # pylint: disable=line-too-long +# pylint: disable=bare-except import os import time @@ -24,7 +25,11 @@ def started_node(): cluster.start() yield node finally: - cluster.shutdown() + # It will print Fatal message after pkill -SEGV, suppress it + try: + cluster.shutdown() + except: + pass def test_send_segfault(started_node): From 53e946ec25e08903073705994bc7db855f6cc2ef Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 19 Sep 2021 19:18:38 +0300 Subject: [PATCH 066/142] Fix executors. --- src/Processors/Executors/PullingAsyncPipelineExecutor.cpp | 2 +- src/Processors/Executors/PullingPipelineExecutor.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index b58139c3dcb..e1c5293302e 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -65,7 +65,7 @@ PullingAsyncPipelineExecutor::~PullingAsyncPipelineExecutor() const Block & PullingAsyncPipelineExecutor::getHeader() const { - return pipeline.output->getHeader(); + return lazy_format->getPort(IOutputFormat::PortKind::Main).getHeader(); } static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads) diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index 7ad228760fb..7da2a6d3059 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -36,7 +36,7 @@ PullingPipelineExecutor::~PullingPipelineExecutor() const Block & PullingPipelineExecutor::getHeader() const { - return pipeline.output->getHeader(); + return pulling_format->getPort(IOutputFormat::PortKind::Main).getHeader(); } bool PullingPipelineExecutor::pull(Chunk & chunk) From c1f0ba1f92ecc094a777f8225cd27947e0495c6e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 19 Sep 2021 21:53:36 +0300 Subject: [PATCH 067/142] Fix some tests. --- src/Interpreters/InterpreterInsertQuery.cpp | 8 ++++---- src/Interpreters/executeQuery.cpp | 16 ++++++++++------ src/Processors/QueryPipeline.cpp | 5 +++++ src/Processors/QueryPipeline.h | 2 ++ src/Processors/QueryPipelineBuilder.cpp | 1 + 5 files changed, 22 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 82cc1a68bdd..492714f3f1c 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -366,15 +366,15 @@ BlockIO InterpreterInsertQuery::execute() } } - pipeline.addStorageHolder(table); + BlockIO res; + + res.pipeline.addStorageHolder(table); if (const auto * mv = dynamic_cast(table.get())) { if (auto inner_table = mv->tryGetTargetTable()) - pipeline.addStorageHolder(inner_table); + res.pipeline.addStorageHolder(inner_table); } - BlockIO res; - /// What type of query: INSERT or INSERT SELECT or INSERT WATCH? if (is_distributed_insert_select) { diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 88c158652a8..0d251a77fb4 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -695,12 +695,16 @@ static std::tuple executeQueryImpl( /// Log into system table start of query execution, if need. if (log_queries) { - const auto & info = context->getQueryAccessInfo(); - elem.query_databases = info.databases; - elem.query_tables = info.tables; - elem.query_columns = info.columns; - elem.query_projections = info.projections; - elem.query_views = info.views; + /// This check is not obvious, but without it 01220_scalar_optimization_in_alter fails. + if (pipeline.initialized()) + { + const auto & info = context->getQueryAccessInfo(); + elem.query_databases = info.databases; + elem.query_tables = info.tables; + elem.query_columns = info.columns; + elem.query_projections = info.projections; + elem.query_views = info.views; + } interpreter->extendQueryLogElem(elem, ast, context, query_database, query_table); diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 17319f927bf..c1b1f57a11e 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -478,6 +478,11 @@ bool QueryPipeline::tryGetResultRowsAndBytes(size_t & result_rows, size_t & resu return true; } +void QueryPipeline::addStorageHolder(StoragePtr storage) +{ + resources.storage_holders.emplace_back(std::move(storage)); +} + void QueryPipeline::reset() { QueryPipeline to_remove = std::move(*this); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 0167d3edeeb..7a670d262a3 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -91,6 +91,8 @@ public: void setLimitsAndQuota(const StreamLocalLimits & limits, std::shared_ptr quota); bool tryGetResultRowsAndBytes(size_t & result_rows, size_t & result_bytes) const; + void addStorageHolder(StoragePtr storage); + const Processors & getProcessors() const { return processors; } void reset(); diff --git a/src/Processors/QueryPipelineBuilder.cpp b/src/Processors/QueryPipelineBuilder.cpp index 437199c5226..6d3d4121729 100644 --- a/src/Processors/QueryPipelineBuilder.cpp +++ b/src/Processors/QueryPipelineBuilder.cpp @@ -90,6 +90,7 @@ void QueryPipelineBuilder::init(QueryPipeline pipeline) pipe.holder = std::move(pipeline.resources); pipe.processors = std::move(pipeline.processors); pipe.output_ports = {pipeline.output}; + pipe.header = pipeline.output->getHeader(); pipe.totals_port = pipeline.totals; pipe.extremes_port = pipeline.extremes; pipe.max_parallel_streams = pipeline.num_threads; From 2b7d71d3a2a1835e36f8e064754753350cf68e5d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 19 Sep 2021 23:15:10 +0300 Subject: [PATCH 068/142] Fix some other tests. --- src/Interpreters/AsynchronousInsertQueue.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 5 +++-- src/Interpreters/InterpreterInsertQuery.h | 4 +++- src/Interpreters/MutationsInterpreter.cpp | 8 ++++++-- src/Interpreters/executeQuery.cpp | 6 +++--- src/Processors/QueryPipeline.cpp | 18 ++++++++++++++---- 6 files changed, 30 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 32df55406c2..b651379b75e 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -364,7 +364,7 @@ try insert_context->makeQueryContext(); insert_context->setSettings(key.settings); - InterpreterInsertQuery interpreter(key.query, insert_context, key.settings.insert_allow_materialized_columns); + InterpreterInsertQuery interpreter(key.query, insert_context, key.settings.insert_allow_materialized_columns, false, false, true); auto pipeline = interpreter.execute().pipeline; assert(pipeline.pushing()); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 492714f3f1c..f8223b73383 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -44,12 +44,13 @@ namespace ErrorCodes } InterpreterInsertQuery::InterpreterInsertQuery( - const ASTPtr & query_ptr_, ContextPtr context_, bool allow_materialized_, bool no_squash_, bool no_destination_) + const ASTPtr & query_ptr_, ContextPtr context_, bool allow_materialized_, bool no_squash_, bool no_destination_, bool async_insert_) : WithContext(context_) , query_ptr(query_ptr_) , allow_materialized(allow_materialized_) , no_squash(no_squash_) , no_destination(no_destination_) + , async_insert(async_insert_) { checkStackSize(); } @@ -421,7 +422,7 @@ BlockIO InterpreterInsertQuery::execute() res.pipeline = QueryPipeline(std::move(out_chains.at(0))); res.pipeline.setNumThreads(std::min(res.pipeline.getNumThreads(), settings.max_threads)); - if (query.hasInlinedData()) + if (query.hasInlinedData() && !async_insert) { /// can execute without additional data auto pipe = getSourceFromASTInsertQuery(query_ptr, true, query_sample_block, getContext(), nullptr); diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index 4afb3218bd6..d08790ca774 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -24,7 +24,8 @@ public: ContextPtr context_, bool allow_materialized_ = false, bool no_squash_ = false, - bool no_destination_ = false); + bool no_destination_ = false, + bool async_insert_ = false); /** Prepare a request for execution. Return block streams * - the stream into which you can write data to execute the query, if INSERT; @@ -53,6 +54,7 @@ private: const bool allow_materialized; const bool no_squash; const bool no_destination; + const bool async_insert; Chain buildChainImpl( const StoragePtr & table, diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 890c0dff0d0..540d5c76c97 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -221,13 +221,17 @@ bool isStorageTouchedByMutations( PullingPipelineExecutor executor(io.pipeline); Block block; - auto should_continue = executor.pull(block); + while (!block.rows()) + executor.pull(block); if (!block.rows()) return false; - else if (block.rows() != 1 || should_continue) + else if (block.rows() != 1) throw Exception("count() expression returned " + toString(block.rows()) + " rows, not 1", ErrorCodes::LOGICAL_ERROR); + Block tmp_block; + while (executor.pull(tmp_block)); + auto count = (*block.getByName("count()").column)[0].get(); return count != 0; } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 0d251a77fb4..74addc57d4f 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -51,8 +51,8 @@ #include #include -#include "IO/CompressionMethod.h" -#include "Processors/printPipeline.h" +#include +#include #include #include @@ -1061,7 +1061,7 @@ void executeQuery( auto out = FormatFactory::instance().getOutputFormatParallelIfPossible( format_name, compressed_buffer ? *compressed_buffer : *out_buf, - pipeline.getHeader(), + materializeBlock(pipeline.getHeader()), context, {}, output_format_settings); diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index c1b1f57a11e..e81ce4c8ff8 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -362,6 +362,17 @@ void QueryPipeline::complete(Pipe pipe) processors.insert(processors.end(), pipe_processors.begin(), pipe_processors.end()); } +static void addMaterializing(OutputPort *& output, Processors & processors) +{ + if (!output) + return; + + auto materializing = std::make_shared(output->getHeader()); + connect(*output, materializing->getInputPort()); + output = &materializing->getOutputPort(); + processors.emplace_back(std::move(materializing)); +} + void QueryPipeline::complete(std::shared_ptr format) { if (!pulling()) @@ -369,10 +380,9 @@ void QueryPipeline::complete(std::shared_ptr format) if (format->expectMaterializedColumns()) { - auto materializing = std::make_shared(output->getHeader()); - connect(*output, materializing->getInputPort()); - output = &materializing->getOutputPort(); - processors.emplace_back(std::move(materializing)); + addMaterializing(output, processors); + addMaterializing(totals, processors); + addMaterializing(extremes, processors); } auto & format_main = format->getPort(IOutputFormat::PortKind::Main); From 9398c22aae1b1bc2d4a62dbb113da9b88966fca1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Sep 2021 12:05:34 +0300 Subject: [PATCH 069/142] Fix more tests. --- src/Interpreters/InterpreterKillQueryQuery.cpp | 7 +++++-- src/Interpreters/executeQuery.cpp | 5 +++-- src/Processors/QueryPipeline.cpp | 13 ++++++------- src/Processors/QueryPipeline.h | 2 ++ src/Processors/QueryPipelineBuilder.cpp | 17 +++++++++++++---- src/Storages/StorageLog.cpp | 2 ++ src/Storages/StorageLog.h | 1 + 7 files changed, 32 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 89fc69e1a3c..7c5d021b98d 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -306,9 +306,12 @@ Block InterpreterKillQueryQuery::getSelectResult(const String & columns, const S auto io = executeQuery(select_query, getContext(), true); PullingPipelineExecutor executor(io.pipeline); Block res; - bool need_another_read = executor.pull(res); + while (!res && executor.pull(res)); - if (res && need_another_read) + Block tmp_block; + while (executor.pull(tmp_block)) + + if (tmp_block) throw Exception("Expected one block from input stream", ErrorCodes::LOGICAL_ERROR); return res; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 74addc57d4f..39221f9308a 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -775,7 +775,8 @@ static std::tuple executeQueryImpl( log_queries, log_queries_min_type = settings.log_queries_min_type, log_queries_min_query_duration_ms = settings.log_queries_min_query_duration_ms.totalMilliseconds(), - status_info_to_query_log + status_info_to_query_log, + pulling_pipeline = pipeline.pulling() ] (QueryPipeline & query_pipeline) mutable { @@ -805,7 +806,7 @@ static std::tuple executeQueryImpl( if (progress_callback) progress_callback(Progress(WriteProgress(info.written_rows, info.written_bytes))); - else if (query_pipeline.pulling()) + if (pulling_pipeline) { query_pipeline.tryGetResultRowsAndBytes(elem.result_rows, elem.result_bytes); } diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index e81ce4c8ff8..c1600dc1c31 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -327,6 +327,8 @@ void QueryPipeline::complete(Chain chain) if (!pulling()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline must be pulling to be completed with chain"); + resources = chain.detachResources(); + drop(totals, processors); drop(extremes, processors); @@ -412,6 +414,7 @@ void QueryPipeline::complete(std::shared_ptr format) extremes = nullptr; initRowsBeforeLimit(format.get()); + output_format = format.get(); processors.emplace_back(std::move(format)); } @@ -476,15 +479,11 @@ void QueryPipeline::setLimitsAndQuota(const StreamLocalLimits & limits, std::sha bool QueryPipeline::tryGetResultRowsAndBytes(size_t & result_rows, size_t & result_bytes) const { - if (!output || !output->isConnected()) + if (!output_format) return false; - const auto * format = typeid_cast(&output->getInputPort().getProcessor()); - if (!format) - return false; - - result_rows = format->getResultRows(); - result_bytes = format->getResultBytes(); + result_rows = output_format->getResultRows(); + result_bytes = output_format->getResultBytes(); return true; } diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 7a670d262a3..de1396e9565 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -109,6 +109,8 @@ private: QueryStatus * process_list_element = nullptr; + IOutputFormat * output_format = nullptr; + size_t num_threads = 0; friend class PushingPipelineExecutor; diff --git a/src/Processors/QueryPipelineBuilder.cpp b/src/Processors/QueryPipelineBuilder.cpp index 6d3d4121729..8ed413166da 100644 --- a/src/Processors/QueryPipelineBuilder.cpp +++ b/src/Processors/QueryPipelineBuilder.cpp @@ -84,13 +84,22 @@ void QueryPipelineBuilder::init(QueryPipeline pipeline) if (initialized()) throw Exception("Pipeline has already been initialized.", ErrorCodes::LOGICAL_ERROR); - if (!pipeline.pulling()) - throw Exception("Can't initialize not pulling pipeline.", ErrorCodes::LOGICAL_ERROR); + if (pipeline.pushing()) + throw Exception("Can't initialize pushing pipeline.", ErrorCodes::LOGICAL_ERROR); pipe.holder = std::move(pipeline.resources); pipe.processors = std::move(pipeline.processors); - pipe.output_ports = {pipeline.output}; - pipe.header = pipeline.output->getHeader(); + if (pipeline.output) + { + pipe.output_ports = {pipeline.output}; + pipe.header = pipeline.output->getHeader(); + } + else + { + pipe.output_ports.clear(); + pipe.header = {}; + } + pipe.totals_port = pipeline.totals; pipe.extremes_port = pipeline.extremes; pipe.max_parallel_streams = pipeline.num_threads; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 623f1ae17f6..0fd94bac95a 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -463,6 +463,8 @@ void LogSink::writeMarks(MarksForColumns && marks) } } +StorageLog::~StorageLog() = default; + StorageLog::StorageLog( DiskPtr disk_, const String & relative_path_, diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 116bdc31520..b4cd15ef258 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -23,6 +23,7 @@ class StorageLog final : public shared_ptr_helper, public IStorage friend struct shared_ptr_helper; public: + ~StorageLog() override; String getName() const override { return "Log"; } Pipe read( From 4eba3fcc8a9004a49dd5a71a1a84a06d7a16b740 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Sep 2021 12:48:53 +0300 Subject: [PATCH 070/142] Fix even more tests. --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 1 + src/Interpreters/executeQuery.cpp | 2 +- src/Processors/Chain.h | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 8a720916776..b19b07a82bb 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -403,6 +403,7 @@ Chain buildPushingToViewsDrain( InterpreterInsertQuery interpreter(nullptr, insert_context, false, false, false); out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, view_runtime_data); + out.addStorageHolder(inner_table); } else if (auto * live_view = dynamic_cast(dependent_table.get())) { diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 39221f9308a..8d750e57278 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1021,6 +1021,7 @@ void executeQuery( std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr); auto & pipeline = streams.pipeline; + std::unique_ptr compressed_buffer; try { if (pipeline.pushing()) @@ -1033,7 +1034,6 @@ void executeQuery( const ASTQueryWithOutput * ast_query_with_output = dynamic_cast(ast.get()); WriteBuffer * out_buf = &ostr; - std::unique_ptr compressed_buffer; if (ast_query_with_output && ast_query_with_output->out_file) { if (!allow_into_outfile) diff --git a/src/Processors/Chain.h b/src/Processors/Chain.h index 9f27e37cc40..da5167f9c7a 100644 --- a/src/Processors/Chain.h +++ b/src/Processors/Chain.h @@ -40,6 +40,7 @@ public: static std::list getProcessors(Chain chain) { return std::move(chain.processors); } void addTableLock(TableLockHolder lock) { holder.table_locks.emplace_back(std::move(lock)); } + void addStorageHolder(StoragePtr storage) { holder.storage_holders.emplace_back(std::move(storage)); } void attachResources(PipelineResourcesHolder holder_) { holder = std::move(holder_); } PipelineResourcesHolder detachResources() { return std::move(holder); } From 729e530fe7f5def470db39b6fb9455593e223593 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Sep 2021 13:36:01 +0300 Subject: [PATCH 071/142] Try fix crash. --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index b19b07a82bb..726d54558c2 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -404,6 +404,11 @@ Chain buildPushingToViewsDrain( InterpreterInsertQuery interpreter(nullptr, insert_context, false, false, false); out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, view_runtime_data); out.addStorageHolder(inner_table); + if (const auto * mv = dynamic_cast(inner_table.get())) + { + if (auto inner_mv_table = mv->tryGetTargetTable()) + out.addStorageHolder(inner_mv_table); + } } else if (auto * live_view = dynamic_cast(dependent_table.get())) { From 00dba6e745f9ca8110b4aa33147da4664545c0bb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Sep 2021 16:08:52 +0300 Subject: [PATCH 072/142] Fix crash in kill --- src/Interpreters/InterpreterKillQueryQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 7c5d021b98d..e94eee25dc3 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -132,6 +132,7 @@ public: , process_list(process_list_) , processes_to_stop(std::move(processes_to_stop_)) , processes_block(std::move(processes_block_)) + , res_sample_block(std::move(res_sample_block_)) { addTotalRowsApprox(processes_to_stop.size()); } @@ -309,7 +310,7 @@ Block InterpreterKillQueryQuery::getSelectResult(const String & columns, const S while (!res && executor.pull(res)); Block tmp_block; - while (executor.pull(tmp_block)) + while (executor.pull(tmp_block)); if (tmp_block) throw Exception("Expected one block from input stream", ErrorCodes::LOGICAL_ERROR); From 211bf07e682ae6b6e8b7de258b86257890c6eab3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Sep 2021 18:10:21 +0300 Subject: [PATCH 073/142] Another try to fix crash. --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 726d54558c2..37a1ffa4781 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -403,12 +403,8 @@ Chain buildPushingToViewsDrain( InterpreterInsertQuery interpreter(nullptr, insert_context, false, false, false); out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, view_runtime_data); + out.addStorageHolder(dependent_table); out.addStorageHolder(inner_table); - if (const auto * mv = dynamic_cast(inner_table.get())) - { - if (auto inner_mv_table = mv->tryGetTargetTable()) - out.addStorageHolder(inner_mv_table); - } } else if (auto * live_view = dynamic_cast(dependent_table.get())) { From c7533d58ee0ee05108ada152bb3d5c3bd961a25c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Sep 2021 19:09:39 +0300 Subject: [PATCH 074/142] Add debug output. --- src/Storages/StorageLog.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 0fd94bac95a..9e37acbdb0f 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -463,7 +463,10 @@ void LogSink::writeMarks(MarksForColumns && marks) } } -StorageLog::~StorageLog() = default; +StorageLog::~StorageLog() +{ + std::cerr << "======================\n" << StackTrace().toString() << std::endl; +} StorageLog::StorageLog( DiskPtr disk_, From d391e3419edf029acf6e9617bb4785f3078e4cc3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Sep 2021 20:54:01 +0300 Subject: [PATCH 075/142] Fixing build. --- src/Interpreters/UserDefinedExecutableFunctionFactory.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp index b6fb4d8d5e5..8e8bb0fdb73 100644 --- a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp +++ b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp @@ -110,8 +110,7 @@ public: shell_command_source_configuration, process_pool)); - QueryPipeline pipeline; - pipeline.init(std::move(pipe)); + QueryPipeline pipeline(std::move(pipe)); PullingPipelineExecutor executor(pipeline); From 4e7d5191e3d846f4130029f9412ac6fdfa3a3b61 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Sep 2021 22:07:33 +0300 Subject: [PATCH 076/142] Fix more tests --- src/Interpreters/InterpreterDescribeQuery.cpp | 1 - src/Interpreters/ProcessList.cpp | 9 ++++----- src/Processors/Executors/CompletedPipelineExecutor.cpp | 2 +- .../Executors/PushingAsyncPipelineExecutor.cpp | 2 +- src/Processors/Executors/PushingPipelineExecutor.cpp | 2 +- src/Storages/StorageLog.cpp | 2 +- 6 files changed, 8 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index b3267d6303f..2ebae17cd6b 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -16,7 +16,6 @@ #include #include #include -#include namespace DB diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 27ecc534763..98dd67b2e12 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -305,13 +305,12 @@ CancellationCode QueryStatus::cancelQuery(bool) if (is_killed.load()) return CancellationCode::CancelSent; - SCOPE_EXIT({ - std::lock_guard lock(executors_mutex); - for (auto * e : executors) - e->cancel(); - }); + std::lock_guard lock(executors_mutex); + for (auto * e : executors) + e->cancel(); is_killed.store(true); + return CancellationCode::CancelSent; } diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index e8f55bea936..ff7901c815d 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -18,7 +18,7 @@ CompletedPipelineExecutor::CompletedPipelineExecutor(QueryPipeline & pipeline_) void CompletedPipelineExecutor::execute() { - PipelineExecutor executor(pipeline.processors); + PipelineExecutor executor(pipeline.processors, pipeline.process_list_element); executor.execute(pipeline.getNumThreads()); } diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 00a5892151c..ee4b73c3222 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -163,7 +163,7 @@ void PushingAsyncPipelineExecutor::start() started = true; data = std::make_unique(); - data->executor = std::make_shared(pipeline.processors); + data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element); data->source = pushing_source.get(); auto func = [&, thread_group = CurrentThread::getGroup()]() diff --git a/src/Processors/Executors/PushingPipelineExecutor.cpp b/src/Processors/Executors/PushingPipelineExecutor.cpp index 61aae02c068..2e2b5e9ca1e 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingPipelineExecutor.cpp @@ -85,7 +85,7 @@ void PushingPipelineExecutor::start() return; started = true; - executor = std::make_shared(pipeline.processors); + executor = std::make_shared(pipeline.processors, pipeline.process_list_element); if (!executor->executeStep(&need_data_flag)) throw Exception(ErrorCodes::LOGICAL_ERROR, diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 9e37acbdb0f..61dacf9b2ee 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -465,7 +465,7 @@ void LogSink::writeMarks(MarksForColumns && marks) StorageLog::~StorageLog() { - std::cerr << "======================\n" << StackTrace().toString() << std::endl; + // std::cerr << "======================\n" << StackTrace().toString() << std::endl; } StorageLog::StorageLog( From 12a4c74af06224d46eb700a59a4751b710188607 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 21 Sep 2021 09:57:55 +0300 Subject: [PATCH 077/142] Fix more tests again --- .../PushingToViewsBlockOutputStream.cpp | 2 -- src/Interpreters/executeDDLQueryOnCluster.cpp | 2 +- src/Interpreters/executeQuery.cpp | 4 ++-- src/Processors/QueryPipeline.cpp | 17 +++++++++++++++-- src/Processors/QueryPipeline.h | 2 ++ 5 files changed, 20 insertions(+), 7 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 37a1ffa4781..5cfb80e5564 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -602,8 +602,6 @@ static void logQueryViews(std::list & views, ContextPtr context try { - //std::cerr << "============ Logging for " << static_cast(view.runtime_stats.thread_status.get()) << ' ' << view.table_id.getNameForLogs() << "\n"; - if (view.runtime_stats.thread_status) view.runtime_stats.thread_status->logToQueryViewsLog(view); } diff --git a/src/Interpreters/executeDDLQueryOnCluster.cpp b/src/Interpreters/executeDDLQueryOnCluster.cpp index f32bf301070..576c1f3ffdd 100644 --- a/src/Interpreters/executeDDLQueryOnCluster.cpp +++ b/src/Interpreters/executeDDLQueryOnCluster.cpp @@ -217,7 +217,7 @@ BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & en io.pipeline = QueryPipeline(std::move(source)); if (context->getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::NONE) - io.pipeline.complete(Pipe(std::make_shared(io.pipeline.getHeader()))); + io.pipeline.complete(std::make_shared(io.pipeline.getHeader())); return io; } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 8d750e57278..d98f3f4bd00 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -657,13 +657,13 @@ static std::tuple executeQueryImpl( /// Hold element of process list till end of query execution. res.process_list_entry = process_list_entry; - if (pipeline.pulling()) + if (pipeline.pulling() || pipeline.completed()) { /// Limits on the result, the quota on the result, and also callback for progress. /// Limits apply only to the final result. pipeline.setProgressCallback(context->getProgressCallback()); pipeline.setProcessListElement(context->getProcessListElement()); - if (stage == QueryProcessingStage::Complete) + if (stage == QueryProcessingStage::Complete && pipeline.pulling()) pipeline.setLimitsAndQuota(limits, quota); } else if (pipeline.pushing()) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index c1600dc1c31..0b4b2c308fe 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -322,6 +322,19 @@ static void drop(OutputPort *& port, Processors & processors) QueryPipeline::QueryPipeline(std::shared_ptr sink) : QueryPipeline(Chain(std::move(sink))) {} +void QueryPipeline::complete(std::shared_ptr sink) +{ + if (!pulling()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline must be pulling to be completed with chain"); + + drop(totals, processors); + drop(extremes, processors); + + connect(*output, sink->getPort()); + processors.emplace_back(std::move(sink)); + output = nullptr; +} + void QueryPipeline::complete(Chain chain) { if (!pulling()) @@ -444,7 +457,7 @@ void QueryPipeline::setProcessListElement(QueryStatus * elem) { process_list_element = elem; - if (pulling()) + if (pulling() || completed()) { for (auto & processor : processors) { @@ -467,7 +480,7 @@ void QueryPipeline::setLimitsAndQuota(const StreamLocalLimits & limits, std::sha if (!pulling()) throw Exception( ErrorCodes::LOGICAL_ERROR, - "It is possible to set limits and quota only to pullint QueryPipeline"); + "It is possible to set limits and quota only to pulling QueryPipeline"); auto transform = std::make_shared(output->getHeader(), limits); transform->setQuota(quota); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index de1396e9565..ff4c80a55c0 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -25,6 +25,7 @@ class Chain; class IOutputFormat; class SinkToStorage; class ISource; +class ISink; class QueryPipeline { @@ -79,6 +80,7 @@ public: void complete(std::shared_ptr format); void complete(Chain chain); void complete(std::shared_ptr sink); + void complete(std::shared_ptr sink); /// Only for pushing and pulling. Block getHeader() const; From 0308f14beaf2acd92e767e6884d3626e139dd400 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 21 Sep 2021 11:01:00 +0300 Subject: [PATCH 078/142] Fix another one test. --- src/Interpreters/ProcessList.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 98dd67b2e12..4fd022772f6 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -305,12 +305,12 @@ CancellationCode QueryStatus::cancelQuery(bool) if (is_killed.load()) return CancellationCode::CancelSent; + is_killed.store(true); + std::lock_guard lock(executors_mutex); for (auto * e : executors) e->cancel(); - is_killed.store(true); - return CancellationCode::CancelSent; } From 02e5e662fd09158412bbfd216be163653ef311ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 21 Sep 2021 13:35:41 +0300 Subject: [PATCH 079/142] Refactor a little bit. --- .../PushingToViewsBlockOutputStream.cpp | 87 +++++++++++-------- .../PushingToViewsBlockOutputStream.h | 14 ++- src/Interpreters/InterpreterInsertQuery.cpp | 17 ++-- src/Interpreters/InterpreterInsertQuery.h | 10 +-- src/Interpreters/QueryViewsLog.h | 2 +- src/Interpreters/ThreadStatusExt.cpp | 12 +-- .../Executors/PushingAsyncPipelineExecutor.h | 1 + .../Executors/PushingPipelineExecutor.h | 1 + src/Processors/QueryPipeline.cpp | 1 - src/Processors/QueryPipeline.h | 1 + .../Transforms/ExceptionKeepingTransform.cpp | 57 +++++------- .../Transforms/ExceptionKeepingTransform.h | 19 +--- 12 files changed, 107 insertions(+), 115 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 5cfb80e5564..af2509df19e 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -126,6 +126,25 @@ private: static void logQueryViews(std::list & views, ContextPtr context); +static std::exception_ptr addStorageToException(std::exception_ptr ptr, const StorageID & storage) +{ + try + { + std::rethrow_exception(std::move(ptr)); + } + catch (DB::Exception & exception) + { + exception.addMessage("while inserting into {}", storage.getNameForLogs()); + return std::current_exception(); + } + catch (...) + { + return std::current_exception(); + } + + __builtin_unreachable(); +} + class FinalizingViewsTransform final : public IProcessor { struct ExceptionStatus @@ -227,12 +246,11 @@ public: if (!any_exception) any_exception = status.exception; - //std::cerr << "=== Setting exception for " << view.table_id.getFullNameNotQuoted() << std::endl; - view.setException(std::move(status.exception)); + view.setException(addStorageToException(std::move(status.exception), view.table_id)); } else { - view.runtime_stats.setStatus(QueryViewsLogElement::ViewStatus::QUERY_FINISH); + view.runtime_stats->setStatus(QueryViewsLogElement::ViewStatus::QUERY_FINISH); LOG_TRACE( &Poco::Logger::get("PushingToViews"), @@ -240,7 +258,7 @@ public: views_data->max_threads <= 1 ? "sequentially" : ("parallel " + std::to_string(views_data->max_threads)), views_data->source_storage_id.getNameForLogs(), view.table_id.getNameForLogs(), - view.runtime_stats.elapsed_ms); + view.runtime_stats->elapsed_ms); } } @@ -288,8 +306,9 @@ Chain buildPushingToViewsDrain( ContextPtr context, const ASTPtr & query_ptr, bool no_destination, - ExceptionKeepingTransformRuntimeDataPtr runtime_data, - const Block & lv_storage) + ThreadStatus * thread_status, + std::atomic_uint64_t * elapsed_counter_ms, + const Block & live_view_header) { checkStackSize(); Chain result_chain; @@ -345,12 +364,10 @@ Chain buildPushingToViewsDrain( ASTPtr query; Chain out; - QueryViewsLogElement::ViewType type = QueryViewsLogElement::ViewType::DEFAULT; - String target_name = database_table.getFullTableName(); /// If the materialized view is executed outside of a query, for example as a result of SYSTEM FLUSH LOGS or /// SYSTEM FLUSH DISTRIBUTED ..., we can't attach to any thread group and we won't log, so there is no point on collecting metrics - std::unique_ptr thread_status = nullptr; + std::unique_ptr view_thread_status_ptr = nullptr; ThreadGroupStatusPtr running_group = current_thread && current_thread->getThreadGroup() ? current_thread->getThreadGroup() @@ -364,17 +381,24 @@ Chain buildPushingToViewsDrain( auto * original_thread = current_thread; SCOPE_EXIT({ current_thread = original_thread; }); - thread_status = std::make_unique(); + view_thread_status_ptr = std::make_unique(); /// Disable query profiler for this ThreadStatus since the running (main query) thread should already have one /// If we didn't disable it, then we could end up with N + 1 (N = number of dependencies) profilers which means /// N times more interruptions - thread_status->disableProfiling(); - thread_status->attachQuery(running_group); + view_thread_status_ptr->disableProfiling(); + view_thread_status_ptr->attachQuery(running_group); } - auto view_runtime_data = std::make_shared( - thread_status.get(), - database_table.getNameForLogs()); + auto runtime_stats = std::make_unique(); + runtime_stats->target_name = database_table.getFullTableName(); + runtime_stats->thread_status = std::move(view_thread_status_ptr); + runtime_stats->event_time = std::chrono::system_clock::now(); + runtime_stats->event_status = QueryViewsLogElement::ViewStatus::EXCEPTION_BEFORE_START; + + auto & type = runtime_stats->type; + auto & target_name = runtime_stats->target_name; + auto * view_thread_status = runtime_stats->thread_status.get(); + auto * view_counter_ms = &runtime_stats->elapsed_ms; if (auto * materialized_view = dynamic_cast(dependent_table.get())) { @@ -402,28 +426,20 @@ Chain buildPushingToViewsDrain( } InterpreterInsertQuery interpreter(nullptr, insert_context, false, false, false); - out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, view_runtime_data); + out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, view_thread_status, view_counter_ms); out.addStorageHolder(dependent_table); out.addStorageHolder(inner_table); } else if (auto * live_view = dynamic_cast(dependent_table.get())) { - type = QueryViewsLogElement::ViewType::LIVE; + runtime_stats->type = QueryViewsLogElement::ViewType::LIVE; query = live_view->getInnerQuery(); // Used only to log in system.query_views_log out = buildPushingToViewsDrain( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true, view_runtime_data, storage_header); + dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true, view_thread_status, view_counter_ms, storage_header); } else out = buildPushingToViewsDrain( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), false, view_runtime_data); - - QueryViewsLogElement::ViewRuntimeStats runtime_stats{ - target_name, - type, - std::move(thread_status), - 0, - std::chrono::system_clock::now(), - QueryViewsLogElement::ViewStatus::EXCEPTION_BEFORE_START}; + dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), false, view_thread_status, view_counter_ms); views_data->views.emplace_back(ViewRuntimeData{ std::move(query), @@ -439,7 +455,7 @@ Chain buildPushingToViewsDrain( { auto executing_inner_query = std::make_shared( storage_header, views_data->views.back(), views_data->source_storage_id, views_data->source_metadata_snapshot, views_data->source_storage); - executing_inner_query->setRuntimeData(view_runtime_data); + executing_inner_query->setRuntimeData(view_thread_status, elapsed_counter_ms); out.addSource(std::move(executing_inner_query)); } @@ -450,7 +466,7 @@ Chain buildPushingToViewsDrain( if (!no_destination) { context->getQueryContext()->addQueryAccessInfo( - backQuoteIfNeed(database_table.getDatabaseName()), target_name, {}, "", database_table.getFullTableName()); + backQuoteIfNeed(database_table.getDatabaseName()), views_data->views.back().runtime_stats->target_name, {}, "", database_table.getFullTableName()); } } @@ -494,8 +510,8 @@ Chain buildPushingToViewsDrain( if (auto * live_view = dynamic_cast(storage.get())) { - auto sink = std::make_shared(lv_storage, *live_view, storage, context); - sink->setRuntimeData(runtime_data); + auto sink = std::make_shared(live_view_header, *live_view, storage, context); + sink->setRuntimeData(thread_status, elapsed_counter_ms); result_chain.addSource(std::move(sink)); } /// Do not push to destination table if the flag is set @@ -503,7 +519,7 @@ Chain buildPushingToViewsDrain( { auto sink = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), context); metadata_snapshot->check(sink->getHeader().getColumnsWithTypeAndName()); - sink->setRuntimeData(runtime_data); + sink->setRuntimeData(thread_status, elapsed_counter_ms); result_chain.addSource(std::move(sink)); } @@ -597,13 +613,14 @@ static void logQueryViews(std::list & views, ContextPtr context for (auto & view : views) { - if ((min_query_duration && view.runtime_stats.elapsed_ms <= min_query_duration) || (view.runtime_stats.event_status < min_status)) + const auto & stats = *view.runtime_stats; + if ((min_query_duration && stats.elapsed_ms <= min_query_duration) || (stats.event_status < min_status)) continue; try { - if (view.runtime_stats.thread_status) - view.runtime_stats.thread_status->logToQueryViewsLog(view); + if (stats.thread_status) + stats.thread_status->logToQueryViewsLog(view); } catch (...) { diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index 96ccc0ac5b6..3f5092b7a2c 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -17,11 +17,6 @@ class Logger; namespace DB { -class ReplicatedMergeTreeSink; - -struct ExceptionKeepingTransformRuntimeData; -using ExceptionKeepingTransformRuntimeDataPtr = std::shared_ptr; - struct ViewRuntimeData { const ASTPtr query; @@ -34,12 +29,12 @@ struct ViewRuntimeData ContextPtr context; std::exception_ptr exception; - QueryViewsLogElement::ViewRuntimeStats runtime_stats; + std::unique_ptr runtime_stats; void setException(std::exception_ptr e) { exception = e; - runtime_stats.setStatus(QueryViewsLogElement::ViewStatus::EXCEPTION_WHILE_PROCESSING); + runtime_stats->setStatus(QueryViewsLogElement::ViewStatus::EXCEPTION_WHILE_PROCESSING); } }; @@ -51,8 +46,9 @@ Chain buildPushingToViewsDrain( ContextPtr context, const ASTPtr & query_ptr, bool no_destination, - ExceptionKeepingTransformRuntimeDataPtr runtime_data, - const Block & lv_storage = {}); + ThreadStatus * thread_status, + std::atomic_uint64_t * elapsed_counter_ms, + const Block & live_view_header = {}); class ExecutingInnerQueryFromViewTransform final : public ExceptionKeepingTransform diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index f8223b73383..837c82b8187 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -163,16 +163,19 @@ Chain InterpreterInsertQuery::buildChain( const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot, const Names & columns, - ExceptionKeepingTransformRuntimeDataPtr runtime_data) + ThreadStatus * thread_status, + std::atomic_uint64_t * elapsed_counter_ms) { - return buildChainImpl(table, metadata_snapshot, getSampleBlock(columns, table, metadata_snapshot), std::move(runtime_data)); + auto sample = getSampleBlock(columns, table, metadata_snapshot); + return buildChainImpl(table, metadata_snapshot, std::move(sample) , thread_status, elapsed_counter_ms); } Chain InterpreterInsertQuery::buildChainImpl( const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block, - ExceptionKeepingTransformRuntimeDataPtr runtime_data) + ThreadStatus * thread_status, + std::atomic_uint64_t * elapsed_counter_ms) { auto context_ptr = getContext(); const ASTInsertQuery * query = nullptr; @@ -190,12 +193,12 @@ Chain InterpreterInsertQuery::buildChainImpl( if (table->noPushingToViews() && !no_destination) { auto sink = table->write(query_ptr, metadata_snapshot, context_ptr); - sink->setRuntimeData(runtime_data); + sink->setRuntimeData(thread_status, elapsed_counter_ms); out.addSource(std::move(sink)); } else { - out = buildPushingToViewsDrain(table, metadata_snapshot, context_ptr, query_ptr, no_destination, runtime_data); + out = buildPushingToViewsDrain(table, metadata_snapshot, context_ptr, query_ptr, no_destination, thread_status, elapsed_counter_ms); } /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. @@ -233,7 +236,7 @@ Chain InterpreterInsertQuery::buildChainImpl( table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0)); } - auto counting = std::make_shared(out.getInputHeader(), runtime_data ? runtime_data->thread_status : nullptr); + auto counting = std::make_shared(out.getInputHeader(), thread_status); counting->setProcessListElement(context_ptr->getProcessListElement()); out.addSource(std::move(counting)); @@ -362,7 +365,7 @@ BlockIO InterpreterInsertQuery::execute() for (size_t i = 0; i < out_streams_size; i++) { - auto out = buildChainImpl(table, metadata_snapshot, query_sample_block); + auto out = buildChainImpl(table, metadata_snapshot, query_sample_block, nullptr, nullptr); out_chains.emplace_back(std::move(out)); } } diff --git a/src/Interpreters/InterpreterInsertQuery.h b/src/Interpreters/InterpreterInsertQuery.h index d08790ca774..0d6fe34c0c2 100644 --- a/src/Interpreters/InterpreterInsertQuery.h +++ b/src/Interpreters/InterpreterInsertQuery.h @@ -9,10 +9,8 @@ namespace DB { -struct ExceptionKeepingTransformRuntimeData; -using ExceptionKeepingTransformRuntimeDataPtr = std::shared_ptr; - class Chain; +class ThreadStatus; /** Interprets the INSERT query. */ @@ -40,7 +38,8 @@ public: const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot, const Names & columns, - ExceptionKeepingTransformRuntimeDataPtr runtime_data = nullptr); + ThreadStatus * thread_status = nullptr, + std::atomic_uint64_t * elapsed_counter_ms = nullptr); void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const override; @@ -60,7 +59,8 @@ private: const StoragePtr & table, const StorageMetadataPtr & metadata_snapshot, const Block & query_sample_block, - ExceptionKeepingTransformRuntimeDataPtr runtime_data = nullptr); + ThreadStatus * thread_status, + std::atomic_uint64_t * elapsed_counter_ms); }; diff --git a/src/Interpreters/QueryViewsLog.h b/src/Interpreters/QueryViewsLog.h index e751224a51e..34e7532d5d4 100644 --- a/src/Interpreters/QueryViewsLog.h +++ b/src/Interpreters/QueryViewsLog.h @@ -37,7 +37,7 @@ struct QueryViewsLogElement String target_name; ViewType type = ViewType::DEFAULT; std::unique_ptr thread_status = nullptr; - UInt64 elapsed_ms = 0; + std::atomic_uint64_t elapsed_ms = 0; std::chrono::time_point event_time; ViewStatus event_status = ViewStatus::QUERY_START; diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 52f9c6b6fbf..d3ebd5ee0b7 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -500,17 +500,17 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo) QueryViewsLogElement element; - element.event_time = time_in_seconds(vinfo.runtime_stats.event_time); - element.event_time_microseconds = time_in_microseconds(vinfo.runtime_stats.event_time); - element.view_duration_ms = vinfo.runtime_stats.elapsed_ms; + element.event_time = time_in_seconds(vinfo.runtime_stats->event_time); + element.event_time_microseconds = time_in_microseconds(vinfo.runtime_stats->event_time); + element.view_duration_ms = vinfo.runtime_stats->elapsed_ms; element.initial_query_id = query_id; element.view_name = vinfo.table_id.getFullTableName(); element.view_uuid = vinfo.table_id.uuid; - element.view_type = vinfo.runtime_stats.type; + element.view_type = vinfo.runtime_stats->type; if (vinfo.query) element.view_query = getCleanQueryAst(vinfo.query, query_context_ptr); - element.view_target = vinfo.runtime_stats.target_name; + element.view_target = vinfo.runtime_stats->target_name; auto events = std::make_shared(performance_counters.getPartiallyAtomicSnapshot()); element.read_rows = progress_in.read_rows.load(std::memory_order_relaxed); @@ -523,7 +523,7 @@ void ThreadStatus::logToQueryViewsLog(const ViewRuntimeData & vinfo) element.profile_counters = events; } - element.status = vinfo.runtime_stats.event_status; + element.status = vinfo.runtime_stats->event_status; element.exception_code = 0; if (vinfo.exception) { diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.h b/src/Processors/Executors/PushingAsyncPipelineExecutor.h index 5682769fe8f..4b4b83a90b5 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.h +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { diff --git a/src/Processors/Executors/PushingPipelineExecutor.h b/src/Processors/Executors/PushingPipelineExecutor.h index 86c9ec02290..b047e62415a 100644 --- a/src/Processors/Executors/PushingPipelineExecutor.h +++ b/src/Processors/Executors/PushingPipelineExecutor.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 0b4b2c308fe..e7127374228 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -16,7 +16,6 @@ #include #include #include - namespace DB { diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index ff4c80a55c0..976cfa1849f 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace DB { diff --git a/src/Processors/Transforms/ExceptionKeepingTransform.cpp b/src/Processors/Transforms/ExceptionKeepingTransform.cpp index 205f11de574..0c312d7cbf8 100644 --- a/src/Processors/Transforms/ExceptionKeepingTransform.cpp +++ b/src/Processors/Transforms/ExceptionKeepingTransform.cpp @@ -12,14 +12,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -ExceptionKeepingTransformRuntimeData::ExceptionKeepingTransformRuntimeData( - ThreadStatus * thread_status_, - std::string additional_exception_message_) - : thread_status(thread_status_) - , additional_exception_message(std::move(additional_exception_message_)) -{ -} - ExceptionKeepingTransform::ExceptionKeepingTransform(const Block & in_header, const Block & out_header, bool ignore_on_start_and_finish_) : IProcessor({in_header}, {out_header}) , input(inputs.front()), output(outputs.front()) @@ -87,58 +79,49 @@ IProcessor::Status ExceptionKeepingTransform::prepare() return Status::Ready; } -static std::exception_ptr runStep(std::function step, ExceptionKeepingTransformRuntimeData * runtime_data) +static std::exception_ptr runStep(std::function step, ThreadStatus * thread_status, std::atomic_uint64_t * elapsed_ms) { + std::exception_ptr res; + std::optional watch; + auto * original_thread = current_thread; SCOPE_EXIT({ current_thread = original_thread; }); - if (runtime_data && runtime_data->thread_status) + if (thread_status) { /// Change thread context to store individual metrics. Once the work in done, go back to the original thread - runtime_data->thread_status->resetPerformanceCountersLastUsage(); - current_thread = runtime_data->thread_status; + thread_status->resetPerformanceCountersLastUsage(); + current_thread = thread_status; } - std::exception_ptr res; - Stopwatch watch; + if (elapsed_ms) + watch.emplace(); try { step(); } - catch (Exception & exception) - { - // std::cerr << "===== got exception " << getExceptionMessage(exception, false); - if (runtime_data && !runtime_data->additional_exception_message.empty()) - exception.addMessage(runtime_data->additional_exception_message); - - res = std::current_exception(); - } catch (...) { - // std::cerr << "===== got exception " << getExceptionMessage(std::current_exception(), false); res = std::current_exception(); } - if (runtime_data) - { - if (runtime_data->thread_status) - runtime_data->thread_status->updatePerformanceCounters(); + if (thread_status) + thread_status->updatePerformanceCounters(); - runtime_data->elapsed_ms += watch.elapsedMilliseconds(); - } + if (elapsed_ms) + elapsed_ms += watch->elapsedMilliseconds(); return res; } void ExceptionKeepingTransform::work() { - // std::cerr << "============ Executing " << getName() << std::endl; if (!ignore_on_start_and_finish && !was_on_start_called) { was_on_start_called = true; - if (auto exception = runStep([this] { onStart(); }, runtime_data.get())) + if (auto exception = runStep([this] { onStart(); }, thread_status, elapsed_counter_ms)) { has_exception = true; ready_output = true; @@ -149,10 +132,8 @@ void ExceptionKeepingTransform::work() { ready_input = false; - if (auto exception = runStep([this] { transform(data.chunk); }, runtime_data.get())) + if (auto exception = runStep([this] { transform(data.chunk); }, thread_status, elapsed_counter_ms)) { - // std::cerr << "===== got exception in " << getName() << std::endl; - // std::cerr << getExceptionMessage(exception, true) << std::endl; has_exception = true; data.chunk.clear(); data.exception = std::move(exception); @@ -165,7 +146,7 @@ void ExceptionKeepingTransform::work() { was_on_finish_called = true; - if (auto exception = runStep([this] { onFinish(); }, runtime_data.get())) + if (auto exception = runStep([this] { onFinish(); }, thread_status, elapsed_counter_ms)) { has_exception = true; ready_output = true; @@ -174,4 +155,10 @@ void ExceptionKeepingTransform::work() } } +void ExceptionKeepingTransform::setRuntimeData(ThreadStatus * thread_status_, std::atomic_uint64_t * elapsed_counter_ms_) +{ + thread_status = thread_status_; + elapsed_counter_ms = elapsed_counter_ms_; +} + } diff --git a/src/Processors/Transforms/ExceptionKeepingTransform.h b/src/Processors/Transforms/ExceptionKeepingTransform.h index 943470b3f59..867f13bf53a 100644 --- a/src/Processors/Transforms/ExceptionKeepingTransform.h +++ b/src/Processors/Transforms/ExceptionKeepingTransform.h @@ -7,20 +7,6 @@ namespace DB class ThreadStatus; - -struct ExceptionKeepingTransformRuntimeData -{ - ThreadStatus * thread_status = nullptr; - UInt64 elapsed_ms = 0; - std::string additional_exception_message; - - ExceptionKeepingTransformRuntimeData( - ThreadStatus * thread_status_, - std::string additional_exception_message_); -}; - -using ExceptionKeepingTransformRuntimeDataPtr = std::shared_ptr; - /// Has one input and one output. /// Works similarly to ISimpleTransform, but with much care about exceptions. /// @@ -64,10 +50,11 @@ public: InputPort & getInputPort() { return input; } OutputPort & getOutputPort() { return output; } - void setRuntimeData(ExceptionKeepingTransformRuntimeDataPtr runtime_data_) { runtime_data = std::move(runtime_data_); } + void setRuntimeData(ThreadStatus * thread_status_, std::atomic_uint64_t * elapsed_counter_ms_); private: - ExceptionKeepingTransformRuntimeDataPtr runtime_data; + ThreadStatus * thread_status = nullptr; + std::atomic_uint64_t * elapsed_counter_ms = nullptr; }; } From db6f1e198cc04a280a86a74a18cdd38e7bc57589 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 21 Sep 2021 19:37:32 +0300 Subject: [PATCH 080/142] Add cancell callback to completed executor. --- .../Executors/CompletedPipelineExecutor.cpp | 48 ++++++++++++++++++- .../Executors/CompletedPipelineExecutor.h | 5 +- src/Processors/Executors/PipelineExecutor.cpp | 3 -- src/Server/TCPHandler.cpp | 1 + src/Storages/LiveView/LiveViewEventsSource.h | 2 +- src/Storages/LiveView/LiveViewSource.h | 5 +- 6 files changed, 55 insertions(+), 9 deletions(-) diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index ff7901c815d..e5922ad8a15 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -16,10 +17,55 @@ CompletedPipelineExecutor::CompletedPipelineExecutor(QueryPipeline & pipeline_) throw Exception(ErrorCodes::LOGICAL_ERROR, "Pipeline for CompletedPipelineExecutor must be completed"); } +void CompletedPipelineExecutor::setCancelCallback(std::function is_cancelled, size_t interactive_timeout_ms_) +{ + is_cancelled_callback = is_cancelled; + interactive_timeout_ms = interactive_timeout_ms_; +} + void CompletedPipelineExecutor::execute() { PipelineExecutor executor(pipeline.processors, pipeline.process_list_element); - executor.execute(pipeline.getNumThreads()); + + if (interactive_timeout_ms) + { + bool is_done = false; + std::mutex mutex; + std::exception_ptr exception; + ThreadFromGlobalPool thread([&]() + { + try + { + executor.execute(pipeline.getNumThreads()); + std::lock_guard lock(mutex); + } + catch (...) + { + exception = std::current_exception(); + } + is_done = true; + }); + + { + std::condition_variable condvar; + std::unique_lock lock(mutex); + while (!is_done) + { + condvar.wait_for(lock, std::chrono::milliseconds(interactive_timeout_ms), [&]() { return is_done; }); + + if (is_cancelled_callback()) + { + executor.cancel(); + is_done = true; + } + } + } + thread.join(); + if (exception) + std::rethrow_exception(exception); + } + else + executor.execute(pipeline.getNumThreads()); } } diff --git a/src/Processors/Executors/CompletedPipelineExecutor.h b/src/Processors/Executors/CompletedPipelineExecutor.h index d93e2ff5920..d20e80552f1 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.h +++ b/src/Processors/Executors/CompletedPipelineExecutor.h @@ -1,4 +1,5 @@ #pragma once +#include namespace DB { @@ -9,11 +10,13 @@ class CompletedPipelineExecutor { public: explicit CompletedPipelineExecutor(QueryPipeline & pipeline_); - + void setCancelCallback(std::function is_cancelled, size_t interactive_timeout_ms_); void execute(); private: QueryPipeline & pipeline; + std::function is_cancelled_callback; + size_t interactive_timeout_ms = 0; }; } diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index b91c1caa4a5..dcc5807fea5 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -429,9 +429,6 @@ void PipelineExecutor::execute(size_t num_threads) bool PipelineExecutor::executeStep(std::atomic_bool * yield_flag) { - if (finished) - return false; - if (!is_execution_initialized) initializeExecution(1); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 7fae4f08db0..1ff0844eb95 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -308,6 +308,7 @@ void TCPHandler::runImpl() else if (state.io.pipeline.completed()) { CompletedPipelineExecutor executor(state.io.pipeline); + executor.setCancelCallback([this]() { return isQueryCancelled(); }, interactive_delay / 1000); executor.execute(); } diff --git a/src/Storages/LiveView/LiveViewEventsSource.h b/src/Storages/LiveView/LiveViewEventsSource.h index daf9edfef95..77ee06c702c 100644 --- a/src/Storages/LiveView/LiveViewEventsSource.h +++ b/src/Storages/LiveView/LiveViewEventsSource.h @@ -56,7 +56,7 @@ public: void onCancel() override { - if (isCancelled() || storage->shutdown_called) + if (storage->shutdown_called) return; std::lock_guard lock(storage->mutex); diff --git a/src/Storages/LiveView/LiveViewSource.h b/src/Storages/LiveView/LiveViewSource.h index af07d8558ad..ec726359581 100644 --- a/src/Storages/LiveView/LiveViewSource.h +++ b/src/Storages/LiveView/LiveViewSource.h @@ -34,11 +34,11 @@ public: active = active_ptr.lock(); } - String getName() const override { return "LiveViewBlockInputStream"; } + String getName() const override { return "LiveViewSource"; } void onCancel() override { - if (isCancelled() || storage->shutdown_called) + if (storage->shutdown_called) return; std::lock_guard lock(storage->mutex); @@ -145,7 +145,6 @@ protected: /// Or spurious wakeup. bool signaled = std::cv_status::no_timeout == storage->condition.wait_for(lock, std::chrono::microseconds(std::max(UInt64(0), heartbeat_interval_usec - (timestamp_usec - last_event_timestamp_usec)))); - if (isCancelled() || storage->shutdown_called) { return { Block(), true }; From 912393a7ed400a564762bc945cdfa43f74c8a856 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 21 Sep 2021 19:55:32 +0300 Subject: [PATCH 081/142] Fix pvs check. --- src/Processors/Transforms/ExceptionKeepingTransform.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/ExceptionKeepingTransform.cpp b/src/Processors/Transforms/ExceptionKeepingTransform.cpp index 0c312d7cbf8..41fc2b7d17c 100644 --- a/src/Processors/Transforms/ExceptionKeepingTransform.cpp +++ b/src/Processors/Transforms/ExceptionKeepingTransform.cpp @@ -109,8 +109,8 @@ static std::exception_ptr runStep(std::function step, ThreadStatus * thr if (thread_status) thread_status->updatePerformanceCounters(); - if (elapsed_ms) - elapsed_ms += watch->elapsedMilliseconds(); + if (elapsed_ms && watch) + *elapsed_ms += watch->elapsedMilliseconds(); return res; } From 2c7239da66c7d807a2571d0c8219d5b2d0fdb26c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Sep 2021 09:49:59 +0300 Subject: [PATCH 082/142] Fix executor. --- .../Executors/CompletedPipelineExecutor.cpp | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index e5922ad8a15..48ab8c06b32 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include #include namespace DB @@ -32,17 +34,29 @@ void CompletedPipelineExecutor::execute() bool is_done = false; std::mutex mutex; std::exception_ptr exception; + auto thread_group = CurrentThread::getGroup(); + ThreadFromGlobalPool thread([&]() { + setThreadName("QueryPipelineEx"); + try { + if (thread_group) + CurrentThread::attachTo(thread_group); + + SCOPE_EXIT_SAFE( + if (thread_group) + CurrentThread::detachQueryIfNotDetached(); + ); + executor.execute(pipeline.getNumThreads()); - std::lock_guard lock(mutex); } catch (...) { exception = std::current_exception(); } + std::lock_guard lock(mutex); is_done = true; }); From d018cb7adfd516ea290b6b3436b027e5780032de Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Sep 2021 11:30:02 +0300 Subject: [PATCH 083/142] Fry fix executor again. --- .../Executors/CompletedPipelineExecutor.cpp | 108 +++++++++++------- .../Executors/CompletedPipelineExecutor.h | 3 + 2 files changed, 72 insertions(+), 39 deletions(-) diff --git a/src/Processors/Executors/CompletedPipelineExecutor.cpp b/src/Processors/Executors/CompletedPipelineExecutor.cpp index 48ab8c06b32..7dc9e50c24b 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.cpp +++ b/src/Processors/Executors/CompletedPipelineExecutor.cpp @@ -13,6 +13,48 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +struct CompletedPipelineExecutor::Data +{ + PipelineExecutorPtr executor; + std::exception_ptr exception; + std::atomic_bool is_finished = false; + std::atomic_bool has_exception = false; + ThreadFromGlobalPool thread; + Poco::Event finish_event; + + ~Data() + { + if (thread.joinable()) + thread.join(); + } +}; + +static void threadFunction(CompletedPipelineExecutor::Data & data, ThreadGroupStatusPtr thread_group, size_t num_threads) +{ + setThreadName("QueryPipelineEx"); + + try + { + if (thread_group) + CurrentThread::attachTo(thread_group); + + SCOPE_EXIT_SAFE( + if (thread_group) + CurrentThread::detachQueryIfNotDetached(); + ); + + data.executor->execute(num_threads); + } + catch (...) + { + data.exception = std::current_exception(); + data.has_exception = true; + } + + data.is_finished = true; + data.finish_event.set(); +} + CompletedPipelineExecutor::CompletedPipelineExecutor(QueryPipeline & pipeline_) : pipeline(pipeline_) { if (!pipeline.completed()) @@ -31,55 +73,43 @@ void CompletedPipelineExecutor::execute() if (interactive_timeout_ms) { - bool is_done = false; - std::mutex mutex; - std::exception_ptr exception; - auto thread_group = CurrentThread::getGroup(); + data = std::make_unique(); + data->executor = std::make_shared(pipeline.processors, pipeline.process_list_element); - ThreadFromGlobalPool thread([&]() + auto func = [&, thread_group = CurrentThread::getGroup()]() { - setThreadName("QueryPipelineEx"); + threadFunction(*data, thread_group, pipeline.getNumThreads()); + }; - try - { - if (thread_group) - CurrentThread::attachTo(thread_group); - - SCOPE_EXIT_SAFE( - if (thread_group) - CurrentThread::detachQueryIfNotDetached(); - ); - - executor.execute(pipeline.getNumThreads()); - } - catch (...) - { - exception = std::current_exception(); - } - std::lock_guard lock(mutex); - is_done = true; - }); + data->thread = ThreadFromGlobalPool(std::move(func)); + while (!data->is_finished) { - std::condition_variable condvar; - std::unique_lock lock(mutex); - while (!is_done) - { - condvar.wait_for(lock, std::chrono::milliseconds(interactive_timeout_ms), [&]() { return is_done; }); + if (data->finish_event.tryWait(interactive_timeout_ms)) + break; - if (is_cancelled_callback()) - { - executor.cancel(); - is_done = true; - } - } + if (is_cancelled_callback()) + data->executor->cancel(); } - thread.join(); - if (exception) - std::rethrow_exception(exception); + + if (data->has_exception) + std::rethrow_exception(data->exception); } else executor.execute(pipeline.getNumThreads()); } +CompletedPipelineExecutor::~CompletedPipelineExecutor() +{ + try + { + if (data && data->executor) + data->executor->cancel(); + } + catch (...) + { + tryLogCurrentException("PullingAsyncPipelineExecutor"); + } +} + } diff --git a/src/Processors/Executors/CompletedPipelineExecutor.h b/src/Processors/Executors/CompletedPipelineExecutor.h index d20e80552f1..d46dea0d038 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.h +++ b/src/Processors/Executors/CompletedPipelineExecutor.h @@ -10,13 +10,16 @@ class CompletedPipelineExecutor { public: explicit CompletedPipelineExecutor(QueryPipeline & pipeline_); + ~CompletedPipelineExecutor(); void setCancelCallback(std::function is_cancelled, size_t interactive_timeout_ms_); void execute(); + struct Data; private: QueryPipeline & pipeline; std::function is_cancelled_callback; size_t interactive_timeout_ms = 0; + std::unique_ptr data; }; } From 81bf13a247dd104a106909020b710309ceaedc39 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Sep 2021 13:57:00 +0300 Subject: [PATCH 084/142] Refactor pushing to views. --- .../PushingToViewsBlockOutputStream.cpp | 509 ++++++++++-------- .../PushingToViewsBlockOutputStream.h | 49 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- .../Executors/CompletedPipelineExecutor.h | 7 + src/Server/TCPHandler.cpp | 4 +- 5 files changed, 302 insertions(+), 269 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index af2509df19e..c5a3ea0fbc8 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -32,11 +32,14 @@ namespace ErrorCodes struct ViewsData { + /// Separate information for every view. std::list views; + /// Some common info about source storage. ContextPtr context; StorageID source_storage_id; StorageMetadataPtr source_metadata_snapshot; StoragePtr source_storage; + /// This value is actually only for logs. size_t max_threads = 1; /// In case of exception happened while inserting into main table, it is pushed to pipeline. @@ -55,68 +58,14 @@ struct ViewsData using ViewsDataPtr = std::shared_ptr; +/// Copies data inserted into table for every dependent table. class CopyingDataToViewsTransform final : public IProcessor { public: - CopyingDataToViewsTransform(const Block & header, ViewsDataPtr data) - : IProcessor({header}, OutputPorts(data->views.size(), header)) - , input(inputs.front()) - , views_data(std::move(data)) - { - if (views_data->views.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "CopyingDataToViewsTransform cannot have zero outputs"); - } + CopyingDataToViewsTransform(const Block & header, ViewsDataPtr data); String getName() const override { return "CopyingDataToViewsTransform"; } - - Status prepare() override - { - bool all_can_push = true; - for (auto & output : outputs) - { - if (output.isFinished()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot push data to view because output port is finished"); - - if (!output.canPush()) - all_can_push = false; - } - - if (!all_can_push) - return Status::PortFull; - - if (input.isFinished()) - { - for (auto & output : outputs) - output.finish(); - - return Status::Finished; - } - - input.setNeeded(); - if (!input.hasData()) - return Status::NeedData; - - auto data = input.pullData(); - if (data.exception) - { - if (!views_data->has_exception) - { - views_data->first_exception = data.exception; - views_data->has_exception = true; - } - - for (auto & output : outputs) - output.pushException(data.exception); - } - else - { - for (auto & output : outputs) - output.push(data.chunk.clone()); - } - - return Status::PortFull; - } - + Status prepare() override; InputPort & getInputPort() { return input; } private: @@ -124,27 +73,40 @@ private: ViewsDataPtr views_data; }; -static void logQueryViews(std::list & views, ContextPtr context); - -static std::exception_ptr addStorageToException(std::exception_ptr ptr, const StorageID & storage) +/// For source chunk, execute view query over it. +class ExecutingInnerQueryFromViewTransform final : public ExceptionKeepingTransform { - try - { - std::rethrow_exception(std::move(ptr)); - } - catch (DB::Exception & exception) - { - exception.addMessage("while inserting into {}", storage.getNameForLogs()); - return std::current_exception(); - } - catch (...) - { - return std::current_exception(); - } +public: + ExecutingInnerQueryFromViewTransform(const Block & header, ViewRuntimeData & view_, ViewsDataPtr views_data_); - __builtin_unreachable(); -} + String getName() const override { return "ExecutingInnerQueryFromView"; } +protected: + void transform(Chunk & chunk) override; + +private: + ViewsDataPtr views_data; + ViewRuntimeData & view; +}; + +/// Insert into LiveView. +class PushingToLiveViewSink final : public SinkToStorage +{ +public: + PushingToLiveViewSink(const Block & header, StorageLiveView & live_view_, StoragePtr storage_holder_, ContextPtr context_); + String getName() const override { return "PushingToLiveViewSink"; } + void consume(Chunk chunk) override; + +private: + StorageLiveView & live_view; + StoragePtr storage_holder; + ContextPtr context; +}; + +/// For every view, collect exception. +/// Has single output with empty header. +/// If any exception happen before view processing, pass it. +/// Othervise return any exception from any view. class FinalizingViewsTransform final : public IProcessor { struct ExceptionStatus @@ -153,119 +115,14 @@ class FinalizingViewsTransform final : public IProcessor bool is_first = false; }; - static InputPorts initPorts(std::vector headers) - { - InputPorts res; - for (auto & header : headers) - res.emplace_back(std::move(header)); - - return res; - } + static InputPorts initPorts(std::vector headers); public: - FinalizingViewsTransform(std::vector headers, ViewsDataPtr data) - : IProcessor(initPorts(std::move(headers)), {Block()}) - , output(outputs.front()) - , views_data(std::move(data)) - { - statuses.resize(views_data->views.size()); - } + FinalizingViewsTransform(std::vector headers, ViewsDataPtr data); String getName() const override { return "FinalizingViewsTransform"; } - - Status prepare() override - { - if (output.isFinished()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot finalize views because output port is finished"); - - if (!output.canPush()) - return Status::PortFull; - - size_t num_finished = 0; - size_t pos = 0; - for (auto & input : inputs) - { - auto i = pos; - ++pos; - - if (input.isFinished()) - { - ++num_finished; - continue; - } - - input.setNeeded(); - if (input.hasData()) - { - auto data = input.pullData(); - //std::cerr << "********** FinalizingViewsTransform got input " << i << " has exc " << bool(data.exception) << std::endl; - if (data.exception) - { - if (views_data->has_exception && views_data->first_exception == data.exception) - statuses[i].is_first = true; - else - statuses[i].exception = data.exception; - - if (i == 0 && statuses[0].is_first) - { - output.pushData(std::move(data)); - return Status::PortFull; - } - } - - if (input.isFinished()) - ++num_finished; - } - } - - if (num_finished == inputs.size()) - { - if (!statuses.empty()) - return Status::Ready; - - if (any_exception) - output.pushException(std::move(any_exception)); - - output.finish(); - return Status::Finished; - } - - return Status::NeedData; - } - - void work() override - { - size_t i = 0; - for (auto & view : views_data->views) - { - auto & status = statuses[i]; - ++i; - - if (status.exception) - { - if (!any_exception) - any_exception = status.exception; - - view.setException(addStorageToException(std::move(status.exception), view.table_id)); - } - else - { - view.runtime_stats->setStatus(QueryViewsLogElement::ViewStatus::QUERY_FINISH); - - LOG_TRACE( - &Poco::Logger::get("PushingToViews"), - "Pushing ({}) from {} to {} took {} ms.", - views_data->max_threads <= 1 ? "sequentially" : ("parallel " + std::to_string(views_data->max_threads)), - views_data->source_storage_id.getNameForLogs(), - view.table_id.getNameForLogs(), - view.runtime_stats->elapsed_ms); - } - } - - logQueryViews(views_data->views, views_data->context); - - statuses.clear(); - } + Status prepare() override; + void work() override; private: OutputPort & output; @@ -274,33 +131,8 @@ private: std::exception_ptr any_exception; }; -class PushingToLiveViewSink final : public SinkToStorage -{ -public: - PushingToLiveViewSink(const Block & header, StorageLiveView & live_view_, StoragePtr storage_holder_, ContextPtr context_) - : SinkToStorage(header) - , live_view(live_view_) - , storage_holder(std::move(storage_holder_)) - , context(std::move(context_)) - { - } - String getName() const override { return "PushingToLiveViewSink"; } - - void consume(Chunk chunk) override - { - Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); - StorageLiveView::writeIntoLiveView(live_view, getHeader().cloneWithColumns(chunk.detachColumns()), context); - CurrentThread::updateProgressIn(local_progress); - } - -private: - StorageLiveView & live_view; - StoragePtr storage_holder; - ContextPtr context; -}; - -Chain buildPushingToViewsDrain( +Chain buildPushingToViewsChain( const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, @@ -336,6 +168,7 @@ Chain buildPushingToViewsDrain( /// We need special context for materialized views insertions ContextMutablePtr select_context; ContextMutablePtr insert_context; + ViewsDataPtr views_data; if (!dependencies.empty()) { select_context = Context::createCopy(context); @@ -352,10 +185,11 @@ Chain buildPushingToViewsDrain( insert_context->setSetting("min_insert_block_size_rows", insert_settings.min_insert_block_size_rows_for_materialized_views.value); if (insert_settings.min_insert_block_size_bytes_for_materialized_views) insert_context->setSetting("min_insert_block_size_bytes", insert_settings.min_insert_block_size_bytes_for_materialized_views.value); + + views_data = std::make_shared(select_context, table_id, metadata_snapshot, storage); } std::vector chains; - auto views_data = std::make_shared(context, table_id, metadata_snapshot, storage); for (const auto & database_table : dependencies) { @@ -434,27 +268,24 @@ Chain buildPushingToViewsDrain( { runtime_stats->type = QueryViewsLogElement::ViewType::LIVE; query = live_view->getInnerQuery(); // Used only to log in system.query_views_log - out = buildPushingToViewsDrain( + out = buildPushingToViewsChain( dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true, view_thread_status, view_counter_ms, storage_header); } else - out = buildPushingToViewsDrain( + out = buildPushingToViewsChain( dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), false, view_thread_status, view_counter_ms); views_data->views.emplace_back(ViewRuntimeData{ std::move(query), out.getInputHeader(), database_table, - dependent_table, - dependent_metadata_snapshot, - select_context, nullptr, std::move(runtime_stats)}); if (type == QueryViewsLogElement::ViewType::MATERIALIZED) { auto executing_inner_query = std::make_shared( - storage_header, views_data->views.back(), views_data->source_storage_id, views_data->source_metadata_snapshot, views_data->source_storage); + storage_header, views_data->views.back(), views_data); executing_inner_query->setRuntimeData(view_thread_status, elapsed_counter_ms); out.addSource(std::move(executing_inner_query)); @@ -470,9 +301,9 @@ Chain buildPushingToViewsDrain( } } - size_t num_views = views_data->views.size(); - if (num_views != 0) + if (views_data) { + size_t num_views = views_data->views.size(); const Settings & settings = context->getSettingsRef(); if (settings.parallel_view_processing) views_data->max_threads = settings.max_threads ? std::min(static_cast(settings.max_threads), num_views) : num_views; @@ -530,19 +361,19 @@ Chain buildPushingToViewsDrain( return result_chain; } -static void process(Block & block, ViewRuntimeData & view, const StorageID & source_storage_id, const StorageMetadataPtr & source_metadata_snapshot, const StoragePtr & source_storage) +static void process(Block & block, ViewRuntimeData & view, const ViewsData & views_data) { - const auto & context = view.context; + const auto & context = views_data.context; /// We create a table with the same name as original table and the same alias columns, /// but it will contain single block (that is INSERT-ed into main table). /// InterpreterSelectQuery will do processing of alias columns. auto local_context = Context::createCopy(context); local_context->addViewSource(StorageValues::create( - source_storage_id, - source_metadata_snapshot->getColumns(), + views_data.source_storage_id, + views_data.source_metadata_snapshot->getColumns(), block, - source_storage->getVirtuals())); + views_data.source_storage->getVirtuals())); /// We need keep InterpreterSelectQuery, until the processing will be finished, since: /// @@ -596,13 +427,6 @@ static void process(Block & block, ViewRuntimeData & view, const StorageID & sou throw Exception(ErrorCodes::LOGICAL_ERROR, "Single chunk is expected from view inner query {}", view.query); } -void ExecutingInnerQueryFromViewTransform::transform(Chunk & chunk) -{ - auto block = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); - process(block, view, source_storage_id, source_metadata_snapshot, source_storage); - chunk.setColumns(block.getColumns(), block.rows()); -} - static void logQueryViews(std::list & views, ContextPtr context) { const auto & settings = context->getSettingsRef(); @@ -629,4 +453,227 @@ static void logQueryViews(std::list & views, ContextPtr context } } + +CopyingDataToViewsTransform::CopyingDataToViewsTransform(const Block & header, ViewsDataPtr data) + : IProcessor({header}, OutputPorts(data->views.size(), header)) + , input(inputs.front()) + , views_data(std::move(data)) +{ + if (views_data->views.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "CopyingDataToViewsTransform cannot have zero outputs"); +} + +IProcessor::Status CopyingDataToViewsTransform::prepare() +{ + bool all_can_push = true; + for (auto & output : outputs) + { + if (output.isFinished()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot push data to view because output port is finished"); + + if (!output.canPush()) + all_can_push = false; + } + + if (!all_can_push) + return Status::PortFull; + + if (input.isFinished()) + { + for (auto & output : outputs) + output.finish(); + + return Status::Finished; + } + + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + + auto data = input.pullData(); + if (data.exception) + { + if (!views_data->has_exception) + { + views_data->first_exception = data.exception; + views_data->has_exception = true; + } + + for (auto & output : outputs) + output.pushException(data.exception); + } + else + { + for (auto & output : outputs) + output.push(data.chunk.clone()); + } + + return Status::PortFull; +} + + +ExecutingInnerQueryFromViewTransform::ExecutingInnerQueryFromViewTransform( + const Block & header, + ViewRuntimeData & view_, + std::shared_ptr views_data_) + : ExceptionKeepingTransform(header, view_.sample_block) + , views_data(std::move(views_data_)) + , view(view_) +{ +} + +void ExecutingInnerQueryFromViewTransform::transform(Chunk & chunk) +{ + auto block = getInputPort().getHeader().cloneWithColumns(chunk.getColumns()); + process(block, view, *views_data); + chunk.setColumns(block.getColumns(), block.rows()); +} + + +PushingToLiveViewSink::PushingToLiveViewSink(const Block & header, StorageLiveView & live_view_, StoragePtr storage_holder_, ContextPtr context_) + : SinkToStorage(header) + , live_view(live_view_) + , storage_holder(std::move(storage_holder_)) + , context(std::move(context_)) +{ +} + +void PushingToLiveViewSink::consume(Chunk chunk) +{ + Progress local_progress(chunk.getNumRows(), chunk.bytes(), 0); + StorageLiveView::writeIntoLiveView(live_view, getHeader().cloneWithColumns(chunk.detachColumns()), context); + CurrentThread::updateProgressIn(local_progress); +} + + +FinalizingViewsTransform::FinalizingViewsTransform(std::vector headers, ViewsDataPtr data) + : IProcessor(initPorts(std::move(headers)), {Block()}) + , output(outputs.front()) + , views_data(std::move(data)) +{ + statuses.resize(views_data->views.size()); +} + +InputPorts FinalizingViewsTransform::initPorts(std::vector headers) +{ + InputPorts res; + for (auto & header : headers) + res.emplace_back(std::move(header)); + + return res; +} + +IProcessor::Status FinalizingViewsTransform::prepare() +{ + if (output.isFinished()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot finalize views because output port is finished"); + + if (!output.canPush()) + return Status::PortFull; + + size_t num_finished = 0; + size_t pos = 0; + for (auto & input : inputs) + { + auto i = pos; + ++pos; + + if (input.isFinished()) + { + ++num_finished; + continue; + } + + input.setNeeded(); + if (input.hasData()) + { + auto data = input.pullData(); + //std::cerr << "********** FinalizingViewsTransform got input " << i << " has exc " << bool(data.exception) << std::endl; + if (data.exception) + { + if (views_data->has_exception && views_data->first_exception == data.exception) + statuses[i].is_first = true; + else + statuses[i].exception = data.exception; + + if (i == 0 && statuses[0].is_first) + { + output.pushData(std::move(data)); + return Status::PortFull; + } + } + + if (input.isFinished()) + ++num_finished; + } + } + + if (num_finished == inputs.size()) + { + if (!statuses.empty()) + return Status::Ready; + + if (any_exception) + output.pushException(std::move(any_exception)); + + output.finish(); + return Status::Finished; + } + + return Status::NeedData; +} + +static std::exception_ptr addStorageToException(std::exception_ptr ptr, const StorageID & storage) +{ + try + { + std::rethrow_exception(std::move(ptr)); + } + catch (DB::Exception & exception) + { + exception.addMessage("while inserting into {}", storage.getNameForLogs()); + return std::current_exception(); + } + catch (...) + { + return std::current_exception(); + } + + __builtin_unreachable(); +} + +void FinalizingViewsTransform::work() +{ + size_t i = 0; + for (auto & view : views_data->views) + { + auto & status = statuses[i]; + ++i; + + if (status.exception) + { + if (!any_exception) + any_exception = status.exception; + + view.setException(addStorageToException(std::move(status.exception), view.table_id)); + } + else + { + view.runtime_stats->setStatus(QueryViewsLogElement::ViewStatus::QUERY_FINISH); + + LOG_TRACE( + &Poco::Logger::get("PushingToViews"), + "Pushing ({}) from {} to {} took {} ms.", + views_data->max_threads <= 1 ? "sequentially" : ("parallel " + std::to_string(views_data->max_threads)), + views_data->source_storage_id.getNameForLogs(), + view.table_id.getNameForLogs(), + view.runtime_stats->elapsed_ms); + } + } + + logQueryViews(views_data->views, views_data->context); + + statuses.clear(); +} + } diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.h b/src/DataStreams/PushingToViewsBlockOutputStream.h index 3f5092b7a2c..75d0528ff7b 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -19,16 +19,17 @@ namespace DB struct ViewRuntimeData { + /// A query we should run over inserted block befire pushing into inner storage. const ASTPtr query; + /// This structure is expected by inner storage. Will convert query result to it. Block sample_block; - + /// Inner storage id. StorageID table_id; - StoragePtr storage; - StorageMetadataPtr metadata_snapshot; - - ContextPtr context; + /// In case of exception at any step (e.g. query execution or insertion into inner table) + /// exception is stored here (will be stored in query views log). std::exception_ptr exception; + /// Info which is needed for query views log. std::unique_ptr runtime_stats; void setException(std::exception_ptr e) @@ -40,44 +41,20 @@ struct ViewRuntimeData /** Writes data to the specified table and to all dependent materialized views. */ -Chain buildPushingToViewsDrain( +Chain buildPushingToViewsChain( const StoragePtr & storage, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, const ASTPtr & query_ptr, + /// It is true when we should not insert into table, but only to views. + /// Used e.g. for kafka. We should try to remove it somehow. bool no_destination, + /// We could specify separate thread_status for each view. + /// Needed mainly to collect counters separately. Should be improved. ThreadStatus * thread_status, + /// Counter to measure time spent separately per view. Should be improved. std::atomic_uint64_t * elapsed_counter_ms, + /// LiveView executes query itself, it needs source block structure. const Block & live_view_header = {}); - -class ExecutingInnerQueryFromViewTransform final : public ExceptionKeepingTransform -{ -public: - ExecutingInnerQueryFromViewTransform( - const Block & header, - ViewRuntimeData & view_data, - const StorageID & source_storage_id_, - const StorageMetadataPtr & source_metadata_snapshot_, - const StoragePtr & source_storage_) - : ExceptionKeepingTransform(header, view_data.sample_block) - , view(view_data) - , source_storage_id(source_storage_id_) - , source_metadata_snapshot(source_metadata_snapshot_) - , source_storage(source_storage_) - { - } - - String getName() const override { return "ExecutingInnerQueryFromView"; } - -protected: - void transform(Chunk & chunk) override; - -private: - ViewRuntimeData & view; - const StorageID & source_storage_id; - const StorageMetadataPtr & source_metadata_snapshot; - const StoragePtr & source_storage; -}; - } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 837c82b8187..31a156f0850 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -198,7 +198,7 @@ Chain InterpreterInsertQuery::buildChainImpl( } else { - out = buildPushingToViewsDrain(table, metadata_snapshot, context_ptr, query_ptr, no_destination, thread_status, elapsed_counter_ms); + out = buildPushingToViewsChain(table, metadata_snapshot, context_ptr, query_ptr, no_destination, thread_status, elapsed_counter_ms); } /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. diff --git a/src/Processors/Executors/CompletedPipelineExecutor.h b/src/Processors/Executors/CompletedPipelineExecutor.h index d46dea0d038..6e82c7547a6 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.h +++ b/src/Processors/Executors/CompletedPipelineExecutor.h @@ -6,12 +6,19 @@ namespace DB class QueryPipeline; +/// Executor for completed QueryPipeline. +/// Allows to specify a callback which checks if execution should be cancelled. +/// If callback is specified, runs execution in a separate thread. class CompletedPipelineExecutor { public: explicit CompletedPipelineExecutor(QueryPipeline & pipeline_); ~CompletedPipelineExecutor(); + + /// This callback will be called each interactive_timeout_ms (if it is not 0). + /// If returns true, query would be cancelled. void setCancelCallback(std::function is_cancelled, size_t interactive_timeout_ms_); + void execute(); struct Data; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 1ff0844eb95..185198c4911 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -308,7 +308,9 @@ void TCPHandler::runImpl() else if (state.io.pipeline.completed()) { CompletedPipelineExecutor executor(state.io.pipeline); - executor.setCancelCallback([this]() { return isQueryCancelled(); }, interactive_delay / 1000); + /// Should not check for cancel in case of input. + if (!state.need_receive_data_for_input) + executor.setCancelCallback([this]() { return isQueryCancelled(); }, interactive_delay / 1000); executor.execute(); } From eed4e8c754f3a96a0aaf5d4dabc3ddcbe88651d3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Sep 2021 16:29:58 +0300 Subject: [PATCH 085/142] Fix progress for insert select. --- .../Executors/CompletedPipelineExecutor.h | 1 + src/Server/TCPHandler.cpp | 15 ++++++++++++++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/src/Processors/Executors/CompletedPipelineExecutor.h b/src/Processors/Executors/CompletedPipelineExecutor.h index 6e82c7547a6..e616cd6a2b7 100644 --- a/src/Processors/Executors/CompletedPipelineExecutor.h +++ b/src/Processors/Executors/CompletedPipelineExecutor.h @@ -1,5 +1,6 @@ #pragma once #include +#include namespace DB { diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 185198c4911..adb66ae924b 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -310,7 +310,20 @@ void TCPHandler::runImpl() CompletedPipelineExecutor executor(state.io.pipeline); /// Should not check for cancel in case of input. if (!state.need_receive_data_for_input) - executor.setCancelCallback([this]() { return isQueryCancelled(); }, interactive_delay / 1000); + { + auto callback = [this]() + { + if (isQueryCancelled()) + return true; + + sendProgress(); + sendLogs(); + + return false; + }; + + executor.setCancelCallback(callback, interactive_delay / 1000); + } executor.execute(); } From 4f802d1cea543dcfeb1b9efdb5fca0491a0c3fd0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Sep 2021 22:31:12 +0300 Subject: [PATCH 086/142] Fix test_executable_table_function --- .../PushingToViewsBlockOutputStream.cpp | 1 + .../MySQL/MaterializedMySQLSyncThread.cpp | 2 +- src/Processors/Executors/PipelineExecutor.cpp | 5 +++ src/Processors/QueryPipeline.cpp | 34 ++++++++++++++++++- src/Processors/QueryPipeline.h | 3 +- src/Storages/StorageExecutable.cpp | 9 ++--- src/Storages/StorageLog.cpp | 5 +-- 7 files changed, 48 insertions(+), 11 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index c5a3ea0fbc8..008a00f1658 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -275,6 +275,7 @@ Chain buildPushingToViewsChain( out = buildPushingToViewsChain( dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), false, view_thread_status, view_counter_ms); + assert(views_data != nullptr); views_data->views.emplace_back(ViewRuntimeData{ std::move(query), out.getInputHeader(), diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index ecb212240ed..772fb10f914 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -339,7 +339,7 @@ static inline void dumpDataForTables( pipeline.getHeader(), mysql_input_stream_settings); auto counting = std::make_shared(pipeline.getHeader()); Pipe pipe(std::move(input)); - pipe.addTransform(std::move(counting)); + pipe.addTransform(counting); pipeline.complete(std::move(pipe)); Stopwatch watch; diff --git a/src/Processors/Executors/PipelineExecutor.cpp b/src/Processors/Executors/PipelineExecutor.cpp index dcc5807fea5..d19eefeb53c 100644 --- a/src/Processors/Executors/PipelineExecutor.cpp +++ b/src/Processors/Executors/PipelineExecutor.cpp @@ -430,8 +430,13 @@ void PipelineExecutor::execute(size_t num_threads) bool PipelineExecutor::executeStep(std::atomic_bool * yield_flag) { if (!is_execution_initialized) + { initializeExecution(1); + if (yield_flag && *yield_flag) + return true; + } + executeStepImpl(0, 1, yield_flag); if (!finished) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index e7127374228..1b3e790133b 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -307,6 +307,38 @@ QueryPipeline::QueryPipeline(Chain chain) input = &chain.getInputPort(); } +QueryPipeline::QueryPipeline(std::shared_ptr format) +{ + auto & format_main = format->getPort(IOutputFormat::PortKind::Main); + auto & format_totals = format->getPort(IOutputFormat::PortKind::Totals); + auto & format_extremes = format->getPort(IOutputFormat::PortKind::Extremes); + + if (!totals) + { + auto source = std::make_shared(format_totals.getHeader()); + totals = &source->getPort(); + processors.emplace_back(std::move(source)); + } + + if (!extremes) + { + auto source = std::make_shared(format_extremes.getHeader()); + extremes = &source->getPort(); + processors.emplace_back(std::move(source)); + } + + connect(*totals, format_totals); + connect(*extremes, format_extremes); + + input = &format_main; + totals = nullptr; + extremes = nullptr; + + output_format = format.get(); + + processors.emplace_back(std::move(format)); +} + static void drop(OutputPort *& port, Processors & processors) { if (!port) @@ -489,7 +521,7 @@ void QueryPipeline::setLimitsAndQuota(const StreamLocalLimits & limits, std::sha } -bool QueryPipeline::tryGetResultRowsAndBytes(size_t & result_rows, size_t & result_bytes) const +bool QueryPipeline::tryGetResultRowsAndBytes(UInt64 & result_rows, UInt64 & result_bytes) const { if (!output_format) return false; diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 976cfa1849f..42fdb429a14 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -46,6 +46,7 @@ public: /// pushing explicit QueryPipeline(Chain chain); explicit QueryPipeline(std::shared_ptr sink); + explicit QueryPipeline(std::shared_ptr format); /// completed QueryPipeline( @@ -92,7 +93,7 @@ public: void setProcessListElement(QueryStatus * elem); void setProgressCallback(const ProgressCallback & callback); void setLimitsAndQuota(const StreamLocalLimits & limits, std::shared_ptr quota); - bool tryGetResultRowsAndBytes(size_t & result_rows, size_t & result_bytes) const; + bool tryGetResultRowsAndBytes(UInt64 & result_rows, UInt64 & result_bytes) const; void addStorageHolder(StoragePtr storage); diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index bbd96f89b82..989cc3a1f91 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include @@ -183,12 +184,12 @@ Pipe StorageExecutable::read( inputs[i].addTransform(std::move(transform)); } - auto out = FormatFactory::instance().getOutputFormat(format, *write_buffer, inputs[i].getHeader(), context); - out->setAutoFlush(); - inputs[i].setOutputFormat(std::move(out)); - auto pipeline = std::make_shared(QueryPipelineBuilder::getPipeline(std::move(inputs[i]))); + auto out = FormatFactory::instance().getOutputFormat(format, *write_buffer, materializeBlock(pipeline->getHeader()), context); + out->setAutoFlush(); + pipeline->complete(std::move(out)); + ShellCommandSource::SendDataTask task = [pipeline, write_buffer, is_executable_pool]() { CompletedPipelineExecutor executor(*pipeline); diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 61dacf9b2ee..0fd94bac95a 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -463,10 +463,7 @@ void LogSink::writeMarks(MarksForColumns && marks) } } -StorageLog::~StorageLog() -{ - // std::cerr << "======================\n" << StackTrace().toString() << std::endl; -} +StorageLog::~StorageLog() = default; StorageLog::StorageLog( DiskPtr disk_, From cb1ca9b33e178aa3356a1ef07a58b89b07ff3b77 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Mon, 20 Sep 2021 14:06:19 +0300 Subject: [PATCH 087/142] Implemented modifying table comments with `ALTER TABLE t MODIFY COMMENT 'value'` --- src/Access/AccessType.h | 1 + src/Databases/DatabaseOnDisk.cpp | 5 + src/Interpreters/InterpreterAlterQuery.cpp | 5 + src/Parsers/ASTAlterQuery.cpp | 11 ++ src/Parsers/ASTAlterQuery.h | 2 + src/Parsers/IAST.cpp | 9 + src/Parsers/IAST.h | 2 + src/Parsers/ParserAlterQuery.cpp | 8 + src/Parsers/ParserAlterQuery.h | 1 + src/Parsers/tests/gtest_Parser.cpp | 186 ++++++++++-------- src/Storages/AlterCommands.cpp | 15 +- src/Storages/AlterCommands.h | 3 +- src/Storages/StorageNull.cpp | 3 +- .../2020_alter_table_modify_comment.reference | 85 ++++++++ .../2020_alter_table_modify_comment.sh | 50 +++++ 15 files changed, 296 insertions(+), 90 deletions(-) create mode 100644 tests/queries/0_stateless/2020_alter_table_modify_comment.reference create mode 100755 tests/queries/0_stateless/2020_alter_table_modify_comment.sh diff --git a/src/Access/AccessType.h b/src/Access/AccessType.h index d031606616d..3e1597dc890 100644 --- a/src/Access/AccessType.h +++ b/src/Access/AccessType.h @@ -45,6 +45,7 @@ enum class AccessType M(ALTER_RENAME_COLUMN, "RENAME COLUMN", COLUMN, ALTER_COLUMN) \ M(ALTER_MATERIALIZE_COLUMN, "MATERIALIZE COLUMN", COLUMN, ALTER_COLUMN) \ M(ALTER_COLUMN, "", GROUP, ALTER_TABLE) /* allow to execute ALTER {ADD|DROP|MODIFY...} COLUMN */\ + M(ALTER_MODIFY_COMMENT, "MODIFY COMMENT", TABLE, ALTER_TABLE) /* modify table comment */\ \ M(ALTER_ORDER_BY, "ALTER MODIFY ORDER BY, MODIFY ORDER BY", TABLE, ALTER_INDEX) \ M(ALTER_SAMPLE_BY, "ALTER MODIFY SAMPLE BY, MODIFY SAMPLE BY", TABLE, ALTER_INDEX) \ diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index e941e18625d..9d1bcb51dba 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -187,6 +187,11 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo if (metadata.settings_changes) storage_ast.set(storage_ast.settings, metadata.settings_changes); } + + if (metadata.comment.empty()) + storage_ast.reset(storage_ast.comment); + else + storage_ast.set(storage_ast.comment, std::make_shared(metadata.comment)); } } diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 6595e1c02be..8b62b6048ed 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -411,6 +411,11 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS break; } case ASTAlterCommand::NO_TYPE: break; + case ASTAlterCommand::MODIFY_COMMENT: + { + required_access.emplace_back(AccessType::ALTER_MODIFY_COMMENT, database, table); + break; + } } return required_access; diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 1aa116fefc4..06a0b6d46f5 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -67,6 +67,11 @@ ASTPtr ASTAlterCommand::clone() const res->rename_to = rename_to->clone(); res->children.push_back(res->rename_to); } + if (comment) + { + res->comment = comment->clone(); + res->children.push_back(res->comment); + } return res; } @@ -138,6 +143,12 @@ void ASTAlterCommand::formatImpl( settings.ostr << " " << (settings.hilite ? hilite_none : ""); comment->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::MODIFY_COMMENT) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COMMENT" << (settings.hilite ? hilite_none : ""); + settings.ostr << " " << (settings.hilite ? hilite_none : ""); + comment->formatImpl(settings, state, frame); + } else if (type == ASTAlterCommand::MODIFY_ORDER_BY) { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY ORDER BY " << (settings.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 7c301d581e6..9b40586e09f 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -72,6 +72,8 @@ public: LIVE_VIEW_REFRESH, MODIFY_DATABASE_SETTING, + + MODIFY_COMMENT, }; Type type = NO_TYPE; diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 3a21d704eb9..2ca72007285 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -47,6 +47,15 @@ size_t IAST::checkSize(size_t max_size) const return res; } +void IAST::reset(IAST *& field) +{ + std::find_if(children.begin(), children.end(), [field](const auto & p) + { + return p.get() == field; + }); + field = nullptr; +} + IAST::Hash IAST::getTreeHash() const { diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 2f9212da632..88736e8137e 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -157,6 +157,8 @@ public: set(field, child); } + void reset(IAST *& field); + /// Convert to a string. /// Format settings. diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index e89302fd212..2eade2079da 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -50,6 +50,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_drop_projection("DROP PROJECTION"); ParserKeyword s_clear_projection("CLEAR PROJECTION"); ParserKeyword s_materialize_projection("MATERIALIZE PROJECTION"); + ParserKeyword s_modify_comment("MODIFY COMMENT"); ParserKeyword s_add("ADD"); ParserKeyword s_drop("DROP"); @@ -754,6 +755,13 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected return false; command->type = ASTAlterCommand::MODIFY_QUERY; } + else if (s_modify_comment.ignore(pos, expected)) + { + if (!parser_string_literal.parse(pos, command->comment, expected)) + return false; + + command->type = ASTAlterCommand::MODIFY_COMMENT; + } else return false; } diff --git a/src/Parsers/ParserAlterQuery.h b/src/Parsers/ParserAlterQuery.h index de9d752d1a3..b0029ff88fd 100644 --- a/src/Parsers/ParserAlterQuery.h +++ b/src/Parsers/ParserAlterQuery.h @@ -18,6 +18,7 @@ namespace DB * [MODIFY SETTING setting_name=setting_value, ...] * [RESET SETTING setting_name, ...] * [COMMENT COLUMN [IF EXISTS] col_name string] + * [MODIFY COMMENT string] * [DROP|DETACH|ATTACH PARTITION|PART partition, ...] * [FETCH PARTITION partition FROM ...] * [FREEZE [PARTITION] [WITH NAME name]] diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 9a5a99f8f5a..f97bc77272c 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -17,22 +18,27 @@ using namespace std::literals; struct ParserTestCase { - std::shared_ptr parser; const std::string_view input_text; const char * expected_ast = nullptr; }; -std::ostream & operator<<(std::ostream & ostr, const ParserTestCase & test_case) +std::ostream & operator<<(std::ostream & ostr, const std::shared_ptr parser) { - return ostr << "parser: " << test_case.parser->getName() << ", input: " << test_case.input_text; + return ostr << "Praser: " << parser->getName(); } -class ParserTest : public ::testing::TestWithParam +std::ostream & operator<<(std::ostream & ostr, const ParserTestCase & test_case) +{ + return ostr << "ParserTestCase input: " << test_case.input_text; +} + +class ParserTest : public ::testing::TestWithParam, ParserTestCase>> {}; TEST_P(ParserTest, parseQuery) { - const auto & [parser, input_text, expected_ast] = GetParam(); + const auto & parser = std::get<0>(GetParam()); + const auto & [input_text, expected_ast] = std::get<1>(GetParam()); ASSERT_NE(nullptr, parser); @@ -49,86 +55,92 @@ TEST_P(ParserTest, parseQuery) } -INSTANTIATE_TEST_SUITE_P(ParserOptimizeQuery, ParserTest, ::testing::Values( - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('a, b')", - "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('a, b')" - }, - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]')", - "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]')" - }, - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]') EXCEPT b", - "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]') EXCEPT b" - }, - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]') EXCEPT (a, b)", - "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]') EXCEPT (a, b)" - }, - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY a, b, c", - "OPTIMIZE TABLE table_name DEDUPLICATE BY a, b, c" - }, - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY *", - "OPTIMIZE TABLE table_name DEDUPLICATE BY *" - }, - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY * EXCEPT a", - "OPTIMIZE TABLE table_name DEDUPLICATE BY * EXCEPT a" - }, - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY * EXCEPT (a, b)", - "OPTIMIZE TABLE table_name DEDUPLICATE BY * EXCEPT (a, b)" - } -)); +INSTANTIATE_TEST_SUITE_P(ParserOptimizeQuery, ParserTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list + { + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('a, b')", + "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('a, b')" + }, + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]')", + "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]')" + }, + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]') EXCEPT b", + "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]') EXCEPT b" + }, + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]') EXCEPT (a, b)", + "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]') EXCEPT (a, b)" + }, + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY a, b, c", + "OPTIMIZE TABLE table_name DEDUPLICATE BY a, b, c" + }, + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY *", + "OPTIMIZE TABLE table_name DEDUPLICATE BY *" + }, + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY * EXCEPT a", + "OPTIMIZE TABLE table_name DEDUPLICATE BY * EXCEPT a" + }, + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY * EXCEPT (a, b)", + "OPTIMIZE TABLE table_name DEDUPLICATE BY * EXCEPT (a, b)" + } + } +))); -INSTANTIATE_TEST_SUITE_P(ParserOptimizeQuery_FAIL, ParserTest, ::testing::Values( - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY", - }, - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]') APPLY(x)", - }, - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]') REPLACE(y)", - }, - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY * APPLY(x)", - }, - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY * REPLACE(y)", - }, - ParserTestCase - { - std::make_shared(), - "OPTIMIZE TABLE table_name DEDUPLICATE BY db.a, db.b, db.c", - } -)); +INSTANTIATE_TEST_SUITE_P(ParserOptimizeQuery_FAIL, ParserTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list + { + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY", + }, + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]') APPLY(x)", + }, + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY COLUMNS('[a]') REPLACE(y)", + }, + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY * APPLY(x)", + }, + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY * REPLACE(y)", + }, + { + "OPTIMIZE TABLE table_name DEDUPLICATE BY db.a, db.b, db.c", + } + } +))); + + +INSTANTIATE_TEST_SUITE_P(ParserAlterCommand_MODIFY_COMMENT, ParserTest, + ::testing::Combine( + ::testing::Values(std::make_shared()), + ::testing::ValuesIn(std::initializer_list + { + { + // Empty comment value + "MODIFY COMMENT ''", + "MODIFY COMMENT ''", + }, +// { +// // No comment - same as empty comment +// "MODIFY COMMENT NULL", +// "MODIFY COMMENT ''", +// }, + { + // Non-empty comment value + "MODIFY COMMENT 'some comment value'", + "MODIFY COMMENT 'some comment value'", + } + } +))); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index b0254da7a44..1d057d1bb10 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -180,6 +180,15 @@ std::optional AlterCommand::parse(const ASTAlterCommand * command_ command.if_exists = command_ast->if_exists; return command; } + else if (command_ast->type == ASTAlterCommand::MODIFY_COMMENT) + { + AlterCommand command; + command.ast = command_ast->clone(); + command.type = COMMENT_TABLE; + const auto & ast_comment = command_ast->comment->as(); + command.comment = ast_comment.value.get(); + return command; + } else if (command_ast->type == ASTAlterCommand::MODIFY_ORDER_BY) { AlterCommand command; @@ -459,6 +468,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) metadata.columns.modify(column_name, [&](ColumnDescription & column) { column.comment = *comment; }); } + else if (type == COMMENT_TABLE) + { + metadata.comment = *comment; + } else if (type == ADD_INDEX) { if (std::any_of( @@ -751,7 +764,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada bool AlterCommand::isCommentAlter() const { - if (type == COMMENT_COLUMN) + if (type == COMMENT_COLUMN || type == COMMENT_TABLE) { return true; } diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index 41b48b4b034..046238bd5f5 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -45,6 +45,7 @@ struct AlterCommand RENAME_COLUMN, REMOVE_TTL, MODIFY_DATABASE_SETTING, + COMMENT_TABLE }; /// Which property user wants to remove from column @@ -75,7 +76,7 @@ struct AlterCommand ColumnDefaultKind default_kind{}; ASTPtr default_expression{}; - /// For COMMENT column + /// For COMMENT column or table std::optional comment; /// For ADD or MODIFY - after which column to add a new one. If an empty string, add to the end. diff --git a/src/Storages/StorageNull.cpp b/src/Storages/StorageNull.cpp index 3076ec733b7..2b3585e360e 100644 --- a/src/Storages/StorageNull.cpp +++ b/src/Storages/StorageNull.cpp @@ -44,7 +44,8 @@ void StorageNull::checkAlterIsPossible(const AlterCommands & commands, ContextPt if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN && command.type != AlterCommand::Type::DROP_COLUMN - && command.type != AlterCommand::Type::COMMENT_COLUMN) + && command.type != AlterCommand::Type::COMMENT_COLUMN + && command.type != AlterCommand::Type::COMMENT_TABLE) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}", command.type, getName()); diff --git a/tests/queries/0_stateless/2020_alter_table_modify_comment.reference b/tests/queries/0_stateless/2020_alter_table_modify_comment.reference new file mode 100644 index 00000000000..4da5634cca2 --- /dev/null +++ b/tests/queries/0_stateless/2020_alter_table_modify_comment.reference @@ -0,0 +1,85 @@ +engine : Null +initial comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Null\nCOMMENT \'Test table with comment\' +comment= Test table with comment + +change a comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Null\nCOMMENT \'new comment on a table\' +comment= new comment on a table + +remove a comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Null +comment= + +add a comment back +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Null\nCOMMENT \'another comment on a table\' +comment= another comment on a table + +engine : Memory +initial comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Memory\nCOMMENT \'Test table with comment\' +comment= Test table with comment + +change a comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Memory\nCOMMENT \'new comment on a table\' +comment= new comment on a table + +remove a comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Memory +comment= + +add a comment back +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Memory\nCOMMENT \'another comment on a table\' +comment= another comment on a table + +engine : MergeTree() ORDER BY k +initial comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = MergeTree\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'Test table with comment\' +comment= Test table with comment + +change a comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = MergeTree\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'new comment on a table\' +comment= new comment on a table + +remove a comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = MergeTree\nORDER BY k\nSETTINGS index_granularity = 8192 +comment= + +add a comment back +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = MergeTree\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'another comment on a table\' +comment= another comment on a table + +engine : Log +initial comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Log\nCOMMENT \'Test table with comment\' +comment= Test table with comment + +change a comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Log\nCOMMENT \'new comment on a table\' +comment= new comment on a table + +remove a comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Log +comment= + +add a comment back +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Log\nCOMMENT \'another comment on a table\' +comment= another comment on a table + +engine : TinyLog +initial comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = TinyLog\nCOMMENT \'Test table with comment\' +comment= Test table with comment + +change a comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = TinyLog\nCOMMENT \'new comment on a table\' +comment= new comment on a table + +remove a comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = TinyLog +comment= + +add a comment back +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = TinyLog\nCOMMENT \'another comment on a table\' +comment= another comment on a table + diff --git a/tests/queries/0_stateless/2020_alter_table_modify_comment.sh b/tests/queries/0_stateless/2020_alter_table_modify_comment.sh new file mode 100755 index 00000000000..73c194c17b1 --- /dev/null +++ b/tests/queries/0_stateless/2020_alter_table_modify_comment.sh @@ -0,0 +1,50 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function get_table_comment_info() +{ + $CLICKHOUSE_CLIENT --query="SHOW CREATE TABLE comment_test_table;" + $CLICKHOUSE_CLIENT --query="SELECT 'comment=', comment FROM system.tables WHERE database=currentDatabase() and name='comment_test_table'" + echo # just a newline +} + +function test_table_comments() +{ + local ENGINE_NAME="$1" + echo "engine : ${ENGINE_NAME}" + + $CLICKHOUSE_CLIENT -nm < Date: Wed, 22 Sep 2021 20:48:25 +0000 Subject: [PATCH 088/142] Fix --- contrib/libhdfs3 | 2 +- src/Storages/HDFS/ReadBufferFromHDFS.cpp | 22 ++++++++++++---------- 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/contrib/libhdfs3 b/contrib/libhdfs3 index 095b9d48b40..85ec1dd420e 160000 --- a/contrib/libhdfs3 +++ b/contrib/libhdfs3 @@ -1 +1 @@ -Subproject commit 095b9d48b400abb72d967cb0539af13b1e3d90cf +Subproject commit 85ec1dd420ea9e9cf9d53a3b6eeaa45e8b78b00b diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index d241bd07294..96d67ad0e08 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -85,10 +85,15 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemoryposition() = impl->buffer().begin() + offset(); auto result = impl->next(); if (result) - { - working_buffer = internal_buffer = impl->buffer(); - pos = working_buffer.begin(); - } - else - return false; - return true; + BufferBase::set(impl->buffer().begin(), impl->buffer().size(), impl->offset); /// use the buffer returned by `impl` + + return result; } From 3b762f10a09e59585839a2c52d8f61523077c02e Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Thu, 23 Sep 2021 10:31:59 +0300 Subject: [PATCH 089/142] Fixed test --- tests/queries/0_stateless/01271_show_privileges.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 5c493dce98a..d347f149230 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -15,6 +15,7 @@ ALTER CLEAR COLUMN ['CLEAR COLUMN'] COLUMN ALTER COLUMN ALTER RENAME COLUMN ['RENAME COLUMN'] COLUMN ALTER COLUMN ALTER MATERIALIZE COLUMN ['MATERIALIZE COLUMN'] COLUMN ALTER COLUMN ALTER COLUMN [] \N ALTER TABLE +ALTER MODIFY COMMENT ['MODIFY COMMENT'] TABLE ALTER TABLE ALTER ORDER BY ['ALTER MODIFY ORDER BY','MODIFY ORDER BY'] TABLE ALTER INDEX ALTER SAMPLE BY ['ALTER MODIFY SAMPLE BY','MODIFY SAMPLE BY'] TABLE ALTER INDEX ALTER ADD INDEX ['ADD INDEX'] TABLE ALTER INDEX From 562ce8c7c204f82f719b89192932224acec09679 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 23 Sep 2021 13:01:52 +0300 Subject: [PATCH 090/142] Fix virtuals. --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 008a00f1658..12e6153c8ea 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -147,8 +147,8 @@ Chain buildPushingToViewsChain( /// If we don't write directly to the destination /// then expect that we're inserting with precalculated virtual columns - auto storage_header = //no_destination ? metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals()) - metadata_snapshot->getSampleBlock(); + auto storage_header = no_destination ? metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals()) + : metadata_snapshot->getSampleBlock(); /** TODO This is a very important line. At any insertion into the table one of streams should own lock. * Although now any insertion into the table is done via PushingToViewsBlockOutputStream, @@ -275,8 +275,7 @@ Chain buildPushingToViewsChain( out = buildPushingToViewsChain( dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), false, view_thread_status, view_counter_ms); - assert(views_data != nullptr); - views_data->views.emplace_back(ViewRuntimeData{ + views_data->views.emplace_back(ViewRuntimeData{ //-V614 std::move(query), out.getInputHeader(), database_table, From 28d1938ac467e1ad0188a0241fb28d3522e53f4d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 23 Sep 2021 13:44:00 +0300 Subject: [PATCH 091/142] Remote 01200_mutations_memory_consumption from fast --- .../queries/0_stateless/01200_mutations_memory_consumption.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01200_mutations_memory_consumption.sql b/tests/queries/0_stateless/01200_mutations_memory_consumption.sql index 6947c327adc..d3eb5dd165c 100644 --- a/tests/queries/0_stateless/01200_mutations_memory_consumption.sql +++ b/tests/queries/0_stateless/01200_mutations_memory_consumption.sql @@ -1,4 +1,4 @@ --- Tags: no-debug, no-parallel +-- Tags: no-debug, no-parallel, no-fasttest DROP TABLE IF EXISTS table_with_single_pk; From 1bb48cbe7275ce3bf9a8efdd0e5c672c0bb1e175 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 23 Sep 2021 13:54:12 +0300 Subject: [PATCH 092/142] Add test for segfault in StorageEmbeddedRocksDB, issue #29227 --- .../02030_rocksdb_race_long.reference | 0 .../0_stateless/02030_rocksdb_race_long.sh | 73 +++++++++++++++++++ 2 files changed, 73 insertions(+) create mode 100644 tests/queries/0_stateless/02030_rocksdb_race_long.reference create mode 100755 tests/queries/0_stateless/02030_rocksdb_race_long.sh diff --git a/tests/queries/0_stateless/02030_rocksdb_race_long.reference b/tests/queries/0_stateless/02030_rocksdb_race_long.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02030_rocksdb_race_long.sh b/tests/queries/0_stateless/02030_rocksdb_race_long.sh new file mode 100755 index 00000000000..3f05f30269d --- /dev/null +++ b/tests/queries/0_stateless/02030_rocksdb_race_long.sh @@ -0,0 +1,73 @@ +#!/usr/bin/env bash +# Tags: race + +unset CLICKHOUSE_LOG_COMMENT + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -o errexit +set -o pipefail + +echo " + DROP TABLE IF EXISTS rocksdb_race; + CREATE TABLE rocksdb_race (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key); + INSERT INTO rocksdb_race SELECT '1_' || toString(number), number FROM numbers(100000); +" | $CLICKHOUSE_CLIENT -n + +function read_thread() +{ + while true; do + echo " + SELECT * FROM rocksdb_race FORMAT Null; + " | $CLICKHOUSE_CLIENT -n + done +} + + +function read_stat_thread() +{ + while true; do + echo " + SELECT * FROM system.rocksdb FORMAT Null; + " | $CLICKHOUSE_CLIENT -n + done +} + +function truncate_thread() +{ + while true; do + sleep 3s; + echo " + TRUNCATE TABLE rocksdb_race; + " | $CLICKHOUSE_CLIENT -n + done +} + + +function insert_thread() +{ + while true; do + echo " + INSERT INTO rocksdb_race SELECT '2_' || toString(rand()), number FROM numbers(100000); + " | $CLICKHOUSE_CLIENT -n + done +} + +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f read_thread; +export -f read_stat_thread; +export -f truncate_thread; + +TIMEOUT=20 + +# timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +# timeout $TIMEOUT bash -c read_thread 2> /dev/null & +timeout $TIMEOUT bash -c read_stat_thread 2> /dev/null & +timeout $TIMEOUT bash -c truncate_thread 2> /dev/null & + + +wait + +$CLICKHOUSE_CLIENT -q "DROP TABLE rocksdb_race" From 9af2e4891d030a9eeb561357e7ddb643f466c53a Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 23 Sep 2021 14:30:39 +0300 Subject: [PATCH 093/142] Cleanup test rocksdb_race_long.sh --- .../0_stateless/02030_rocksdb_race_long.sh | 24 ------------------- 1 file changed, 24 deletions(-) diff --git a/tests/queries/0_stateless/02030_rocksdb_race_long.sh b/tests/queries/0_stateless/02030_rocksdb_race_long.sh index 3f05f30269d..88c30852c86 100755 --- a/tests/queries/0_stateless/02030_rocksdb_race_long.sh +++ b/tests/queries/0_stateless/02030_rocksdb_race_long.sh @@ -16,16 +16,6 @@ echo " INSERT INTO rocksdb_race SELECT '1_' || toString(number), number FROM numbers(100000); " | $CLICKHOUSE_CLIENT -n -function read_thread() -{ - while true; do - echo " - SELECT * FROM rocksdb_race FORMAT Null; - " | $CLICKHOUSE_CLIENT -n - done -} - - function read_stat_thread() { while true; do @@ -45,29 +35,15 @@ function truncate_thread() done } - -function insert_thread() -{ - while true; do - echo " - INSERT INTO rocksdb_race SELECT '2_' || toString(rand()), number FROM numbers(100000); - " | $CLICKHOUSE_CLIENT -n - done -} - # https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout -export -f read_thread; export -f read_stat_thread; export -f truncate_thread; TIMEOUT=20 -# timeout $TIMEOUT bash -c insert_thread 2> /dev/null & -# timeout $TIMEOUT bash -c read_thread 2> /dev/null & timeout $TIMEOUT bash -c read_stat_thread 2> /dev/null & timeout $TIMEOUT bash -c truncate_thread 2> /dev/null & - wait $CLICKHOUSE_CLIENT -q "DROP TABLE rocksdb_race" From 81a051e3eecc10863dda7daf8ddca98fa4e9762d Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 23 Sep 2021 14:30:59 +0300 Subject: [PATCH 094/142] Use dynamic_pointer_cast instead of dynamic_cast in StorageSystemRocksDB --- src/Storages/RocksDB/StorageSystemRocksDB.cpp | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Storages/RocksDB/StorageSystemRocksDB.cpp b/src/Storages/RocksDB/StorageSystemRocksDB.cpp index 7d31d5ddc21..61606d19760 100644 --- a/src/Storages/RocksDB/StorageSystemRocksDB.cpp +++ b/src/Storages/RocksDB/StorageSystemRocksDB.cpp @@ -43,7 +43,8 @@ void StorageSystemRocksDB::fillData(MutableColumns & res_columns, ContextPtr con const auto access = context->getAccess(); const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES); - std::map> tables; + using RocksDBStoragePtr = std::shared_ptr; + std::map> tables; for (const auto & db : DatabaseCatalog::instance().getDatabases()) { const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); @@ -51,18 +52,17 @@ void StorageSystemRocksDB::fillData(MutableColumns & res_columns, ContextPtr con for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { StoragePtr table = iterator->table(); - if (!table) + RocksDBStoragePtr rocksdb_table = table ? std::dynamic_pointer_cast(table) : nullptr; + if (!rocksdb_table) continue; - if (!dynamic_cast(table.get())) - continue; if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, db.first, iterator->name())) continue; - tables[db.first][iterator->name()] = table; + + tables[db.first][iterator->name()] = rocksdb_table; } } - MutableColumnPtr col_database_mut = ColumnString::create(); MutableColumnPtr col_table_mut = ColumnString::create(); @@ -101,8 +101,7 @@ void StorageSystemRocksDB::fillData(MutableColumns & res_columns, ContextPtr con String database = (*col_database_to_filter)[i].safeGet(); String table = (*col_table_to_filter)[i].safeGet(); - auto & rocksdb_table = dynamic_cast(*tables[database][table]); - auto statistics = rocksdb_table.getRocksDBStatistics(); + auto statistics = tables[database][table]->getRocksDBStatistics(); if (!statistics) throw Exception(ErrorCodes::LOGICAL_ERROR, "rocksdb statistics is not enabled"); From 9ca9a66a70f048a1855d775055f554fd1a95a1d6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 23 Sep 2021 17:03:33 +0300 Subject: [PATCH 095/142] Fix race in StorageSystemRocksDB truncate --- .../RocksDB/StorageEmbeddedRocksDB.cpp | 23 +++++++++++++++---- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 7 +++++- src/Storages/RocksDB/StorageSystemRocksDB.cpp | 2 +- 3 files changed, 26 insertions(+), 6 deletions(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 459c0879cda..a8aac870d2d 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -36,6 +36,8 @@ #include #include +#include +#include namespace fs = std::filesystem; @@ -213,9 +215,9 @@ public: std::vector slices_keys(num_keys); const auto & sample_block = metadata_snapshot->getSampleBlock(); - const auto & key_column = sample_block.getByName(storage.primary_key); + const auto & key_column = sample_block.getByName(storage.getPrimaryKey()); auto columns = sample_block.cloneEmptyColumns(); - size_t primary_key_pos = sample_block.getPositionByName(storage.primary_key); + size_t primary_key_pos = sample_block.getPositionByName(storage.getPrimaryKey()); size_t rows_processed = 0; while (it < end && rows_processed < max_block_size) @@ -230,8 +232,7 @@ public: } std::vector values; - auto statuses = storage.rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); - + auto statuses = storage.multiGet(slices_keys, values); for (size_t i = 0; i < statuses.size(); ++i) { if (statuses[i].ok()) @@ -285,7 +286,10 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &) { + std::unique_lock lock(rocksdb_ptr_mx); rocksdb_ptr->Close(); + rocksdb_ptr = nullptr; + fs::remove_all(rocksdb_dir); fs::create_directories(rocksdb_dir); initDb(); @@ -460,9 +464,20 @@ static StoragePtr create(const StorageFactory::Arguments & args) std::shared_ptr StorageEmbeddedRocksDB::getRocksDBStatistics() const { + std::shared_lock lock(rocksdb_ptr_mx); + if (!rocksdb_ptr) + return nullptr; return rocksdb_ptr->GetOptions().statistics; } +std::vector StorageEmbeddedRocksDB::multiGet(const std::vector & slices_keys, std::vector & values) const +{ + std::shared_lock lock(rocksdb_ptr_mx); + if (!rocksdb_ptr) + return {}; + return rocksdb_ptr->MultiGet(rocksdb::ReadOptions(), slices_keys, &values); +} + void registerStorageEmbeddedRocksDB(StorageFactory & factory) { StorageFactory::StorageFeatures features{ diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 3f1b3b49492..89352a7516b 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -1,14 +1,17 @@ #pragma once #include +#include #include #include +#include namespace rocksdb { class DB; class Statistics; + class Slice; } @@ -20,7 +23,6 @@ class Context; class StorageEmbeddedRocksDB final : public shared_ptr_helper, public IStorage, WithContext { friend struct shared_ptr_helper; - friend class EmbeddedRocksDBSource; friend class EmbeddedRocksDBSink; friend class EmbeddedRocksDBBlockInputStream; public: @@ -50,6 +52,8 @@ public: Strings getDataPaths() const override { return {rocksdb_dir}; } std::shared_ptr getRocksDBStatistics() const; + std::vector multiGet(const std::vector & slices_keys, std::vector & values) const; + const String & getPrimaryKey() const { return primary_key; } protected: StorageEmbeddedRocksDB(const StorageID & table_id_, @@ -63,6 +67,7 @@ private: const String primary_key; using RocksDBPtr = std::unique_ptr; RocksDBPtr rocksdb_ptr; + mutable std::shared_mutex rocksdb_ptr_mx; String rocksdb_dir; void initDb(); diff --git a/src/Storages/RocksDB/StorageSystemRocksDB.cpp b/src/Storages/RocksDB/StorageSystemRocksDB.cpp index 61606d19760..57b4832bccb 100644 --- a/src/Storages/RocksDB/StorageSystemRocksDB.cpp +++ b/src/Storages/RocksDB/StorageSystemRocksDB.cpp @@ -103,7 +103,7 @@ void StorageSystemRocksDB::fillData(MutableColumns & res_columns, ContextPtr con auto statistics = tables[database][table]->getRocksDBStatistics(); if (!statistics) - throw Exception(ErrorCodes::LOGICAL_ERROR, "rocksdb statistics is not enabled"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "RocksDB statistics is not avaliable"); for (auto [tick, name] : rocksdb::TickersNameMap) { From 68edcf893ce4f62c5f2947df8b3ec055f4d49904 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 Sep 2021 14:12:57 +0000 Subject: [PATCH 096/142] Update libhdfs3 --- contrib/libhdfs3 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libhdfs3 b/contrib/libhdfs3 index 85ec1dd420e..22781c4ebd3 160000 --- a/contrib/libhdfs3 +++ b/contrib/libhdfs3 @@ -1 +1 @@ -Subproject commit 85ec1dd420ea9e9cf9d53a3b6eeaa45e8b78b00b +Subproject commit 22781c4ebd31aee918c38b3af7b2e8da5adec93e From 604dc8861de14fb7f133904edfcae776a3dadd5c Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Fri, 24 Sep 2021 13:07:24 +0300 Subject: [PATCH 097/142] Apply suggestions from code review Co-authored-by: Azat Khuzhin --- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 1 - src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 1 - src/Storages/RocksDB/StorageSystemRocksDB.cpp | 2 +- 3 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index a8aac870d2d..7c5d36b37ce 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -36,7 +36,6 @@ #include #include -#include #include diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 89352a7516b..71460a1667e 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -11,7 +11,6 @@ namespace rocksdb { class DB; class Statistics; - class Slice; } diff --git a/src/Storages/RocksDB/StorageSystemRocksDB.cpp b/src/Storages/RocksDB/StorageSystemRocksDB.cpp index 57b4832bccb..cbb96ed4001 100644 --- a/src/Storages/RocksDB/StorageSystemRocksDB.cpp +++ b/src/Storages/RocksDB/StorageSystemRocksDB.cpp @@ -103,7 +103,7 @@ void StorageSystemRocksDB::fillData(MutableColumns & res_columns, ContextPtr con auto statistics = tables[database][table]->getRocksDBStatistics(); if (!statistics) - throw Exception(ErrorCodes::LOGICAL_ERROR, "RocksDB statistics is not avaliable"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "RocksDB statistics are not available"); for (auto [tick, name] : rocksdb::TickersNameMap) { From 1d20014d5dd6978454eeb7ce0e45fbdb24312f76 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Fri, 24 Sep 2021 13:08:38 +0300 Subject: [PATCH 098/142] Fixed clang12 build and updated tests --- src/Parsers/IAST.cpp | 7 ++- .../2020_alter_table_modify_comment.reference | 59 +++++++++++++++++++ .../2020_alter_table_modify_comment.sh | 9 +++ 3 files changed, 74 insertions(+), 1 deletion(-) diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 2ca72007285..f5b3630c7f1 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -49,10 +49,15 @@ size_t IAST::checkSize(size_t max_size) const void IAST::reset(IAST *& field) { - std::find_if(children.begin(), children.end(), [field](const auto & p) + const auto child = std::find_if(children.begin(), children.end(), [field](const auto & p) { return p.get() == field; }); + + if (child == children.end()) + throw Exception("AST subtree not found in children", ErrorCodes::LOGICAL_ERROR); + + children.erase(child); field = nullptr; } diff --git a/tests/queries/0_stateless/2020_alter_table_modify_comment.reference b/tests/queries/0_stateless/2020_alter_table_modify_comment.reference index 4da5634cca2..29ade427196 100644 --- a/tests/queries/0_stateless/2020_alter_table_modify_comment.reference +++ b/tests/queries/0_stateless/2020_alter_table_modify_comment.reference @@ -15,6 +15,13 @@ add a comment back CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Null\nCOMMENT \'another comment on a table\' comment= another comment on a table +detach table +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Null\nCOMMENT \'another comment on a table\' + +re-attach table +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Null\nCOMMENT \'another comment on a table\' +comment= another comment on a table + engine : Memory initial comment CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Memory\nCOMMENT \'Test table with comment\' @@ -32,6 +39,13 @@ add a comment back CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Memory\nCOMMENT \'another comment on a table\' comment= another comment on a table +detach table +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Memory\nCOMMENT \'another comment on a table\' + +re-attach table +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Memory\nCOMMENT \'another comment on a table\' +comment= another comment on a table + engine : MergeTree() ORDER BY k initial comment CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = MergeTree\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'Test table with comment\' @@ -49,6 +63,13 @@ add a comment back CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = MergeTree\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'another comment on a table\' comment= another comment on a table +detach table +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = MergeTree\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'another comment on a table\' + +re-attach table +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = MergeTree\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'another comment on a table\' +comment= another comment on a table + engine : Log initial comment CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Log\nCOMMENT \'Test table with comment\' @@ -66,6 +87,13 @@ add a comment back CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Log\nCOMMENT \'another comment on a table\' comment= another comment on a table +detach table +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Log\nCOMMENT \'another comment on a table\' + +re-attach table +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Log\nCOMMENT \'another comment on a table\' +comment= another comment on a table + engine : TinyLog initial comment CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = TinyLog\nCOMMENT \'Test table with comment\' @@ -83,3 +111,34 @@ add a comment back CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = TinyLog\nCOMMENT \'another comment on a table\' comment= another comment on a table +detach table +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = TinyLog\nCOMMENT \'another comment on a table\' + +re-attach table +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = TinyLog\nCOMMENT \'another comment on a table\' +comment= another comment on a table + +engine : ReplicatedMergeTree('/clickhouse/2020_alter_table_modify_comment_default', '1') ORDER BY k +initial comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/2020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'Test table with comment\' +comment= Test table with comment + +change a comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/2020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'Test table with comment\' +comment= Test table with comment + +remove a comment +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/2020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'Test table with comment\' +comment= Test table with comment + +add a comment back +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/2020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'Test table with comment\' +comment= Test table with comment + +detach table +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/2020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'Test table with comment\' + +re-attach table +CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/2020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'Test table with comment\' +comment= Test table with comment + diff --git a/tests/queries/0_stateless/2020_alter_table_modify_comment.sh b/tests/queries/0_stateless/2020_alter_table_modify_comment.sh index 73c194c17b1..c674f21034c 100755 --- a/tests/queries/0_stateless/2020_alter_table_modify_comment.sh +++ b/tests/queries/0_stateless/2020_alter_table_modify_comment.sh @@ -41,6 +41,14 @@ EOF echo add a comment back $CLICKHOUSE_CLIENT --query="ALTER TABLE comment_test_table MODIFY COMMENT 'another comment on a table';" get_table_comment_info + + echo detach table + $CLICKHOUSE_CLIENT --query="DETACH TABLE comment_test_table NO DELAY;" + get_table_comment_info + + echo re-attach table + $CLICKHOUSE_CLIENT --query="ATTACH TABLE comment_test_table;" + get_table_comment_info } test_table_comments "Null" @@ -48,3 +56,4 @@ test_table_comments "Memory" test_table_comments "MergeTree() ORDER BY k" test_table_comments "Log" test_table_comments "TinyLog" +test_table_comments "ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') ORDER BY k" From 165765659d878a265f7fbde89a65448a06bb6304 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 24 Sep 2021 13:22:22 +0300 Subject: [PATCH 099/142] fix assertion in DDLDependencyVisitor --- src/Databases/DDLDependencyVisitor.cpp | 13 +++++++++++-- .../0_stateless/01160_table_dependencies.reference | 1 + .../queries/0_stateless/01160_table_dependencies.sh | 2 ++ 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/src/Databases/DDLDependencyVisitor.cpp b/src/Databases/DDLDependencyVisitor.cpp index 0399ec59b16..73800e84256 100644 --- a/src/Databases/DDLDependencyVisitor.cpp +++ b/src/Databases/DDLDependencyVisitor.cpp @@ -18,9 +18,18 @@ void DDLDependencyVisitor::visit(const ASTPtr & ast, Data & data) visit(*dict_source, data); } -bool DDLDependencyVisitor::needChildVisit(const ASTPtr & node, const ASTPtr & /*child*/) +bool DDLDependencyVisitor::needChildVisit(const ASTPtr & node, const ASTPtr & child) { - return !node->as(); + if (node->as()) + return false; + + if (auto * create = node->as()) + { + if (child.get() == create->select) + return false; + } + + return true; } void DDLDependencyVisitor::visit(const ASTFunction & function, Data & data) diff --git a/tests/queries/0_stateless/01160_table_dependencies.reference b/tests/queries/0_stateless/01160_table_dependencies.reference index 39a58b06076..a893d4882df 100644 --- a/tests/queries/0_stateless/01160_table_dependencies.reference +++ b/tests/queries/0_stateless/01160_table_dependencies.reference @@ -2,5 +2,6 @@ dict1 dict2 dict_src join +mv s t diff --git a/tests/queries/0_stateless/01160_table_dependencies.sh b/tests/queries/0_stateless/01160_table_dependencies.sh index 149439f2981..05d086ae1a4 100755 --- a/tests/queries/0_stateless/01160_table_dependencies.sh +++ b/tests/queries/0_stateless/01160_table_dependencies.sh @@ -30,6 +30,8 @@ $CLICKHOUSE_CLIENT -q "create table s (x default joinGet($CLICKHOUSE_DATABASE.jo $CLICKHOUSE_CLIENT -q "create table t (n int, m int default joinGet($CLICKHOUSE_DATABASE.join, 'm', 42::int), s String default dictGet($CLICKHOUSE_DATABASE.dict1, 's', 42::UInt64), x default in(1, $CLICKHOUSE_DATABASE.s)) engine=MergeTree order by n;" +$CLICKHOUSE_CLIENT -q "create materialized view mv to s as select n from t where n in (select n from join)" + CLICKHOUSE_CLIENT_DEFAULT_DB=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--database=${CLICKHOUSE_DATABASE}"'/--database=default/g') for _ in {1..10}; do From cc53ba283d04f2ab63b32df65aa68d20bf0aca89 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Sep 2021 13:23:28 +0300 Subject: [PATCH 100/142] Try fix other tests. --- src/Storages/Kafka/StorageKafka.cpp | 2 +- .../PostgreSQL/PostgreSQLReplicationHandler.cpp | 14 ++++++++------ src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 9 ++++----- 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 4c9542abc94..81930ac1085 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -653,8 +653,8 @@ bool StorageKafka::streamToViews() executor.push(std::move(block)); } - executor.finish(); in->readSuffix(); + executor.finish(); bool some_stream_is_stalled = false; for (auto & stream : streams) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 3344f1ab1dd..53dc7d01728 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -251,20 +251,22 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection & materialized_storage->createNestedIfNeeded(fetchTableStructure(*tx, table_name)); auto nested_storage = materialized_storage->getNested(); + auto insert = std::make_shared(); + insert->table_id = nested_storage->getStorageID(); + auto insert_context = materialized_storage->getNestedTableContext(); - InterpreterInsertQuery interpreter(nullptr, insert_context); - auto chain = interpreter.buildChain(nested_storage, nested_storage->getInMemoryMetadataPtr(), {}); + InterpreterInsertQuery interpreter(insert, insert_context); + auto block_io = interpreter.execute(); const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); auto input = std::make_unique>(tx, query_str, sample_block, DEFAULT_BLOCK_SIZE); - assertBlocksHaveEqualStructure(input->getPort().getHeader(), chain.getInputHeader(), "postgresql replica load from snapshot"); - QueryPipeline pipeline(std::move(chain)); - pipeline.complete(Pipe(std::move(input))); + assertBlocksHaveEqualStructure(input->getPort().getHeader(), block_io.pipeline.getHeader(), "postgresql replica load from snapshot"); + block_io.pipeline.complete(Pipe(std::move(input))); - CompletedPipelineExecutor executor(pipeline); + CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); nested_storage = materialized_storage->prepare(); diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 43891956567..f70d5f10a0b 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -953,11 +953,11 @@ bool StorageRabbitMQ::streamToViews() insert->table_id = table_id; // Only insert into dependent views and expect that input blocks contain virtual columns - InterpreterInsertQuery interpreter(nullptr, rabbitmq_context, false, true, true); - auto chain = interpreter.buildChain(table, table->getInMemoryMetadataPtr(), {}); + InterpreterInsertQuery interpreter(insert, rabbitmq_context, false, true, true); + auto block_io = interpreter.execute(); auto metadata_snapshot = getInMemoryMetadataPtr(); - auto column_names = chain.getInputHeader().getNames(); + auto column_names = block_io.pipeline.getHeader().getNames(); auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); auto block_size = getMaxBlockSize(); @@ -997,8 +997,7 @@ bool StorageRabbitMQ::streamToViews() looping_task->activateAndSchedule(); } - QueryPipeline pipeline(std::move(chain)); - PushingPipelineExecutor executor(pipeline); + PushingPipelineExecutor executor(block_io.pipeline); executor.start(); in->readPrefix(); while (auto block = in->read()) From 24dac111b70b3f4b03bb0b967489e204b4f63dd4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Sep 2021 13:30:57 +0300 Subject: [PATCH 101/142] Try fix kafka test. --- tests/integration/test_storage_kafka/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 698d2032d32..88477882515 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2161,7 +2161,7 @@ def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): # we have 0.25 (sleepEachRow) * 20 ( Rows ) = 5 sec window after "Polled batch of 20 messages" # while materialized view is working to inject zookeeper failure pm.drop_instance_zk_connections(instance) - instance.wait_for_log_line("Error.*(session has been expired|Connection loss).*while writing suffix to view") + instance.wait_for_log_line("Error.*(session has been expired|Connection loss).*") pm.heal_all() instance.wait_for_log_line("Committed offset 22") From 9d3e8fb9c397dbb9940de6a718893043dd2840dd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Sep 2021 13:41:52 +0300 Subject: [PATCH 102/142] Tiny fix. --- src/Storages/Kafka/StorageKafka.cpp | 24 ++++++++++++----------- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 16 ++++++++------- 2 files changed, 22 insertions(+), 18 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 81930ac1085..8ab9127bf13 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -642,19 +642,21 @@ bool StorageKafka::streamToViews() // It will be cancelled on underlying layer (kafka buffer) size_t rows = 0; - PushingPipelineExecutor executor(block_io.pipeline); - - in->readPrefix(); - executor.start(); - - while (auto block = in->read()) { - rows += block.rows(); - executor.push(std::move(block)); - } + PushingPipelineExecutor executor(block_io.pipeline); - in->readSuffix(); - executor.finish(); + in->readPrefix(); + executor.start(); + + while (auto block = in->read()) + { + rows += block.rows(); + executor.push(std::move(block)); + } + + in->readSuffix(); + executor.finish(); + } bool some_stream_is_stalled = false; for (auto & stream : streams) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index f70d5f10a0b..ba851b2f92c 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -997,13 +997,15 @@ bool StorageRabbitMQ::streamToViews() looping_task->activateAndSchedule(); } - PushingPipelineExecutor executor(block_io.pipeline); - executor.start(); - in->readPrefix(); - while (auto block = in->read()) - executor.push(std::move(block)); - executor.finish(); - in->readSuffix(); + { + PushingPipelineExecutor executor(block_io.pipeline); + in->readPrefix(); + executor.start(); + while (auto block = in->read()) + executor.push(std::move(block)); + in->readSuffix(); + executor.finish(); + } /* Note: sending ack() with loop running in another thread will lead to a lot of data races inside the library, but only in case * error occurs or connection is lost while ack is being sent From 193f56656892f76e955e4f351e3af6c03ff5b7fb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Sep 2021 14:54:39 +0300 Subject: [PATCH 103/142] Fix exception message. --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 2 +- tests/integration/test_storage_kafka/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 12e6153c8ea..5a55c8f4b48 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -631,7 +631,7 @@ static std::exception_ptr addStorageToException(std::exception_ptr ptr, const St } catch (DB::Exception & exception) { - exception.addMessage("while inserting into {}", storage.getNameForLogs()); + exception.addMessage("while pushing to view {}", storage.getNameForLogs()); return std::current_exception(); } catch (...) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 88477882515..8883684730f 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2161,7 +2161,7 @@ def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): # we have 0.25 (sleepEachRow) * 20 ( Rows ) = 5 sec window after "Polled batch of 20 messages" # while materialized view is working to inject zookeeper failure pm.drop_instance_zk_connections(instance) - instance.wait_for_log_line("Error.*(session has been expired|Connection loss).*") + instance.wait_for_log_line("Error.*(session has been expired|Connection loss).*while pushing to view") pm.heal_all() instance.wait_for_log_line("Committed offset 22") From 23d3d894e683c1a2d61a2a0c2d315d5477d8c55c Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 24 Sep 2021 21:57:44 +0800 Subject: [PATCH 104/142] Fix projection merges and mutations. --- .../MergeTree/MergeFromLogEntryTask.cpp | 1 + src/Storages/MergeTree/MergeList.h | 2 + .../MergeTree/MergePlainMergeTreeTask.cpp | 1 + src/Storages/MergeTree/MergeProgress.h | 14 ++-- src/Storages/MergeTree/MergeTask.cpp | 64 +++++++++---------- src/Storages/MergeTree/MergeTask.h | 12 +++- .../MergeTree/MergeTreeDataMergerMutator.cpp | 13 ++-- .../MergeTree/MergeTreeDataMergerMutator.h | 1 + src/Storages/MergeTree/MutateTask.cpp | 5 +- ...01710_projection_vertical_merges.reference | 0 .../01710_projection_vertical_merges.sql | 19 ++++++ 11 files changed, 81 insertions(+), 51 deletions(-) create mode 100644 tests/queries/0_stateless/01710_projection_vertical_merges.reference create mode 100644 tests/queries/0_stateless/01710_projection_vertical_merges.sql diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index d9d691ab2e7..3a495fccc0e 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -194,6 +194,7 @@ std::pair MergeFromLogEntryT future_merged_part, metadata_snapshot, merge_mutate_entry.get(), + {} /* projection_merge_list_element */, table_lock_holder, entry.create_time, storage.getContext(), diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index bab4420de83..24df4ba5e42 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -114,6 +114,8 @@ struct MergeListElement : boost::noncopyable MergeInfo getInfo() const; + MergeListElement * ptr() { return this; } + ~MergeListElement(); }; diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index e8770fdc76e..d52ffe32f7f 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -95,6 +95,7 @@ void MergePlainMergeTreeTask::prepare() future_part, metadata_snapshot, merge_list_entry.get(), + {} /* projection_merge_list_element */, table_lock_holder, time(nullptr), storage.getContext(), diff --git a/src/Storages/MergeTree/MergeProgress.h b/src/Storages/MergeTree/MergeProgress.h index 2862a934411..d21edac76df 100644 --- a/src/Storages/MergeTree/MergeProgress.h +++ b/src/Storages/MergeTree/MergeProgress.h @@ -47,21 +47,21 @@ class MergeProgressCallback { public: MergeProgressCallback( - MergeList::Entry & merge_entry_, UInt64 & watch_prev_elapsed_, MergeStageProgress & stage_) - : merge_entry(merge_entry_) + MergeListElement * merge_list_element_ptr_, UInt64 & watch_prev_elapsed_, MergeStageProgress & stage_) + : merge_list_element_ptr(merge_list_element_ptr_) , watch_prev_elapsed(watch_prev_elapsed_) , stage(stage_) { updateWatch(); } - MergeList::Entry & merge_entry; + MergeListElement * merge_list_element_ptr; UInt64 & watch_prev_elapsed; MergeStageProgress & stage; void updateWatch() { - UInt64 watch_curr_elapsed = merge_entry->watch.elapsed(); + UInt64 watch_curr_elapsed = merge_list_element_ptr->watch.elapsed(); ProfileEvents::increment(ProfileEvents::MergesTimeMilliseconds, (watch_curr_elapsed - watch_prev_elapsed) / 1000000); watch_prev_elapsed = watch_curr_elapsed; } @@ -76,15 +76,15 @@ public: } updateWatch(); - merge_entry->bytes_read_uncompressed += value.read_bytes; + merge_list_element_ptr->bytes_read_uncompressed += value.read_bytes; if (stage.is_first) - merge_entry->rows_read += value.read_rows; + merge_list_element_ptr->rows_read += value.read_rows; stage.total_rows += value.total_rows_to_read; stage.rows_read += value.read_rows; if (stage.total_rows > 0) { - merge_entry->progress.store( + merge_list_element_ptr->progress.store( stage.initial_progress + stage.weight * stage.rows_read / stage.total_rows, std::memory_order_relaxed); } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 855198f697e..c22d83275a4 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -141,7 +141,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->future_part->part_info, local_single_disk_volume, local_tmp_prefix + global_ctx->future_part->name + (global_ctx->parent_part ? ".proj" : ""), - global_ctx->parent_part.get()); + global_ctx->parent_part); global_ctx->new_data_part->uuid = global_ctx->future_part->uuid; global_ctx->new_data_part->setColumns(global_ctx->storage_columns); @@ -171,10 +171,10 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() ctx->need_remove_expired_values = false; } - ctx->sum_input_rows_upper_bound = (*global_ctx->merge_entry)->total_rows_count; - ctx->sum_compressed_bytes_upper_bound = (*global_ctx->merge_entry)->total_size_bytes_compressed; + ctx->sum_input_rows_upper_bound = global_ctx->merge_list_element_ptr->total_rows_count; + ctx->sum_compressed_bytes_upper_bound = global_ctx->merge_list_element_ptr->total_size_bytes_compressed; global_ctx->chosen_merge_algorithm = chooseMergeAlgorithm(); - (*global_ctx->merge_entry)->merge_algorithm.store(global_ctx->chosen_merge_algorithm, std::memory_order_relaxed); + global_ctx->merge_list_element_ptr->merge_algorithm.store(global_ctx->chosen_merge_algorithm, std::memory_order_relaxed); LOG_DEBUG(ctx->log, "Selected MergeAlgorithm: {}", toString(global_ctx->chosen_merge_algorithm)); @@ -184,7 +184,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() /// the order is reverse. This annoys TSan even though one lock is locked in shared mode and thus /// deadlock is impossible. ctx->compression_codec = global_ctx->data->getCompressionCodecForPart( - (*global_ctx->merge_entry)->total_size_bytes_compressed, global_ctx->new_data_part->ttl_infos, global_ctx->time_of_merge); + global_ctx->merge_list_element_ptr->total_size_bytes_compressed, global_ctx->new_data_part->ttl_infos, global_ctx->time_of_merge); ctx->tmp_disk = global_ctx->context->getTemporaryVolume()->getDisk(); @@ -307,8 +307,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() const_cast(*global_ctx->to).write(block); - (*global_ctx->merge_entry)->rows_written = global_ctx->merged_stream->getProfileInfo().rows; - (*global_ctx->merge_entry)->bytes_written_uncompressed = global_ctx->merged_stream->getProfileInfo().bytes; + global_ctx->merge_list_element_ptr->rows_written = global_ctx->merged_stream->getProfileInfo().rows; + global_ctx->merge_list_element_ptr->bytes_written_uncompressed = global_ctx->merged_stream->getProfileInfo().bytes; /// Reservation updates is not performed yet, during the merge it may lead to higher free space requirements if (global_ctx->space_reservation && ctx->sum_input_rows_upper_bound) @@ -317,7 +317,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() /// But now we are using inaccurate row-based estimation in Horizontal case for backward compatibility Float64 progress = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Horizontal) ? std::min(1., 1. * global_ctx->rows_written / ctx->sum_input_rows_upper_bound) - : std::min(1., (*global_ctx->merge_entry)->progress.load(std::memory_order_relaxed)); + : std::min(1., global_ctx->merge_list_element_ptr->progress.load(std::memory_order_relaxed)); global_ctx->space_reservation->update(static_cast((1. - progress) * ctx->initial_reservation)); } @@ -336,7 +336,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() throw Exception("Cancelled merging parts with expired TTL", ErrorCodes::ABORTED); const auto data_settings = global_ctx->data->getSettings(); - const size_t sum_compressed_bytes_upper_bound = (*global_ctx->merge_entry)->total_size_bytes_compressed; + const size_t sum_compressed_bytes_upper_bound = global_ctx->merge_list_element_ptr->total_size_bytes_compressed; ctx->need_sync = needSyncPart(ctx->sum_input_rows_upper_bound, sum_compressed_bytes_upper_bound, *data_settings); return false; @@ -349,9 +349,9 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const if (global_ctx->chosen_merge_algorithm != MergeAlgorithm::Vertical) return false; - size_t sum_input_rows_exact = (*global_ctx->merge_entry)->rows_read; - (*global_ctx->merge_entry)->columns_written = global_ctx->merging_column_names.size(); - (*global_ctx->merge_entry)->progress.store(ctx->column_sizes->keyColumnsWeight(), std::memory_order_relaxed); + size_t sum_input_rows_exact = global_ctx->merge_list_element_ptr->rows_read; + global_ctx->merge_list_element_ptr->columns_written = global_ctx->merging_column_names.size(); + global_ctx->merge_list_element_ptr->progress.store(ctx->column_sizes->keyColumnsWeight(), std::memory_order_relaxed); ctx->column_part_streams = BlockInputStreams(global_ctx->future_part->parts.size()); @@ -385,7 +385,7 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const const String & column_name = ctx->it_name_and_type->name; Names column_names{column_name}; - ctx->progress_before = (*global_ctx->merge_entry)->progress.load(std::memory_order_relaxed); + ctx->progress_before = global_ctx->merge_list_element_ptr->progress.load(std::memory_order_relaxed); global_ctx->column_progress = std::make_unique(ctx->progress_before, ctx->column_sizes->columnWeight(column_name)); @@ -396,7 +396,7 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const /// Dereference unique_ptr column_part_source->setProgressCallback( - MergeProgressCallback(*global_ctx->merge_entry, global_ctx->watch_prev_elapsed, *global_ctx->column_progress)); + MergeProgressCallback(global_ctx->merge_list_element_ptr, global_ctx->watch_prev_elapsed, *global_ctx->column_progress)); QueryPipeline column_part_pipeline; column_part_pipeline.init(Pipe(std::move(column_part_source))); @@ -460,9 +460,9 @@ void MergeTask::VerticalMergeStage::finalizeVerticalMergeForOneColumn() const /// NOTE: 'progress' is modified by single thread, but it may be concurrently read from MergeListElement::getInfo() (StorageSystemMerges). - (*global_ctx->merge_entry)->columns_written += 1; - (*global_ctx->merge_entry)->bytes_written_uncompressed += ctx->column_gathered_stream->getProfileInfo().bytes; - (*global_ctx->merge_entry)->progress.store(ctx->progress_before + ctx->column_sizes->columnWeight(column_name), std::memory_order_relaxed); + global_ctx->merge_list_element_ptr->columns_written += 1; + global_ctx->merge_list_element_ptr->bytes_written_uncompressed += ctx->column_gathered_stream->getProfileInfo().bytes; + global_ctx->merge_list_element_ptr->progress.store(ctx->progress_before + ctx->column_sizes->columnWeight(column_name), std::memory_order_relaxed); /// This is the external cycle increment. ++ctx->column_num_for_vertical_merge; @@ -487,16 +487,16 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c /// Print overall profiling info. NOTE: it may duplicates previous messages { - double elapsed_seconds = (*global_ctx->merge_entry)->watch.elapsedSeconds(); + double elapsed_seconds = global_ctx->merge_list_element_ptr->watch.elapsedSeconds(); LOG_DEBUG(ctx->log, "Merge sorted {} rows, containing {} columns ({} merged, {} gathered) in {} sec., {} rows/sec., {}/sec.", - (*global_ctx->merge_entry)->rows_read, + global_ctx->merge_list_element_ptr->rows_read, global_ctx->all_column_names.size(), global_ctx->merging_column_names.size(), global_ctx->gathering_column_names.size(), elapsed_seconds, - (*global_ctx->merge_entry)->rows_read / elapsed_seconds, - ReadableSize((*global_ctx->merge_entry)->bytes_read_uncompressed / elapsed_seconds)); + global_ctx->merge_list_element_ptr->rows_read / elapsed_seconds, + ReadableSize(global_ctx->merge_list_element_ptr->bytes_read_uncompressed / elapsed_seconds)); } @@ -536,18 +536,18 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c if (projection.type == ProjectionDescription::Type::Aggregate) projection_merging_params.mode = MergeTreeData::MergingParams::Aggregating; - // TODO Should we use a new merge_entry for projection? ctx->tasks_for_projections.emplace_back(std::make_shared( projection_future_part, projection.metadata, global_ctx->merge_entry, + std::make_unique((*global_ctx->merge_entry)->table_id, projection_future_part), global_ctx->time_of_merge, global_ctx->context, global_ctx->space_reservation, global_ctx->deduplicate, global_ctx->deduplicate_by_columns, projection_merging_params, - global_ctx->new_data_part, + global_ctx->new_data_part.get(), "", // empty string for projection global_ctx->data, global_ctx->merges_blocker, @@ -576,21 +576,17 @@ bool MergeTask::MergeProjectionsStage::executeProjections() const bool MergeTask::MergeProjectionsStage::finalizeProjectionsAndWholeMerge() const { - const auto & projections = global_ctx->metadata_snapshot->getProjections(); - - size_t iter = 0; - - for (const auto & projection : projections) + for (const auto & task : ctx->tasks_for_projections) { - auto future = ctx->tasks_for_projections[iter]->getFuture(); - ++iter; - global_ctx->new_data_part->addProjectionPart(projection.name, future.get()); + auto part = task->getFuture().get(); + global_ctx->new_data_part->addProjectionPart(part->name, std::move(part)); } if (global_ctx->chosen_merge_algorithm != MergeAlgorithm::Vertical) global_ctx->to->writeSuffixAndFinalizePart(global_ctx->new_data_part, ctx->need_sync); else - global_ctx->to->writeSuffixAndFinalizePart(global_ctx->new_data_part, ctx->need_sync, &global_ctx->storage_columns, &global_ctx->checksums_gathered_columns); + global_ctx->to->writeSuffixAndFinalizePart( + global_ctx->new_data_part, ctx->need_sync, &global_ctx->storage_columns, &global_ctx->checksums_gathered_columns); global_ctx->promise.set_value(global_ctx->new_data_part); @@ -717,7 +713,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() /// Dereference unique_ptr and pass horizontal_stage_progress by reference input->setProgressCallback( - MergeProgressCallback(*global_ctx->merge_entry, global_ctx->watch_prev_elapsed, *global_ctx->horizontal_stage_progress)); + MergeProgressCallback(global_ctx->merge_list_element_ptr, global_ctx->watch_prev_elapsed, *global_ctx->horizontal_stage_progress)); Pipe pipe(std::move(input)); @@ -822,7 +818,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() MergeAlgorithm MergeTask::ExecuteAndFinalizeHorizontalPart::chooseMergeAlgorithm() const { - const size_t sum_rows_upper_bound = (*global_ctx->merge_entry)->total_rows_count; + const size_t sum_rows_upper_bound = global_ctx->merge_list_element_ptr->total_rows_count; const auto data_settings = global_ctx->data->getSettings(); if (global_ctx->deduplicate) diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index e6bee9a16a3..54b0255fd5c 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -49,13 +49,14 @@ public: FutureMergedMutatedPartPtr future_part_, StorageMetadataPtr metadata_snapshot_, MergeList::Entry * merge_entry_, + std::unique_ptr projection_merge_list_element_, time_t time_of_merge_, ContextPtr context_, ReservationSharedPtr space_reservation_, bool deduplicate_, Names deduplicate_by_columns_, MergeTreeData::MergingParams merging_params_, - MergeTreeDataPartPtr parent_part_, + const IMergeTreeDataPart * parent_part_, String prefix_, MergeTreeData * data_, ActionBlocker * merges_blocker_, @@ -66,6 +67,9 @@ public: global_ctx->future_part = std::move(future_part_); global_ctx->metadata_snapshot = std::move(metadata_snapshot_); global_ctx->merge_entry = std::move(merge_entry_); + global_ctx->projection_merge_list_element = std::move(projection_merge_list_element_); + global_ctx->merge_list_element_ptr + = global_ctx->projection_merge_list_element ? global_ctx->projection_merge_list_element.get() : (*global_ctx->merge_entry)->ptr(); global_ctx->time_of_merge = std::move(time_of_merge_); global_ctx->context = std::move(context_); global_ctx->space_reservation = std::move(space_reservation_); @@ -112,12 +116,16 @@ private: struct GlobalRuntimeContext : public IStageRuntimeContext //-V730 { MergeList::Entry * merge_entry{nullptr}; + /// If not null, use this instead of the global MergeList::Entry. This is for merging projections. + std::unique_ptr projection_merge_list_element; + MergeListElement * merge_list_element_ptr{nullptr}; MergeTreeData * data{nullptr}; ActionBlocker * merges_blocker{nullptr}; ActionBlocker * ttl_merges_blocker{nullptr}; StorageMetadataPtr metadata_snapshot{nullptr}; FutureMergedMutatedPartPtr future_part{nullptr}; - MergeTreeDataPartPtr parent_part{nullptr}; + /// This will be either nullptr or new_data_part, so raw pointer is ok. + const IMergeTreeDataPart * parent_part{nullptr}; ContextPtr context{nullptr}; time_t time_of_merge{0}; ReservationSharedPtr space_reservation{nullptr}; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index bf3d98e3fcf..0c97598dc37 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -416,29 +416,30 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( FutureMergedMutatedPartPtr future_part, const StorageMetadataPtr & metadata_snapshot, MergeList::Entry * merge_entry, - TableLockHolder holder, + std::unique_ptr projection_merge_list_element, + TableLockHolder, time_t time_of_merge, ContextPtr context, ReservationSharedPtr space_reservation, bool deduplicate, const Names & deduplicate_by_columns, const MergeTreeData::MergingParams & merging_params, - const IMergeTreeDataPart * /*parent_part*/, - const String & /*prefix*/) + const IMergeTreeDataPart * parent_part, + const String & prefix) { - (void)holder; return std::make_shared( future_part, const_cast(metadata_snapshot), merge_entry, + std::move(projection_merge_list_element), time_of_merge, context, space_reservation, deduplicate, deduplicate_by_columns, merging_params, - nullptr, - "", + parent_part, + prefix, &data, &merges_blocker, &ttl_merges_blocker); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index cda8cfd2c57..82a7b541369 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -99,6 +99,7 @@ public: FutureMergedMutatedPartPtr future_part, const StorageMetadataPtr & metadata_snapshot, MergeListEntry * merge_entry, + std::unique_ptr projection_merge_list_element, TableLockHolder table_lock_holder, time_t time_of_merge, ContextPtr context, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index f48a9285c53..0655806bf0e 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -631,8 +631,9 @@ public: LOG_DEBUG(log, "Merged {} parts in level {} to {}", selected_parts.size(), current_level, projection_future_part->name); auto tmp_part_merge_task = ctx->mutator->mergePartsToTemporaryPart( projection_future_part, - ctx->metadata_snapshot, + projection.metadata, ctx->mutate_entry, + std::make_unique((*ctx->mutate_entry)->table_id, projection_future_part), *ctx->holder, ctx->time_of_mutation, ctx->context, @@ -1261,7 +1262,7 @@ bool MutateTask::prepare() ctx->mutation_kind = ctx->interpreter->getMutationKind(); ctx->mutating_stream = ctx->interpreter->execute(); ctx->updated_header = ctx->interpreter->getUpdatedHeader(); - ctx->mutating_stream->setProgressCallback(MergeProgressCallback(*ctx->mutate_entry, ctx->watch_prev_elapsed, *ctx->stage_progress)); + ctx->mutating_stream->setProgressCallback(MergeProgressCallback((*ctx->mutate_entry)->ptr(), ctx->watch_prev_elapsed, *ctx->stage_progress)); } ctx->single_disk_volume = std::make_shared("volume_" + ctx->future_part->name, ctx->space_reservation->getDisk(), 0); diff --git a/tests/queries/0_stateless/01710_projection_vertical_merges.reference b/tests/queries/0_stateless/01710_projection_vertical_merges.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01710_projection_vertical_merges.sql b/tests/queries/0_stateless/01710_projection_vertical_merges.sql new file mode 100644 index 00000000000..cb06f473b44 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_vertical_merges.sql @@ -0,0 +1,19 @@ +-- Tags: long + +drop table if exists t; + +create table t (c1 Int64, c2 String, c3 DateTime, c4 Int8, c5 String, c6 String, c7 String, c8 String, c9 String, c10 String, c11 String, c12 String, c13 Int8, c14 Int64, c15 String, c16 String, c17 String, c18 Int64, c19 Int64, c20 Int64) engine MergeTree order by c18; + +insert into t (c1, c18) select number, -number from numbers(2000000); + +alter table t add projection p_norm (select * order by c1); + +optimize table t final; + +alter table t materialize projection p_norm settings mutations_sync = 1; + +set allow_experimental_projection_optimization = 1, max_rows_to_read = 3; + +select c18 from t where c1 < 0; + +drop table t; From 48bdabbc26b79dcb5fc6e93e6750a1a347e99a89 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Sep 2021 18:25:19 +0300 Subject: [PATCH 105/142] Less sleeps in integration tests. --- tests/integration/helpers/cluster.py | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index e56c0d4f3c2..42e406036e9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1748,7 +1748,7 @@ CLICKHOUSE_START_COMMAND = "clickhouse server --config-file=/etc/clickhouse-serv " --log-file=/var/log/clickhouse-server/clickhouse-server.log " \ " --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log" -CLICKHOUSE_STAY_ALIVE_COMMAND = 'bash -c "{} --daemon; tail -f /dev/null"'.format(CLICKHOUSE_START_COMMAND) +CLICKHOUSE_STAY_ALIVE_COMMAND = 'bash -c "trap \'killall tail\' INT TERM; {} --daemon; coproc tail -f /dev/null; wait $$!"'.format(CLICKHOUSE_START_COMMAND) DOCKER_COMPOSE_TEMPLATE = ''' version: '2.3' @@ -2001,10 +2001,18 @@ class ClickHouseInstance: logging.warning("ClickHouse process already stopped") return - self.exec_in_container(["bash", "-c", "pkill {} clickhouse".format("-9" if kill else "")], user='root') - time.sleep(stop_wait_sec) - ps_clickhouse = self.exec_in_container(["bash", "-c", "ps -C clickhouse"], user='root') - if ps_clickhouse != " PID TTY STAT TIME COMMAND" : + sleep_time = 0.1 + num_steps = int(stop_wait_sec / sleep_time) + stopped = False + for step in range(num_steps): + self.exec_in_container(["bash", "-c", "pkill {} clickhouse".format("-9" if kill else "")], user='root') + time.sleep(sleep_time) + ps_clickhouse = self.exec_in_container(["bash", "-c", "ps -C clickhouse"], user='root') + if ps_clickhouse == " PID TTY STAT TIME COMMAND": + stopped = True + break + + if not stopped: logging.warning(f"Force kill clickhouse in stop_clickhouse. ps:{ps_clickhouse}") self.stop_clickhouse(kill=True) except Exception as e: From fe44be522ee11822ac8b86c1ffab8560b5fb37d9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Sep 2021 05:48:24 +0300 Subject: [PATCH 106/142] Fix assert in table function `merge` with database regexp --- src/Interpreters/IdentifierSemantic.cpp | 4 +-- src/Interpreters/IdentifierSemantic.h | 2 +- src/Interpreters/JoinedTables.cpp | 2 +- .../TranslateQualifiedNamesVisitor.cpp | 2 +- .../evaluateConstantExpression.cpp | 18 ---------- src/Interpreters/evaluateConstantExpression.h | 2 -- .../RocksDB/StorageEmbeddedRocksDB.cpp | 6 ++-- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 2 +- src/Storages/StorageMerge.cpp | 34 ++++++++++++++++--- src/Storages/StorageMerge.h | 9 +++-- src/TableFunctions/TableFunctionMerge.cpp | 15 +++----- src/TableFunctions/TableFunctionMerge.h | 6 ++-- .../2024_merge_regexp_assert.reference | 0 .../0_stateless/2024_merge_regexp_assert.sql | 1 + 14 files changed, 52 insertions(+), 51 deletions(-) create mode 100644 tests/queries/0_stateless/2024_merge_regexp_assert.reference create mode 100644 tests/queries/0_stateless/2024_merge_regexp_assert.sql diff --git a/src/Interpreters/IdentifierSemantic.cpp b/src/Interpreters/IdentifierSemantic.cpp index 098bf033399..1112b5c3bda 100644 --- a/src/Interpreters/IdentifierSemantic.cpp +++ b/src/Interpreters/IdentifierSemantic.cpp @@ -162,7 +162,7 @@ IdentifierSemantic::ColumnMatch IdentifierSemantic::canReferColumnToTable(const { /// database.table.column if (doesIdentifierBelongTo(identifier, db_and_table.database, db_and_table.table)) - return ColumnMatch::DbAndTable; + return ColumnMatch::DBAndTable; /// alias.column if (doesIdentifierBelongTo(identifier, db_and_table.alias)) @@ -199,7 +199,7 @@ void IdentifierSemantic::setColumnShortName(ASTIdentifier & identifier, const Da case ColumnMatch::TableAlias: to_strip = 1; break; - case ColumnMatch::DbAndTable: + case ColumnMatch::DBAndTable: to_strip = 2; break; default: diff --git a/src/Interpreters/IdentifierSemantic.h b/src/Interpreters/IdentifierSemantic.h index b36c1ad00dd..cabe755027c 100644 --- a/src/Interpreters/IdentifierSemantic.h +++ b/src/Interpreters/IdentifierSemantic.h @@ -31,7 +31,7 @@ struct IdentifierSemantic ColumnName, /// column qualified with column names list AliasedTableName, /// column qualified with table name (but table has an alias so its priority is lower than TableName) TableName, /// column qualified with table name - DbAndTable, /// column qualified with database and table name + DBAndTable, /// column qualified with database and table name TableAlias, /// column qualified with table alias Ambiguous, }; diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 271d7371425..3aae3982758 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -128,7 +128,7 @@ private: /// Table has an alias. We do not need to rewrite qualified names with table alias (match == ColumnMatch::TableName). auto match = IdentifierSemantic::canReferColumnToTable(identifier, table); if (match == IdentifierSemantic::ColumnMatch::AliasedTableName || - match == IdentifierSemantic::ColumnMatch::DbAndTable) + match == IdentifierSemantic::ColumnMatch::DBAndTable) { if (rewritten) throw Exception("Failed to rewrite distributed table names. Ambiguous column '" + identifier.name() + "'", diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index bf3bbf22b8c..2d1b6b3f239 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -334,7 +334,7 @@ void RestoreQualifiedNamesMatcher::Data::changeTable(ASTIdentifier & identifier) { case IdentifierSemantic::ColumnMatch::AliasedTableName: case IdentifierSemantic::ColumnMatch::TableName: - case IdentifierSemantic::ColumnMatch::DbAndTable: + case IdentifierSemantic::ColumnMatch::DBAndTable: IdentifierSemantic::setColumnLongName(identifier, remote_table); break; default: diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index c05118b7c6a..ae304906476 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -104,24 +104,6 @@ ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, ContextPtr return res; } -std::tuple evaluateDatabaseNameForMergeEngine(const ASTPtr & node, ContextPtr context) -{ - if (const auto * func = node->as(); func && func->name == "REGEXP") - { - if (func->arguments->children.size() != 1) - throw Exception("Arguments for REGEXP in Merge ENGINE should be 1", ErrorCodes::BAD_ARGUMENTS); - - auto * literal = func->arguments->children[0]->as(); - if (!literal || literal->value.safeGet().empty()) - throw Exception("Argument for REGEXP in Merge ENGINE should be a non empty String Literal", ErrorCodes::BAD_ARGUMENTS); - - return std::tuple{true, func->arguments->children[0]}; - } - - auto ast = evaluateConstantExpressionForDatabaseName(node, context); - return std::tuple{false, ast}; -} - namespace { diff --git a/src/Interpreters/evaluateConstantExpression.h b/src/Interpreters/evaluateConstantExpression.h index 3b817080fe0..b95982f5b99 100644 --- a/src/Interpreters/evaluateConstantExpression.h +++ b/src/Interpreters/evaluateConstantExpression.h @@ -53,6 +53,4 @@ ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, ContextPtr */ std::optional evaluateExpressionOverConstantCondition(const ASTPtr & node, const ExpressionActionsPtr & target_expr, size_t & limit); -// Evaluate database name or regexp for StorageMerge and TableFunction merge -std::tuple evaluateDatabaseNameForMergeEngine(const ASTPtr & node, ContextPtr context); } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 459c0879cda..549d939c3cc 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -280,7 +280,7 @@ StorageEmbeddedRocksDB::StorageEmbeddedRocksDB(const StorageID & table_id_, { fs::create_directories(rocksdb_dir); } - initDb(); + initDB(); } void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & , ContextPtr, TableExclusiveLockHolder &) @@ -288,10 +288,10 @@ void StorageEmbeddedRocksDB::truncate(const ASTPtr &, const StorageMetadataPtr & rocksdb_ptr->Close(); fs::remove_all(rocksdb_dir); fs::create_directories(rocksdb_dir); - initDb(); + initDB(); } -void StorageEmbeddedRocksDB::initDb() +void StorageEmbeddedRocksDB::initDB() { rocksdb::Status status; rocksdb::Options base; diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 3f1b3b49492..358da9835ce 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -65,6 +65,6 @@ private: RocksDBPtr rocksdb_ptr; String rocksdb_dir; - void initDb(); + void initDB(); }; } diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index c1066329e6f..df39662de05 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -49,7 +49,7 @@ StorageMerge::StorageMerge( const String & comment, const String & source_database_name_or_regexp_, bool database_is_regexp_, - const DbToTableSetMap & source_databases_and_tables_, + const DBToTableSetMap & source_databases_and_tables_, ContextPtr context_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) @@ -573,11 +573,14 @@ DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const String & datab { auto database = DatabaseCatalog::instance().getDatabase(database_name); - auto table_name_match = [this, &database_name](const String & table_name_) -> bool { + auto table_name_match = [this, database_name](const String & table_name_) -> bool + { if (source_databases_and_tables) { - const auto & source_tables = (*source_databases_and_tables).at(database_name); - return source_tables.count(table_name_); + if (auto it = source_databases_and_tables->find(database_name); it != source_databases_and_tables->end()) + return it->second.count(table_name_); + else + return false; } else return source_table_regexp->match(table_name_); @@ -742,6 +745,26 @@ IStorage::ColumnSizeByName StorageMerge::getColumnSizes() const return first_materialized_mysql->getColumnSizes(); } + +std::tuple StorageMerge::evaluateDatabaseName(const ASTPtr & node, ContextPtr context_) +{ + if (const auto * func = node->as(); func && func->name == "REGEXP") + { + if (func->arguments->children.size() != 1) + throw Exception("REGEXP in Merge ENGINE takes only one argument", ErrorCodes::BAD_ARGUMENTS); + + auto * literal = func->arguments->children[0]->as(); + if (!literal || literal->value.safeGet().empty()) + throw Exception("Argument for REGEXP in Merge ENGINE should be a non empty String Literal", ErrorCodes::BAD_ARGUMENTS); + + return {true, func->arguments->children[0]}; + } + + auto ast = evaluateConstantExpressionForDatabaseName(node, context_); + return {false, ast}; +} + + void registerStorageMerge(StorageFactory & factory) { factory.registerStorage("Merge", [](const StorageFactory::Arguments & args) @@ -757,10 +780,11 @@ void registerStorageMerge(StorageFactory & factory) " - name of source database and regexp for table names.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - auto [is_regexp, database_ast] = evaluateDatabaseNameForMergeEngine(engine_args[0], args.getLocalContext()); + auto [is_regexp, database_ast] = StorageMerge::evaluateDatabaseName(engine_args[0], args.getLocalContext()); if (!is_regexp) engine_args[0] = database_ast; + String source_database_name_or_regexp = database_ast->as().value.safeGet(); engine_args[1] = evaluateConstantExpressionAsLiteral(engine_args[1], args.getLocalContext()); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 20460e95156..b6001815f85 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -48,12 +48,15 @@ public: bool mayBenefitFromIndexForIn( const ASTPtr & left_in_operand, ContextPtr query_context, const StorageMetadataPtr & metadata_snapshot) const override; + /// Evaluate database name or regexp for StorageMerge and TableFunction merge + static std::tuple evaluateDatabaseName(const ASTPtr & node, ContextPtr context); + private: - using DbToTableSetMap = std::map>; + using DBToTableSetMap = std::map>; std::optional source_database_regexp; std::optional source_table_regexp; - std::optional source_databases_and_tables; + std::optional source_databases_and_tables; String source_database_name_or_regexp; bool database_is_regexp = false; @@ -86,7 +89,7 @@ protected: const String & comment, const String & source_database_name_or_regexp_, bool database_is_regexp_, - const DbToTableSetMap & source_databases_and_tables_, + const DBToTableSetMap & source_databases_and_tables_, ContextPtr context_); StorageMerge( diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index 81dde4a12a4..f1ef4262d08 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -52,7 +52,7 @@ void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, ContextPtr " - name of source database and regexp for table names.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - auto [is_regexp, database_ast] = evaluateDatabaseNameForMergeEngine(args[0], context); + auto [is_regexp, database_ast] = StorageMerge::evaluateDatabaseName(args[0], context); database_is_regexp = is_regexp; @@ -65,7 +65,7 @@ void TableFunctionMerge::parseArguments(const ASTPtr & ast_function, ContextPtr } -const TableFunctionMerge::DbToTableSetMap & TableFunctionMerge::getSourceDatabasesAndTables(ContextPtr context) const +const TableFunctionMerge::DBToTableSetMap & TableFunctionMerge::getSourceDatabasesAndTables(ContextPtr context) const { if (source_databases_and_tables) return *source_databases_and_tables; @@ -88,17 +88,10 @@ const TableFunctionMerge::DbToTableSetMap & TableFunctionMerge::getSourceDatabas auto databases = DatabaseCatalog::instance().getDatabases(); for (const auto & db : databases) - { if (database_re.match(db.first)) - { - auto source_tables = getMatchedTablesWithAccess(db.first, source_table_regexp, context); + (*source_databases_and_tables)[db.first] = getMatchedTablesWithAccess(db.first, source_table_regexp, context); - if (!source_tables.empty()) - (*source_databases_and_tables)[db.first] = source_tables; - } - } - - if ((*source_databases_and_tables).empty()) + if (source_databases_and_tables->empty()) throwNoTablesMatchRegexp(source_database_name_or_regexp, source_table_regexp); } diff --git a/src/TableFunctions/TableFunctionMerge.h b/src/TableFunctions/TableFunctionMerge.h index 73b61f8eb79..10221c8c72c 100644 --- a/src/TableFunctions/TableFunctionMerge.h +++ b/src/TableFunctions/TableFunctionMerge.h @@ -21,8 +21,8 @@ private: const char * getStorageTypeName() const override { return "Merge"; } using TableSet = std::set; - using DbToTableSetMap = std::map; - const DbToTableSetMap & getSourceDatabasesAndTables(ContextPtr context) const; + using DBToTableSetMap = std::map; + const DBToTableSetMap & getSourceDatabasesAndTables(ContextPtr context) const; ColumnsDescription getActualTableStructure(ContextPtr context) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; static TableSet getMatchedTablesWithAccess(const String & database_name, const String & table_regexp, const ContextPtr & context); @@ -30,7 +30,7 @@ private: String source_database_name_or_regexp; String source_table_regexp; bool database_is_regexp = false; - mutable std::optional source_databases_and_tables; + mutable std::optional source_databases_and_tables; }; diff --git a/tests/queries/0_stateless/2024_merge_regexp_assert.reference b/tests/queries/0_stateless/2024_merge_regexp_assert.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/2024_merge_regexp_assert.sql b/tests/queries/0_stateless/2024_merge_regexp_assert.sql new file mode 100644 index 00000000000..3a3c2920f0b --- /dev/null +++ b/tests/queries/0_stateless/2024_merge_regexp_assert.sql @@ -0,0 +1 @@ +SELECT a FROM merge(REGEXP('.'), 'query_log'); -- { serverError 47 } From 34017c942e5135b12ba758d23c913da1f92f89a8 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 25 Sep 2021 13:38:56 +0800 Subject: [PATCH 107/142] More fixes --- src/Storages/MergeTree/MergePlainMergeTreeTask.h | 10 +++++----- src/Storages/MergeTree/MutatePlainMergeTreeTask.h | 12 ++++++------ src/Storages/StorageMergeTree.cpp | 11 +++++++---- 3 files changed, 18 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.h b/src/Storages/MergeTree/MergePlainMergeTreeTask.h index f199557684c..9a95a404135 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.h @@ -23,12 +23,12 @@ public: TableLockHolder table_lock_holder_, Callback && task_result_callback_) : storage(storage_) - , metadata_snapshot(metadata_snapshot_) + , metadata_snapshot(std::move(metadata_snapshot_)) , deduplicate(deduplicate_) - , deduplicate_by_columns(deduplicate_by_columns_) - , merge_mutate_entry(merge_mutate_entry_) - , table_lock_holder(table_lock_holder_) - , task_result_callback(task_result_callback_) {} + , deduplicate_by_columns(std::move(deduplicate_by_columns_)) + , merge_mutate_entry(std::move(merge_mutate_entry_)) + , table_lock_holder(std::move(table_lock_holder_)) + , task_result_callback(std::forward(task_result_callback_)) {} bool executeStep() override; diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h index 0b65c202eb6..e8f210f4175 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.h +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.h @@ -27,13 +27,13 @@ public: StorageMergeTree & storage_, StorageMetadataPtr metadata_snapshot_, MergeMutateSelectedEntryPtr merge_mutate_entry_, - TableLockHolder & table_lock_holder_, + TableLockHolder table_lock_holder_, Callback && task_result_callback_) : storage(storage_) - , metadata_snapshot(metadata_snapshot_) - , merge_mutate_entry(merge_mutate_entry_) - , table_lock_holder(table_lock_holder_) - , task_result_callback(task_result_callback_) {} + , metadata_snapshot(std::move(metadata_snapshot_)) + , merge_mutate_entry(std::move(merge_mutate_entry_)) + , table_lock_holder(std::move(table_lock_holder_)) + , task_result_callback(std::forward(task_result_callback_)) {} bool executeStep() override; @@ -60,7 +60,7 @@ private: StorageMetadataPtr metadata_snapshot; MergeMutateSelectedEntryPtr merge_mutate_entry{nullptr}; - TableLockHolder & table_lock_holder; + TableLockHolder table_lock_holder; FutureMergedMutatedPartPtr future_part{nullptr}; std::unique_ptr stopwatch; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index e2a0c36b463..ff45496b0e4 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1035,7 +1035,8 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign if (merge_entry) { - auto task = std::make_shared(*this, metadata_snapshot, false, Names{}, merge_entry, share_lock, common_assignee_trigger); + auto task = std::make_shared( + *this, metadata_snapshot, false, Names{}, merge_entry, share_lock, common_assignee_trigger); assignee.scheduleMergeMutateTask(task); return true; } @@ -1054,7 +1055,8 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign } bool scheduled = false; - if (time_after_previous_cleanup_temporary_directories.compareAndRestartDeferred(getContext()->getSettingsRef().merge_tree_clear_old_temporary_directories_interval_seconds)) + if (time_after_previous_cleanup_temporary_directories.compareAndRestartDeferred( + getContext()->getSettingsRef().merge_tree_clear_old_temporary_directories_interval_seconds)) { assignee.scheduleMergeMutateTask(ExecutableLambdaAdapter::create( [this, share_lock] () @@ -1064,7 +1066,8 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign }, common_assignee_trigger, getStorageID())); scheduled = true; } - if (auto lock = time_after_previous_cleanup_parts.compareAndRestartDeferred(getContext()->getSettingsRef().merge_tree_clear_old_parts_interval_seconds)) + if (auto lock = time_after_previous_cleanup_parts.compareAndRestartDeferred( + getContext()->getSettingsRef().merge_tree_clear_old_parts_interval_seconds)) { assignee.scheduleMergeMutateTask(ExecutableLambdaAdapter::create( [this, share_lock] () @@ -1078,7 +1081,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign return true; }, common_assignee_trigger, getStorageID())); scheduled = true; - } + } return scheduled; } From 3022092a369b6175ff85a797129b748529109397 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 25 Sep 2021 16:16:33 +0800 Subject: [PATCH 108/142] Make long test no parallel --- tests/queries/0_stateless/01710_projection_vertical_merges.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01710_projection_vertical_merges.sql b/tests/queries/0_stateless/01710_projection_vertical_merges.sql index cb06f473b44..d54fef7e71d 100644 --- a/tests/queries/0_stateless/01710_projection_vertical_merges.sql +++ b/tests/queries/0_stateless/01710_projection_vertical_merges.sql @@ -1,4 +1,4 @@ --- Tags: long +-- Tags: long, no-parallel drop table if exists t; From ef425310ffbbbf481b4d6e2a2bcdaad02104eb08 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 25 Sep 2021 11:36:52 +0300 Subject: [PATCH 109/142] Try use empty sink --- src/Core/ExternalTable.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 1f08e252cdf..ccef23cc833 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -161,7 +162,7 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, auto storage = temporary_table.getTable(); getContext()->addExternalTable(data->table_name, std::move(temporary_table)); auto sink = storage->write(ASTPtr(), storage->getInMemoryMetadataPtr(), getContext()); - auto exception_handling = std::make_shared(sink->getOutputPort().getHeader()); + auto exception_handling = std::make_shared(sink->getOutputPort().getHeader()); /// Write data data->pipe->resize(1); From 2121682882b43c89af3c612d91187b979e47f9c0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 Sep 2021 14:50:58 +0000 Subject: [PATCH 110/142] Update libhdfs update --- contrib/libhdfs3 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libhdfs3 b/contrib/libhdfs3 index 22781c4ebd3..082e55f17d1 160000 --- a/contrib/libhdfs3 +++ b/contrib/libhdfs3 @@ -1 +1 @@ -Subproject commit 22781c4ebd31aee918c38b3af7b2e8da5adec93e +Subproject commit 082e55f17d1c58bf124290fb044fea40e985ec11 From 9dac348893dc32fb9be02122f5474a570788662a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 25 Sep 2021 19:41:50 +0300 Subject: [PATCH 111/142] Update StorageMerge.cpp --- src/Storages/StorageMerge.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index df39662de05..aaa375d645e 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -35,6 +35,7 @@ namespace DB namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; extern const int ILLEGAL_PREWHERE; From 2deb9643af5644d68d62b0808315e0fa853a25c2 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 25 Sep 2021 10:12:32 -0700 Subject: [PATCH 112/142] Rename h3ResIsClassIII -> h3IsResClassIII --- .../{h3ResIsClassIII.cpp => h3IsResClassIII.cpp} | 10 +++++----- src/Functions/registerFunctionsGeo.cpp | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) rename src/Functions/{h3ResIsClassIII.cpp => h3IsResClassIII.cpp} (88%) diff --git a/src/Functions/h3ResIsClassIII.cpp b/src/Functions/h3IsResClassIII.cpp similarity index 88% rename from src/Functions/h3ResIsClassIII.cpp rename to src/Functions/h3IsResClassIII.cpp index 044e6d565ac..25a2fc1952f 100644 --- a/src/Functions/h3ResIsClassIII.cpp +++ b/src/Functions/h3IsResClassIII.cpp @@ -24,12 +24,12 @@ extern const int ILLEGAL_TYPE_OF_ARGUMENT; namespace { -class FunctionH3ResIsClassIII : public IFunction +class FunctionH3IsResClassIII : public IFunction { public: - static constexpr auto name = "h3ResIsClassIII"; + static constexpr auto name = "h3IsResClassIII"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } + static FunctionPtr create(ContextPtr) { return std::make_shared(); } std::string getName() const override { return name; } @@ -69,9 +69,9 @@ public: } -void registerFunctionH3ResIsClassIII(FunctionFactory & factory) +void registerFunctionH3IsResClassIII(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/registerFunctionsGeo.cpp b/src/Functions/registerFunctionsGeo.cpp index a86ff18ec65..a0ae38f6b85 100644 --- a/src/Functions/registerFunctionsGeo.cpp +++ b/src/Functions/registerFunctionsGeo.cpp @@ -42,7 +42,7 @@ void registerFunctionH3IndexesAreNeighbors(FunctionFactory &); void registerFunctionStringToH3(FunctionFactory &); void registerFunctionH3ToString(FunctionFactory &); void registerFunctionH3HexAreaM2(FunctionFactory &); -void registerFunctionH3ResIsClassIII(FunctionFactory &); +void registerFunctionH3IsResClassIII(FunctionFactory &); void registerFunctionH3IsPentagon(FunctionFactory &); void registerFunctionH3GetFaces(FunctionFactory &); #endif @@ -98,7 +98,7 @@ void registerFunctionsGeo(FunctionFactory & factory) registerFunctionStringToH3(factory); registerFunctionH3ToString(factory); registerFunctionH3HexAreaM2(factory); - registerFunctionH3ResIsClassIII(factory); + registerFunctionH3IsResClassIII(factory); registerFunctionH3IsPentagon(factory); registerFunctionH3GetFaces(factory); #endif From 79473f2edc4b2221d179a1a443ca7503399f2f6a Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 25 Sep 2021 10:13:42 -0700 Subject: [PATCH 113/142] Fix tests --- ...{2021_h3_res_is_classIII.sql => 2021_h3_is_res_classIII.sql} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename tests/queries/0_stateless/{2021_h3_res_is_classIII.sql => 2021_h3_is_res_classIII.sql} (88%) diff --git a/tests/queries/0_stateless/2021_h3_res_is_classIII.sql b/tests/queries/0_stateless/2021_h3_is_res_classIII.sql similarity index 88% rename from tests/queries/0_stateless/2021_h3_res_is_classIII.sql rename to tests/queries/0_stateless/2021_h3_is_res_classIII.sql index 67dae6d1fa2..7b9cc7acb48 100644 --- a/tests/queries/0_stateless/2021_h3_res_is_classIII.sql +++ b/tests/queries/0_stateless/2021_h3_is_res_classIII.sql @@ -11,6 +11,6 @@ INSERT INTO h3_indexes VALUES (geoToH3(10.0, 0.0, 1)); INSERT INTO h3_indexes VALUES (geoToH3(0.0, 10.0, 2)); INSERT INTO h3_indexes VALUES (geoToH3(10.0, 10.0, 3)); -SELECT h3ResIsClassIII(h3_index) FROM h3_indexes ORDER BY h3_index; +SELECT h3IsResClassIII(h3_index) FROM h3_indexes ORDER BY h3_index; DROP TABLE h3_indexes; From a5a8d132d0b4dfd0893608b54133505c01b18086 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 25 Sep 2021 10:15:12 -0700 Subject: [PATCH 114/142] Fix doc references --- docs/en/sql-reference/functions/geo/h3.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md index faa86527c7d..3c3ed7b8932 100644 --- a/docs/en/sql-reference/functions/geo/h3.md +++ b/docs/en/sql-reference/functions/geo/h3.md @@ -594,14 +594,14 @@ Result: └─────┘ ``` -## h3ResIsClassIII {#h3resisclassIII} +## h3IsResClassIII {#h3isresclassIII} Returns whether [H3](#h3index) index has a resolution with Class III orientation. **Syntax** ``` sql -h3ResIsClassIII(index) +h3IsResClassIII(index) ``` **Parameter** @@ -620,7 +620,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT h3ResIsClassIII(617420388352917503) as res; +SELECT h3IsResClassIII(617420388352917503) as res; ``` Result: From cd178496079287e540c2c36fc2ccf11eb5a90183 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 25 Sep 2021 13:11:23 -0700 Subject: [PATCH 115/142] fix test - rename reference file --- ...es_is_classIII.reference => 2021_h3_is_res_classIII.reference} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{2021_h3_res_is_classIII.reference => 2021_h3_is_res_classIII.reference} (100%) diff --git a/tests/queries/0_stateless/2021_h3_res_is_classIII.reference b/tests/queries/0_stateless/2021_h3_is_res_classIII.reference similarity index 100% rename from tests/queries/0_stateless/2021_h3_res_is_classIII.reference rename to tests/queries/0_stateless/2021_h3_is_res_classIII.reference From 23b6ac82a7fd85d6c37c9d5d472899665133515d Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 25 Sep 2021 18:06:49 -0700 Subject: [PATCH 116/142] Fix comments in AggregateFunctionFactory --- src/AggregateFunctions/AggregateFunctionFactory.cpp | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index c9a44dba6f2..71ed4608134 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -155,7 +155,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( } /// Combinators of aggregate functions. - /// For every aggregate function 'agg' and combiner '-Comb' there is combined aggregate function with name 'aggComb', + /// For every aggregate function 'agg' and combiner '-Comb' there is a combined aggregate function with the name 'aggComb', /// that can have different number and/or types of arguments, different result type and different behaviour. if (AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix(name)) @@ -172,13 +172,12 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( String nested_name = name.substr(0, name.size() - combinator_name.size()); /// Nested identical combinators (i.e. uniqCombinedIfIf) is not - /// supported (since they even don't work -- silently). + /// supported (since they don't work -- silently). /// - /// But non-identical does supported and works, for example - /// uniqCombinedIfMergeIf, it is useful in case when the underlying + /// But non-identical is supported and works. For example, + /// uniqCombinedIfMergeIf is useful in cases when the underlying /// storage stores AggregateFunction(uniqCombinedIf) and in SELECT you - /// need to filter aggregation result based on another column for - /// example. + /// need to filter aggregation result based on another column. if (!combinator->supportsNesting() && nested_name.ends_with(combinator_name)) { throw Exception(ErrorCodes::ILLEGAL_AGGREGATION, @@ -234,7 +233,7 @@ std::optional AggregateFunctionFactory::tryGetPrope return found.properties; /// Combinators of aggregate functions. - /// For every aggregate function 'agg' and combiner '-Comb' there is combined aggregate function with name 'aggComb', + /// For every aggregate function 'agg' and combiner '-Comb' there is a combined aggregate function with name 'aggComb', /// that can have different number and/or types of arguments, different result type and different behaviour. if (AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix(name)) From 0e9af142dfb6c9418a277aa78a77ffb9a4e58297 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 25 Sep 2021 18:19:47 -0700 Subject: [PATCH 117/142] Fix one more comment --- src/AggregateFunctions/AggregateFunctionFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index 71ed4608134..4c2c64ef5e1 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -233,7 +233,7 @@ std::optional AggregateFunctionFactory::tryGetPrope return found.properties; /// Combinators of aggregate functions. - /// For every aggregate function 'agg' and combiner '-Comb' there is a combined aggregate function with name 'aggComb', + /// For every aggregate function 'agg' and combiner '-Comb' there is a combined aggregate function with the name 'aggComb', /// that can have different number and/or types of arguments, different result type and different behaviour. if (AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix(name)) From 121dfe26b6a8ec181bba91182111a61a8cf0bbce Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Sun, 26 Sep 2021 08:13:58 +0300 Subject: [PATCH 118/142] Fixed resetting nullptr field --- src/Parsers/IAST.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index f5b3630c7f1..bdbf3b2b448 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -49,6 +49,9 @@ size_t IAST::checkSize(size_t max_size) const void IAST::reset(IAST *& field) { + if (field == nullptr) + return; + const auto child = std::find_if(children.begin(), children.end(), [field](const auto & p) { return p.get() == field; From 2c6b9aa174772d3aca3eb3fbdf4f18f42e9689ee Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 21 Sep 2021 19:43:46 +0300 Subject: [PATCH 119/142] Better exception messages for some String-related functions --- src/Functions/CountSubstringsImpl.h | 5 +++-- src/Functions/FunctionsMultiStringFuzzySearch.h | 4 ++-- src/Functions/FunctionsMultiStringSearch.h | 4 ++-- src/Functions/FunctionsStringSearch.h | 6 +++--- src/Functions/FunctionsVisitParam.h | 11 ++++++----- src/Functions/HasTokenImpl.h | 11 ++++++----- src/Functions/MatchImpl.h | 10 ++++++---- src/Functions/MultiMatchAllIndicesImpl.h | 4 +++- src/Functions/MultiMatchAnyImpl.h | 6 ++++-- src/Functions/MultiSearchFirstIndexImpl.h | 4 +++- src/Functions/MultiSearchFirstPositionImpl.h | 4 +++- src/Functions/MultiSearchImpl.h | 4 +++- src/Functions/PositionImpl.h | 5 +++-- src/Functions/countSubstrings.cpp | 2 +- src/Functions/countSubstringsCaseInsensitive.cpp | 2 +- src/Functions/countSubstringsCaseInsensitiveUTF8.cpp | 3 ++- src/Functions/hasToken.cpp | 2 +- src/Functions/hasTokenCaseInsensitive.cpp | 2 +- src/Functions/ilike.cpp | 4 ++-- src/Functions/like.cpp | 4 ++-- src/Functions/match.cpp | 2 +- src/Functions/multiFuzzyMatchAllIndices.cpp | 3 +-- src/Functions/multiFuzzyMatchAny.cpp | 3 +-- src/Functions/multiFuzzyMatchAnyIndex.cpp | 3 +-- src/Functions/multiMatchAllIndices.cpp | 3 +-- src/Functions/multiMatchAny.cpp | 3 +-- src/Functions/multiMatchAnyIndex.cpp | 3 +-- src/Functions/multiSearchAny.cpp | 2 +- src/Functions/multiSearchAnyCaseInsensitive.cpp | 2 +- src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp | 2 +- src/Functions/multiSearchAnyUTF8.cpp | 2 +- src/Functions/multiSearchFirstIndex.cpp | 2 +- .../multiSearchFirstIndexCaseInsensitive.cpp | 2 +- .../multiSearchFirstIndexCaseInsensitiveUTF8.cpp | 2 +- src/Functions/multiSearchFirstIndexUTF8.cpp | 2 +- src/Functions/multiSearchFirstPosition.cpp | 2 +- .../multiSearchFirstPositionCaseInsensitive.cpp | 2 +- .../multiSearchFirstPositionCaseInsensitiveUTF8.cpp | 3 +-- src/Functions/multiSearchFirstPositionUTF8.cpp | 2 +- src/Functions/notILike.cpp | 4 ++-- src/Functions/notLike.cpp | 2 +- src/Functions/position.cpp | 2 +- src/Functions/positionCaseInsensitive.cpp | 2 +- src/Functions/positionCaseInsensitiveUTF8.cpp | 2 +- src/Functions/positionUTF8.cpp | 2 +- src/Functions/visitParamExtractBool.cpp | 4 ++-- src/Functions/visitParamExtractFloat.cpp | 4 ++-- src/Functions/visitParamExtractInt.cpp | 4 ++-- src/Functions/visitParamExtractUInt.cpp | 4 ++-- src/Functions/visitParamHas.cpp | 4 ++-- 50 files changed, 93 insertions(+), 83 deletions(-) diff --git a/src/Functions/CountSubstringsImpl.h b/src/Functions/CountSubstringsImpl.h index 2c2ae20488f..6668ca0a392 100644 --- a/src/Functions/CountSubstringsImpl.h +++ b/src/Functions/CountSubstringsImpl.h @@ -19,11 +19,12 @@ namespace ErrorCodes /// NOTE: Intersecting substrings in haystack accounted only once, i.e.: /// /// countSubstrings('aaaa', 'aa') == 2 -template +template struct CountSubstringsImpl { static constexpr bool use_default_implementation_for_constants = false; static constexpr bool supports_start_pos = true; + static constexpr auto name = Name::name; using ResultType = UInt64; @@ -225,7 +226,7 @@ struct CountSubstringsImpl template static void vectorFixedConstant(Args &&...) { - throw Exception("Functions 'position' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); } }; diff --git a/src/Functions/FunctionsMultiStringFuzzySearch.h b/src/Functions/FunctionsMultiStringFuzzySearch.h index f0e1437b2aa..0bbb8ade939 100644 --- a/src/Functions/FunctionsMultiStringFuzzySearch.h +++ b/src/Functions/FunctionsMultiStringFuzzySearch.h @@ -28,13 +28,13 @@ namespace ErrorCodes } -template +template class FunctionsMultiStringFuzzySearch : public IFunction { static_assert(LimitArgs > 0); public: - static constexpr auto name = Name::name; + static constexpr auto name = Impl::name; static FunctionPtr create(ContextPtr context) { if (Impl::is_using_hyperscan && !context->getSettingsRef().allow_hyperscan) diff --git a/src/Functions/FunctionsMultiStringSearch.h b/src/Functions/FunctionsMultiStringSearch.h index 6535035469f..4576979681b 100644 --- a/src/Functions/FunctionsMultiStringSearch.h +++ b/src/Functions/FunctionsMultiStringSearch.h @@ -41,13 +41,13 @@ namespace ErrorCodes /// The argument limiting raises from Volnitsky searcher -- it is performance crucial to save only one byte for pattern number. /// But some other searchers use this function, for example, multiMatchAny -- hyperscan does not have such restrictions -template ::max()> +template ::max()> class FunctionsMultiStringSearch : public IFunction { static_assert(LimitArgs > 0); public: - static constexpr auto name = Name::name; + static constexpr auto name = Impl::name; static FunctionPtr create(ContextPtr context) { if (Impl::is_using_hyperscan && !context->getSettingsRef().allow_hyperscan) diff --git a/src/Functions/FunctionsStringSearch.h b/src/Functions/FunctionsStringSearch.h index d8463e69cf3..4aa76ceec28 100644 --- a/src/Functions/FunctionsStringSearch.h +++ b/src/Functions/FunctionsStringSearch.h @@ -46,11 +46,11 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -template +template class FunctionsStringSearch : public IFunction { public: - static constexpr auto name = Name::name; + static constexpr auto name = Impl::name; static FunctionPtr create(ContextPtr) { return std::make_shared(); } String getName() const override { return name; } @@ -80,7 +80,7 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() < 2 || 3 < arguments.size()) - throw Exception("Number of arguments for function " + String(Name::name) + " doesn't match: passed " + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 2 or 3.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); diff --git a/src/Functions/FunctionsVisitParam.h b/src/Functions/FunctionsVisitParam.h index 922ea44be9a..362c3bcd693 100644 --- a/src/Functions/FunctionsVisitParam.h +++ b/src/Functions/FunctionsVisitParam.h @@ -74,13 +74,14 @@ struct ExtractNumericType * If a field was not found or an incorrect value is associated with the field, * then the default value used - 0. */ -template +template struct ExtractParamImpl { using ResultType = typename ParamExtractor::ResultType; static constexpr bool use_default_implementation_for_constants = true; static constexpr bool supports_start_pos = false; + static constexpr auto name = Name::name; /// It is assumed that `res` is the correct size and initialized with zeros. static void vectorConstant( @@ -91,7 +92,7 @@ struct ExtractParamImpl PaddedPODArray & res) { if (start_pos != nullptr) - throw Exception("Functions 'visitParamHas' and 'visitParamExtract*' doesn't support start_pos argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' doesn't support start_pos argument", name); /// We are looking for a parameter simply as a substring of the form "name" needle = "\"" + needle + "\":"; @@ -131,18 +132,18 @@ struct ExtractParamImpl template static void vectorVector(Args &&...) { - throw Exception("Functions 'visitParamHas' and 'visitParamExtract*' doesn't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support non-constant needle argument", name); } template static void constantVector(Args &&...) { - throw Exception("Functions 'visitParamHas' and 'visitParamExtract*' doesn't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support non-constant needle argument", name); } template static void vectorFixedConstant(Args &&...) { - throw Exception("Functions 'visitParamHas' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); } }; diff --git a/src/Functions/HasTokenImpl.h b/src/Functions/HasTokenImpl.h index 044c50b6742..ec33a07fce3 100644 --- a/src/Functions/HasTokenImpl.h +++ b/src/Functions/HasTokenImpl.h @@ -14,13 +14,14 @@ namespace ErrorCodes /** Token search the string, means that needle must be surrounded by some separator chars, like whitespace or puctuation. */ -template +template struct HasTokenImpl { using ResultType = UInt8; static constexpr bool use_default_implementation_for_constants = true; static constexpr bool supports_start_pos = false; + static constexpr auto name = Name::name; static void vectorConstant( const ColumnString::Chars & data, @@ -30,7 +31,7 @@ struct HasTokenImpl PaddedPODArray & res) { if (start_pos != nullptr) - throw Exception("Function 'hasToken' does not support start_pos argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' does not support start_pos argument", name); if (offsets.empty()) return; @@ -72,20 +73,20 @@ struct HasTokenImpl template static void vectorVector(Args &&...) { - throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support non-constant needle argument", name); } /// Search different needles in single haystack. template static void constantVector(Args &&...) { - throw Exception("Function 'hasToken' does not support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support non-constant needle argument", name); } template static void vectorFixedConstant(Args &&...) { - throw Exception("Functions 'hasToken' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); } }; diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index e1e8394f7b1..4c8db97aded 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -73,11 +73,12 @@ static inline bool likePatternIsStrstr(const String & pattern, String & res) * NOTE: We want to run regexp search for whole columns by one call (as implemented in function 'position') * but for that, regexp engine must support \0 bytes and their interpretation as string boundaries. */ -template +template struct MatchImpl { static constexpr bool use_default_implementation_for_constants = true; static constexpr bool supports_start_pos = false; + static constexpr auto name = Name::name; using ResultType = UInt8; @@ -93,7 +94,8 @@ struct MatchImpl PaddedPODArray & res) { if (start_pos != nullptr) - throw Exception("Functions 'like' and 'match' don't support start_pos argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Function '{}' doesn't support start_pos argument", name); if (offsets.empty()) return; @@ -406,14 +408,14 @@ struct MatchImpl template static void vectorVector(Args &&...) { - throw Exception("Functions 'like' and 'match' don't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support non-constant needle argument", name); } /// Search different needles in single haystack. template static void constantVector(Args &&...) { - throw Exception("Functions 'like' and 'match' don't support non-constant needle argument", ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support non-constant needle argument", name); } }; diff --git a/src/Functions/MultiMatchAllIndicesImpl.h b/src/Functions/MultiMatchAllIndicesImpl.h index f0a94ed5eed..922ec7cf06b 100644 --- a/src/Functions/MultiMatchAllIndicesImpl.h +++ b/src/Functions/MultiMatchAllIndicesImpl.h @@ -29,7 +29,7 @@ namespace ErrorCodes } -template +template struct MultiMatchAllIndicesImpl { using ResultType = Type; @@ -37,6 +37,8 @@ struct MultiMatchAllIndicesImpl /// Variable for understanding, if we used offsets for the output, most /// likely to determine whether the function returns ColumnVector of ColumnArray. static constexpr bool is_column_array = true; + static constexpr auto name = Name::name; + static auto getReturnType() { return std::make_shared(std::make_shared()); diff --git a/src/Functions/MultiMatchAnyImpl.h b/src/Functions/MultiMatchAnyImpl.h index c8d0ea6c87f..e2e54887a32 100644 --- a/src/Functions/MultiMatchAnyImpl.h +++ b/src/Functions/MultiMatchAnyImpl.h @@ -29,7 +29,7 @@ namespace ErrorCodes } -template +template struct MultiMatchAnyImpl { static_assert(static_cast(FindAny) + static_cast(FindAnyIndex) == 1); @@ -38,6 +38,8 @@ struct MultiMatchAnyImpl /// Variable for understanding, if we used offsets for the output, most /// likely to determine whether the function returns ColumnVector of ColumnArray. static constexpr bool is_column_array = false; + static constexpr auto name = Name::name; + static auto getReturnType() { return std::make_shared>(); @@ -120,7 +122,7 @@ struct MultiMatchAnyImpl memset(accum.data(), 0, accum.size()); for (size_t j = 0; j < needles.size(); ++j) { - MatchImpl::vectorConstant(haystack_data, haystack_offsets, needles[j].toString(), nullptr, accum); + MatchImpl::vectorConstant(haystack_data, haystack_offsets, needles[j].toString(), nullptr, accum); for (size_t i = 0; i < res.size(); ++i) { if constexpr (FindAny) diff --git a/src/Functions/MultiSearchFirstIndexImpl.h b/src/Functions/MultiSearchFirstIndexImpl.h index 4b29577d0eb..26709119f6e 100644 --- a/src/Functions/MultiSearchFirstIndexImpl.h +++ b/src/Functions/MultiSearchFirstIndexImpl.h @@ -7,7 +7,7 @@ namespace DB { -template +template struct MultiSearchFirstIndexImpl { using ResultType = UInt64; @@ -15,6 +15,8 @@ struct MultiSearchFirstIndexImpl /// Variable for understanding, if we used offsets for the output, most /// likely to determine whether the function returns ColumnVector of ColumnArray. static constexpr bool is_column_array = false; + static constexpr auto name = Name::name; + static auto getReturnType() { return std::make_shared>(); } static void vectorConstant( diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index bb1017c43ee..1db8dcbde83 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -7,7 +7,7 @@ namespace DB { -template +template struct MultiSearchFirstPositionImpl { using ResultType = UInt64; @@ -15,6 +15,8 @@ struct MultiSearchFirstPositionImpl /// Variable for understanding, if we used offsets for the output, most /// likely to determine whether the function returns ColumnVector of ColumnArray. static constexpr bool is_column_array = false; + static constexpr auto name = Name::name; + static auto getReturnType() { return std::make_shared>(); } static void vectorConstant( diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index 461af5c3295..7cb0cefe580 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -7,7 +7,7 @@ namespace DB { -template +template struct MultiSearchImpl { using ResultType = UInt8; @@ -15,6 +15,8 @@ struct MultiSearchImpl /// Variable for understanding, if we used offsets for the output, most /// likely to determine whether the function returns ColumnVector of ColumnArray. static constexpr bool is_column_array = false; + static constexpr auto name = Name::name; + static auto getReturnType() { return std::make_shared>(); } static void vectorConstant( diff --git a/src/Functions/PositionImpl.h b/src/Functions/PositionImpl.h index fe9b49bd2b8..d3b6d74c3cd 100644 --- a/src/Functions/PositionImpl.h +++ b/src/Functions/PositionImpl.h @@ -175,11 +175,12 @@ struct PositionCaseInsensitiveUTF8 }; -template +template struct PositionImpl { static constexpr bool use_default_implementation_for_constants = false; static constexpr bool supports_start_pos = true; + static constexpr auto name = Name::name; using ResultType = UInt64; @@ -408,7 +409,7 @@ struct PositionImpl template static void vectorFixedConstant(Args &&...) { - throw Exception("Functions 'position' don't support FixedString haystack argument", ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Function '{}' doesn't support FixedString haystack argument", name); } }; diff --git a/src/Functions/countSubstrings.cpp b/src/Functions/countSubstrings.cpp index d0dd469b962..1bf95f9526f 100644 --- a/src/Functions/countSubstrings.cpp +++ b/src/Functions/countSubstrings.cpp @@ -13,7 +13,7 @@ struct NameCountSubstrings static constexpr auto name = "countSubstrings"; }; -using FunctionCountSubstrings = FunctionsStringSearch, NameCountSubstrings>; +using FunctionCountSubstrings = FunctionsStringSearch>; } diff --git a/src/Functions/countSubstringsCaseInsensitive.cpp b/src/Functions/countSubstringsCaseInsensitive.cpp index 9207f2d4f20..fa234953cc3 100644 --- a/src/Functions/countSubstringsCaseInsensitive.cpp +++ b/src/Functions/countSubstringsCaseInsensitive.cpp @@ -13,7 +13,7 @@ struct NameCountSubstringsCaseInsensitive static constexpr auto name = "countSubstringsCaseInsensitive"; }; -using FunctionCountSubstringsCaseInsensitive = FunctionsStringSearch, NameCountSubstringsCaseInsensitive>; +using FunctionCountSubstringsCaseInsensitive = FunctionsStringSearch>; } diff --git a/src/Functions/countSubstringsCaseInsensitiveUTF8.cpp b/src/Functions/countSubstringsCaseInsensitiveUTF8.cpp index b1747bbd7ff..93f77fddd7f 100644 --- a/src/Functions/countSubstringsCaseInsensitiveUTF8.cpp +++ b/src/Functions/countSubstringsCaseInsensitiveUTF8.cpp @@ -13,7 +13,8 @@ struct NameCountSubstringsCaseInsensitiveUTF8 static constexpr auto name = "countSubstringsCaseInsensitiveUTF8"; }; -using FunctionCountSubstringsCaseInsensitiveUTF8 = FunctionsStringSearch, NameCountSubstringsCaseInsensitiveUTF8>; +using FunctionCountSubstringsCaseInsensitiveUTF8 = FunctionsStringSearch< + CountSubstringsImpl>; } diff --git a/src/Functions/hasToken.cpp b/src/Functions/hasToken.cpp index f20edffbdd8..72d6c11a5fa 100644 --- a/src/Functions/hasToken.cpp +++ b/src/Functions/hasToken.cpp @@ -14,7 +14,7 @@ struct NameHasToken static constexpr auto name = "hasToken"; }; -using FunctionHasToken = FunctionsStringSearch, NameHasToken>; +using FunctionHasToken = FunctionsStringSearch>; } diff --git a/src/Functions/hasTokenCaseInsensitive.cpp b/src/Functions/hasTokenCaseInsensitive.cpp index 28f5b9e80c1..a0280bc12a5 100644 --- a/src/Functions/hasTokenCaseInsensitive.cpp +++ b/src/Functions/hasTokenCaseInsensitive.cpp @@ -15,7 +15,7 @@ struct NameHasTokenCaseInsensitive }; using FunctionHasTokenCaseInsensitive - = FunctionsStringSearch, NameHasTokenCaseInsensitive>; + = FunctionsStringSearch>; } diff --git a/src/Functions/ilike.cpp b/src/Functions/ilike.cpp index fc3e38daeba..116c945e04f 100644 --- a/src/Functions/ilike.cpp +++ b/src/Functions/ilike.cpp @@ -12,8 +12,8 @@ struct NameILike static constexpr auto name = "ilike"; }; -using ILikeImpl = MatchImpl; -using FunctionILike = FunctionsStringSearch; +using ILikeImpl = MatchImpl; +using FunctionILike = FunctionsStringSearch; } diff --git a/src/Functions/like.cpp b/src/Functions/like.cpp index 37d9f006187..1ac9a8d7dab 100644 --- a/src/Functions/like.cpp +++ b/src/Functions/like.cpp @@ -13,8 +13,8 @@ struct NameLike static constexpr auto name = "like"; }; -using LikeImpl = MatchImpl; -using FunctionLike = FunctionsStringSearch; +using LikeImpl = MatchImpl; +using FunctionLike = FunctionsStringSearch; } diff --git a/src/Functions/match.cpp b/src/Functions/match.cpp index 3460d54c6b6..31d36577445 100644 --- a/src/Functions/match.cpp +++ b/src/Functions/match.cpp @@ -13,7 +13,7 @@ struct NameMatch static constexpr auto name = "match"; }; -using FunctionMatch = FunctionsStringSearch, NameMatch>; +using FunctionMatch = FunctionsStringSearch>; } diff --git a/src/Functions/multiFuzzyMatchAllIndices.cpp b/src/Functions/multiFuzzyMatchAllIndices.cpp index 8b104e9ed2d..d0121ee3981 100644 --- a/src/Functions/multiFuzzyMatchAllIndices.cpp +++ b/src/Functions/multiFuzzyMatchAllIndices.cpp @@ -14,8 +14,7 @@ struct NameMultiFuzzyMatchAllIndices }; using FunctionMultiFuzzyMatchAllIndices = FunctionsMultiStringFuzzySearch< - MultiMatchAllIndicesImpl, - NameMultiFuzzyMatchAllIndices, + MultiMatchAllIndicesImpl, std::numeric_limits::max()>; } diff --git a/src/Functions/multiFuzzyMatchAny.cpp b/src/Functions/multiFuzzyMatchAny.cpp index 4d0c3470d91..640e93a23b0 100644 --- a/src/Functions/multiFuzzyMatchAny.cpp +++ b/src/Functions/multiFuzzyMatchAny.cpp @@ -14,8 +14,7 @@ struct NameMultiFuzzyMatchAny }; using FunctionMultiFuzzyMatchAny = FunctionsMultiStringFuzzySearch< - MultiMatchAnyImpl, - NameMultiFuzzyMatchAny, + MultiMatchAnyImpl, std::numeric_limits::max()>; } diff --git a/src/Functions/multiFuzzyMatchAnyIndex.cpp b/src/Functions/multiFuzzyMatchAnyIndex.cpp index 1680f413154..f8bad1bc461 100644 --- a/src/Functions/multiFuzzyMatchAnyIndex.cpp +++ b/src/Functions/multiFuzzyMatchAnyIndex.cpp @@ -14,8 +14,7 @@ struct NameMultiFuzzyMatchAnyIndex }; using FunctionMultiFuzzyMatchAnyIndex = FunctionsMultiStringFuzzySearch< - MultiMatchAnyImpl, - NameMultiFuzzyMatchAnyIndex, + MultiMatchAnyImpl, std::numeric_limits::max()>; } diff --git a/src/Functions/multiMatchAllIndices.cpp b/src/Functions/multiMatchAllIndices.cpp index 171fa6baf74..940c9e7e3bf 100644 --- a/src/Functions/multiMatchAllIndices.cpp +++ b/src/Functions/multiMatchAllIndices.cpp @@ -14,8 +14,7 @@ struct NameMultiMatchAllIndices }; using FunctionMultiMatchAllIndices = FunctionsMultiStringSearch< - MultiMatchAllIndicesImpl, - NameMultiMatchAllIndices, + MultiMatchAllIndicesImpl, std::numeric_limits::max()>; } diff --git a/src/Functions/multiMatchAny.cpp b/src/Functions/multiMatchAny.cpp index 146c27e250c..47510e0ecc2 100644 --- a/src/Functions/multiMatchAny.cpp +++ b/src/Functions/multiMatchAny.cpp @@ -14,8 +14,7 @@ struct NameMultiMatchAny }; using FunctionMultiMatchAny = FunctionsMultiStringSearch< - MultiMatchAnyImpl, - NameMultiMatchAny, + MultiMatchAnyImpl, std::numeric_limits::max()>; } diff --git a/src/Functions/multiMatchAnyIndex.cpp b/src/Functions/multiMatchAnyIndex.cpp index c43cd061187..a56d41dc95b 100644 --- a/src/Functions/multiMatchAnyIndex.cpp +++ b/src/Functions/multiMatchAnyIndex.cpp @@ -14,8 +14,7 @@ struct NameMultiMatchAnyIndex }; using FunctionMultiMatchAnyIndex = FunctionsMultiStringSearch< - MultiMatchAnyImpl, - NameMultiMatchAnyIndex, + MultiMatchAnyImpl, std::numeric_limits::max()>; } diff --git a/src/Functions/multiSearchAny.cpp b/src/Functions/multiSearchAny.cpp index 5cd688ac65d..113289b83ed 100644 --- a/src/Functions/multiSearchAny.cpp +++ b/src/Functions/multiSearchAny.cpp @@ -14,7 +14,7 @@ struct NameMultiSearchAny static constexpr auto name = "multiSearchAny"; }; -using FunctionMultiSearch = FunctionsMultiStringSearch, NameMultiSearchAny>; +using FunctionMultiSearch = FunctionsMultiStringSearch>; } diff --git a/src/Functions/multiSearchAnyCaseInsensitive.cpp b/src/Functions/multiSearchAnyCaseInsensitive.cpp index 2358ce64bf8..9bc950c0d3d 100644 --- a/src/Functions/multiSearchAnyCaseInsensitive.cpp +++ b/src/Functions/multiSearchAnyCaseInsensitive.cpp @@ -14,7 +14,7 @@ struct NameMultiSearchAnyCaseInsensitive static constexpr auto name = "multiSearchAnyCaseInsensitive"; }; using FunctionMultiSearchCaseInsensitive - = FunctionsMultiStringSearch, NameMultiSearchAnyCaseInsensitive>; + = FunctionsMultiStringSearch>; } diff --git a/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp b/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp index f84762d2bb4..c83dc843f78 100644 --- a/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp +++ b/src/Functions/multiSearchAnyCaseInsensitiveUTF8.cpp @@ -15,7 +15,7 @@ struct NameMultiSearchAnyCaseInsensitiveUTF8 }; using FunctionMultiSearchCaseInsensitiveUTF8 - = FunctionsMultiStringSearch, NameMultiSearchAnyCaseInsensitiveUTF8>; + = FunctionsMultiStringSearch>; } diff --git a/src/Functions/multiSearchAnyUTF8.cpp b/src/Functions/multiSearchAnyUTF8.cpp index b2c8342ba7f..3f34f70ab51 100644 --- a/src/Functions/multiSearchAnyUTF8.cpp +++ b/src/Functions/multiSearchAnyUTF8.cpp @@ -13,7 +13,7 @@ struct NameMultiSearchAnyUTF8 { static constexpr auto name = "multiSearchAnyUTF8"; }; -using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch, NameMultiSearchAnyUTF8>; +using FunctionMultiSearchUTF8 = FunctionsMultiStringSearch>; } diff --git a/src/Functions/multiSearchFirstIndex.cpp b/src/Functions/multiSearchFirstIndex.cpp index fcbeb552ae1..a96ebed029c 100644 --- a/src/Functions/multiSearchFirstIndex.cpp +++ b/src/Functions/multiSearchFirstIndex.cpp @@ -15,7 +15,7 @@ struct NameMultiSearchFirstIndex }; using FunctionMultiSearchFirstIndex - = FunctionsMultiStringSearch, NameMultiSearchFirstIndex>; + = FunctionsMultiStringSearch>; } diff --git a/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp b/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp index 87483734cf9..cc4869d1200 100644 --- a/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp +++ b/src/Functions/multiSearchFirstIndexCaseInsensitive.cpp @@ -15,7 +15,7 @@ struct NameMultiSearchFirstIndexCaseInsensitive }; using FunctionMultiSearchFirstIndexCaseInsensitive - = FunctionsMultiStringSearch, NameMultiSearchFirstIndexCaseInsensitive>; + = FunctionsMultiStringSearch>; } diff --git a/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp b/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp index 69e14adb32a..fd95947bc67 100644 --- a/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp +++ b/src/Functions/multiSearchFirstIndexCaseInsensitiveUTF8.cpp @@ -15,7 +15,7 @@ struct NameMultiSearchFirstIndexCaseInsensitiveUTF8 }; using FunctionMultiSearchFirstIndexCaseInsensitiveUTF8 - = FunctionsMultiStringSearch, NameMultiSearchFirstIndexCaseInsensitiveUTF8>; + = FunctionsMultiStringSearch>; } diff --git a/src/Functions/multiSearchFirstIndexUTF8.cpp b/src/Functions/multiSearchFirstIndexUTF8.cpp index 699281dad4b..6854201d14d 100644 --- a/src/Functions/multiSearchFirstIndexUTF8.cpp +++ b/src/Functions/multiSearchFirstIndexUTF8.cpp @@ -15,7 +15,7 @@ struct NameMultiSearchFirstIndexUTF8 }; using FunctionMultiSearchFirstIndexUTF8 - = FunctionsMultiStringSearch, NameMultiSearchFirstIndexUTF8>; + = FunctionsMultiStringSearch>; } diff --git a/src/Functions/multiSearchFirstPosition.cpp b/src/Functions/multiSearchFirstPosition.cpp index 003345afde6..4ca1ac35a4d 100644 --- a/src/Functions/multiSearchFirstPosition.cpp +++ b/src/Functions/multiSearchFirstPosition.cpp @@ -15,7 +15,7 @@ struct NameMultiSearchFirstPosition }; using FunctionMultiSearchFirstPosition - = FunctionsMultiStringSearch, NameMultiSearchFirstPosition>; + = FunctionsMultiStringSearch>; } diff --git a/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp b/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp index 7aa1ef991f3..4e356335e98 100644 --- a/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp +++ b/src/Functions/multiSearchFirstPositionCaseInsensitive.cpp @@ -15,7 +15,7 @@ struct NameMultiSearchFirstPositionCaseInsensitive }; using FunctionMultiSearchFirstPositionCaseInsensitive - = FunctionsMultiStringSearch, NameMultiSearchFirstPositionCaseInsensitive>; + = FunctionsMultiStringSearch>; } diff --git a/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp b/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp index d20ce6c2de3..647fc3a2cc8 100644 --- a/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp +++ b/src/Functions/multiSearchFirstPositionCaseInsensitiveUTF8.cpp @@ -15,8 +15,7 @@ struct NameMultiSearchFirstPositionCaseInsensitiveUTF8 }; using FunctionMultiSearchFirstPositionCaseInsensitiveUTF8 = FunctionsMultiStringSearch< - MultiSearchFirstPositionImpl, - NameMultiSearchFirstPositionCaseInsensitiveUTF8>; + MultiSearchFirstPositionImpl>; } diff --git a/src/Functions/multiSearchFirstPositionUTF8.cpp b/src/Functions/multiSearchFirstPositionUTF8.cpp index c0739808f7b..fbb1099ec35 100644 --- a/src/Functions/multiSearchFirstPositionUTF8.cpp +++ b/src/Functions/multiSearchFirstPositionUTF8.cpp @@ -15,7 +15,7 @@ struct NameMultiSearchFirstPositionUTF8 }; using FunctionMultiSearchFirstPositionUTF8 - = FunctionsMultiStringSearch, NameMultiSearchFirstPositionUTF8>; + = FunctionsMultiStringSearch>; } diff --git a/src/Functions/notILike.cpp b/src/Functions/notILike.cpp index 1fc0ab65ea8..be40e2b989e 100644 --- a/src/Functions/notILike.cpp +++ b/src/Functions/notILike.cpp @@ -12,8 +12,8 @@ struct NameNotILike static constexpr auto name = "notILike"; }; -using NotILikeImpl = MatchImpl; -using FunctionNotILike = FunctionsStringSearch; +using NotILikeImpl = MatchImpl; +using FunctionNotILike = FunctionsStringSearch; } diff --git a/src/Functions/notLike.cpp b/src/Functions/notLike.cpp index 54e2c7481f0..7c4ea8ab2dc 100644 --- a/src/Functions/notLike.cpp +++ b/src/Functions/notLike.cpp @@ -12,7 +12,7 @@ struct NameNotLike static constexpr auto name = "notLike"; }; -using FunctionNotLike = FunctionsStringSearch, NameNotLike>; +using FunctionNotLike = FunctionsStringSearch>; } diff --git a/src/Functions/position.cpp b/src/Functions/position.cpp index 5b8af16fef1..e38dc52b9af 100644 --- a/src/Functions/position.cpp +++ b/src/Functions/position.cpp @@ -13,7 +13,7 @@ struct NamePosition static constexpr auto name = "position"; }; -using FunctionPosition = FunctionsStringSearch, NamePosition>; +using FunctionPosition = FunctionsStringSearch>; } diff --git a/src/Functions/positionCaseInsensitive.cpp b/src/Functions/positionCaseInsensitive.cpp index f72766a1875..ed9d86c033c 100644 --- a/src/Functions/positionCaseInsensitive.cpp +++ b/src/Functions/positionCaseInsensitive.cpp @@ -13,7 +13,7 @@ struct NamePositionCaseInsensitive static constexpr auto name = "positionCaseInsensitive"; }; -using FunctionPositionCaseInsensitive = FunctionsStringSearch, NamePositionCaseInsensitive>; +using FunctionPositionCaseInsensitive = FunctionsStringSearch>; } diff --git a/src/Functions/positionCaseInsensitiveUTF8.cpp b/src/Functions/positionCaseInsensitiveUTF8.cpp index 0af545cc6a3..f6e344b119e 100644 --- a/src/Functions/positionCaseInsensitiveUTF8.cpp +++ b/src/Functions/positionCaseInsensitiveUTF8.cpp @@ -14,7 +14,7 @@ struct NamePositionCaseInsensitiveUTF8 }; using FunctionPositionCaseInsensitiveUTF8 - = FunctionsStringSearch, NamePositionCaseInsensitiveUTF8>; + = FunctionsStringSearch>; } diff --git a/src/Functions/positionUTF8.cpp b/src/Functions/positionUTF8.cpp index 68b2f5c274e..ecb2a1e9e97 100644 --- a/src/Functions/positionUTF8.cpp +++ b/src/Functions/positionUTF8.cpp @@ -13,7 +13,7 @@ struct NamePositionUTF8 static constexpr auto name = "positionUTF8"; }; -using FunctionPositionUTF8 = FunctionsStringSearch, NamePositionUTF8>; +using FunctionPositionUTF8 = FunctionsStringSearch>; } diff --git a/src/Functions/visitParamExtractBool.cpp b/src/Functions/visitParamExtractBool.cpp index 059115b5b13..48fb78ba9b6 100644 --- a/src/Functions/visitParamExtractBool.cpp +++ b/src/Functions/visitParamExtractBool.cpp @@ -17,10 +17,10 @@ struct ExtractBool }; struct NameVisitParamExtractBool { static constexpr auto name = "visitParamExtractBool"; }; -using FunctionVisitParamExtractBool = FunctionsStringSearch, NameVisitParamExtractBool>; +using FunctionVisitParamExtractBool = FunctionsStringSearch>; struct NameSimpleJSONExtractBool { static constexpr auto name = "simpleJSONExtractBool"; }; -using FunctionSimpleJSONExtractBool = FunctionsStringSearch, NameSimpleJSONExtractBool>; +using FunctionSimpleJSONExtractBool = FunctionsStringSearch>; void registerFunctionVisitParamExtractBool(FunctionFactory & factory) { diff --git a/src/Functions/visitParamExtractFloat.cpp b/src/Functions/visitParamExtractFloat.cpp index 7a55cff365c..e7967b6de2c 100644 --- a/src/Functions/visitParamExtractFloat.cpp +++ b/src/Functions/visitParamExtractFloat.cpp @@ -7,10 +7,10 @@ namespace DB { struct NameVisitParamExtractFloat { static constexpr auto name = "visitParamExtractFloat"; }; -using FunctionVisitParamExtractFloat = FunctionsStringSearch>, NameVisitParamExtractFloat>; +using FunctionVisitParamExtractFloat = FunctionsStringSearch>>; struct NameSimpleJSONExtractFloat { static constexpr auto name = "simpleJSONExtractFloat"; }; -using FunctionSimpleJSONExtractFloat = FunctionsStringSearch>, NameSimpleJSONExtractFloat>; +using FunctionSimpleJSONExtractFloat = FunctionsStringSearch>>; void registerFunctionVisitParamExtractFloat(FunctionFactory & factory) { diff --git a/src/Functions/visitParamExtractInt.cpp b/src/Functions/visitParamExtractInt.cpp index 7c2188c10fc..b7f1050972c 100644 --- a/src/Functions/visitParamExtractInt.cpp +++ b/src/Functions/visitParamExtractInt.cpp @@ -7,10 +7,10 @@ namespace DB { struct NameVisitParamExtractInt { static constexpr auto name = "visitParamExtractInt"; }; -using FunctionVisitParamExtractInt = FunctionsStringSearch>, NameVisitParamExtractInt>; +using FunctionVisitParamExtractInt = FunctionsStringSearch>>; struct NameSimpleJSONExtractInt { static constexpr auto name = "simpleJSONExtractInt"; }; -using FunctionSimpleJSONExtractInt = FunctionsStringSearch>, NameSimpleJSONExtractInt>; +using FunctionSimpleJSONExtractInt = FunctionsStringSearch>>; void registerFunctionVisitParamExtractInt(FunctionFactory & factory) { diff --git a/src/Functions/visitParamExtractUInt.cpp b/src/Functions/visitParamExtractUInt.cpp index f5466a63b0d..d89b796263e 100644 --- a/src/Functions/visitParamExtractUInt.cpp +++ b/src/Functions/visitParamExtractUInt.cpp @@ -7,10 +7,10 @@ namespace DB { struct NameVisitParamExtractUInt { static constexpr auto name = "visitParamExtractUInt"; }; -using FunctionVisitParamExtractUInt = FunctionsStringSearch>, NameVisitParamExtractUInt>; +using FunctionVisitParamExtractUInt = FunctionsStringSearch>>; struct NameSimpleJSONExtractUInt { static constexpr auto name = "simpleJSONExtractUInt"; }; -using FunctionSimpleJSONExtractUInt = FunctionsStringSearch>, NameSimpleJSONExtractUInt>; +using FunctionSimpleJSONExtractUInt = FunctionsStringSearch>>; void registerFunctionVisitParamExtractUInt(FunctionFactory & factory) diff --git a/src/Functions/visitParamHas.cpp b/src/Functions/visitParamHas.cpp index f4f377f9e8f..71d69ef5768 100644 --- a/src/Functions/visitParamHas.cpp +++ b/src/Functions/visitParamHas.cpp @@ -17,10 +17,10 @@ struct HasParam }; struct NameVisitParamHas { static constexpr auto name = "visitParamHas"; }; -using FunctionVisitParamHas = FunctionsStringSearch, NameVisitParamHas>; +using FunctionVisitParamHas = FunctionsStringSearch>; struct NameSimpleJSONHas { static constexpr auto name = "simpleJSONHas"; }; -using FunctionSimpleJSONHas = FunctionsStringSearch, NameSimpleJSONHas>; +using FunctionSimpleJSONHas = FunctionsStringSearch>; void registerFunctionVisitParamHas(FunctionFactory & factory) { From d9e3419197155da39ab3d3016714c0638cfee9f9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 26 Sep 2021 08:21:59 +0300 Subject: [PATCH 120/142] Restore test_rocksdb_options::test_invalid_column_family_options test This reverts commit 1881c67eff6390caf22d8186131626b539a50c61. --- tests/integration/test_rocksdb_options/test.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/integration/test_rocksdb_options/test.py b/tests/integration/test_rocksdb_options/test.py index 6689c232081..35285905170 100644 --- a/tests/integration/test_rocksdb_options/test.py +++ b/tests/integration/test_rocksdb_options/test.py @@ -58,6 +58,16 @@ def test_valid_column_family_options(start_cluster): DROP TABLE test; """) +def test_invalid_column_family_options(start_cluster): + node.exec_in_container(['bash', '-c', "sed -i 's/num_levels/no_such_column_family_option/g' /etc/clickhouse-server/config.d/rocksdb.xml"]) + node.restart_clickhouse() + with pytest.raises(QueryRuntimeException): + node.query(""" + CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key); + """) + node.exec_in_container(['bash', '-c', "sed -i 's/no_such_column_family_option/num_levels/g' /etc/clickhouse-server/config.d/rocksdb.xml"]) + node.restart_clickhouse() + def test_table_valid_column_family_options(start_cluster): node.query(""" CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key); From ad1e9e42fc3a49840ae11acc9dd0933f130f760d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 26 Sep 2021 08:23:21 +0300 Subject: [PATCH 121/142] Restore test_rocksdb_options::test_table_invalid_column_family_options test This reverts commit e40a13bd1f959131eec2f9d8c138036410773059. --- tests/integration/test_rocksdb_options/test.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/integration/test_rocksdb_options/test.py b/tests/integration/test_rocksdb_options/test.py index 35285905170..e8542749d8d 100644 --- a/tests/integration/test_rocksdb_options/test.py +++ b/tests/integration/test_rocksdb_options/test.py @@ -73,3 +73,13 @@ def test_table_valid_column_family_options(start_cluster): CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key); DROP TABLE test; """) + +def test_table_invalid_column_family_options(start_cluster): + node.exec_in_container(['bash', '-c', "sed -i 's/max_bytes_for_level_base/no_such_table_column_family_option/g' /etc/clickhouse-server/config.d/rocksdb.xml"]) + node.restart_clickhouse() + with pytest.raises(QueryRuntimeException): + node.query(""" + CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key); + """) + node.exec_in_container(['bash', '-c', "sed -i 's/no_such_table_column_family_option/max_bytes_for_level_base/g' /etc/clickhouse-server/config.d/rocksdb.xml"]) + node.restart_clickhouse() From 874aef3e9a916ab31432d86c49778ac0e94bfd40 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sun, 26 Sep 2021 15:57:11 +0300 Subject: [PATCH 122/142] Revert to head15 Revert to head15 --- docs/ru/getting-started/example-datasets/uk-price-paid.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index 1e5acde6ee3..a08dc35fa2a 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -647,4 +647,3 @@ no projection: 100 rows in set. Elapsed: 0.069 sec. Processed 26.32 million rows ### Online Playground {#playground} Этот набор данных доступен в [Online Playground](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). - From b9a0e04a154141f6802bb5d83c467c06f05e0a84 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sun, 26 Sep 2021 16:34:48 +0300 Subject: [PATCH 123/142] Fix doc check --- docs/ru/getting-started/example-datasets/opensky.md | 1 + docs/ru/getting-started/example-datasets/uk-price-paid.md | 1 + 2 files changed, 2 insertions(+) diff --git a/docs/ru/getting-started/example-datasets/opensky.md b/docs/ru/getting-started/example-datasets/opensky.md index 4000173aa79..bda5dec3c47 100644 --- a/docs/ru/getting-started/example-datasets/opensky.md +++ b/docs/ru/getting-started/example-datasets/opensky.md @@ -419,3 +419,4 @@ ORDER BY k ASC; ### Online Playground {#playground} Вы можете протестировать другие запросы к этому набору данным с помощью интерактивного ресурса [Online Playground](https://gh-api.clickhouse.tech/play?user=play). Например, [вот так](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICBvcmlnaW4sCiAgICBjb3VudCgpLAogICAgcm91bmQoYXZnKGdlb0Rpc3RhbmNlKGxvbmdpdHVkZV8xLCBsYXRpdHVkZV8xLCBsb25naXR1ZGVfMiwgbGF0aXR1ZGVfMikpKSBBUyBkaXN0YW5jZSwKICAgIGJhcihkaXN0YW5jZSwgMCwgMTAwMDAwMDAsIDEwMCkgQVMgYmFyCkZST00gb3BlbnNreQpXSEVSRSBvcmlnaW4gIT0gJycKR1JPVVAgQlkgb3JpZ2luCk9SREVSIEJZIGNvdW50KCkgREVTQwpMSU1JVCAxMDA=). Однако обратите внимание, что здесь нельзя создавать временные таблицы. + diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index a08dc35fa2a..1e5acde6ee3 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -647,3 +647,4 @@ no projection: 100 rows in set. Elapsed: 0.069 sec. Processed 26.32 million rows ### Online Playground {#playground} Этот набор данных доступен в [Online Playground](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIHRvd24sIGRpc3RyaWN0LCBjb3VudCgpIEFTIGMsIHJvdW5kKGF2ZyhwcmljZSkpIEFTIHByaWNlLCBiYXIocHJpY2UsIDAsIDUwMDAwMDAsIDEwMCkgRlJPTSB1a19wcmljZV9wYWlkIFdIRVJFIGRhdGUgPj0gJzIwMjAtMDEtMDEnIEdST1VQIEJZIHRvd24sIGRpc3RyaWN0IEhBVklORyBjID49IDEwMCBPUkRFUiBCWSBwcmljZSBERVNDIExJTUlUIDEwMA==). + From b05a03660b6ae8a3ac0ac0438ceb3f3d5f28c2fb Mon Sep 17 00:00:00 2001 From: Mohamad Fadhil Date: Sun, 26 Sep 2021 21:54:55 +0800 Subject: [PATCH 124/142] Add GraphCDN in adopter list --- docs/en/introduction/adopters.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 9060cd37d6a..dcc0d812a03 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -56,6 +56,7 @@ toc_title: Adopters | Geniee | Ad network | Main product | — | — | [Blog post in Japanese, July 2017](https://tech.geniee.co.jp/entry/2017/07/20/160100) | | Genotek | Bioinformatics | Main product | — | — | [Video, August 2020](https://youtu.be/v3KyZbz9lEE) | | Glaber | Monitoring | Main product | — | — | [Website](https://glaber.io/) | +| GraphCDN | CDN | Traffic Analytics | — | — | [Blog Post in English, August 2021](https://altinity.com/blog/delivering-insight-on-graphql-apis-with-clickhouse-at-graphcdn/) | | HUYA | Video Streaming | Analytics | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/7.%20ClickHouse万亿数据分析实践%20李本旺(sundy-li)%20虎牙.pdf) | | ICA | FinTech | Risk Management | — | — | [Blog Post in English, Sep 2020](https://altinity.com/blog/clickhouse-vs-redshift-performance-for-fintech-risk-management?utm_campaign=ClickHouse%20vs%20RedShift&utm_content=143520807&utm_medium=social&utm_source=twitter&hss_channel=tw-3894792263) | | Idealista | Real Estate | Analytics | — | — | [Blog Post in English, April 2019](https://clickhouse.com/blog/en/clickhouse-meetup-in-madrid-on-april-2-2019) | From 824a54d2181c422f0363955770c284698cf8b4ec Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sun, 26 Sep 2021 17:07:52 +0300 Subject: [PATCH 125/142] Links fixed --- docs/en/getting-started/example-datasets/uk-price-paid.md | 2 +- docs/ru/getting-started/example-datasets/uk-price-paid.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/uk-price-paid.md b/docs/en/getting-started/example-datasets/uk-price-paid.md index 61f31a0cfb1..72b6e4b3718 100644 --- a/docs/en/getting-started/example-datasets/uk-price-paid.md +++ b/docs/en/getting-started/example-datasets/uk-price-paid.md @@ -57,7 +57,7 @@ The preprocessing is: - coverting the `time` field to date as it only contains 00:00 time; - ignoring the [UUid](../../sql-reference/data-types/uuid.md) field because we don't need it for analysis; - transforming `type` and `duration` to more readable Enum fields with function [transform](../../sql-reference/functions/other-functions.md#transform); -- transforming `is_new` and `category` fields from single-character string (`Y`/`N` and `A`/`B`) to [UInt8](../../sql-reference/data-types/int-uint.md##uint8-uint16-uint32-uint64-uint256-int8-int16-int32-int64-int128-int256) field with 0 and 1. +- transforming `is_new` and `category` fields from single-character string (`Y`/`N` and `A`/`B`) to [UInt8](../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-uint256-int8-int16-int32-int64-int128-int256) field with 0 and 1. Preprocessed data is piped directly to `clickhouse-client` to be inserted into ClickHouse table in streaming fashion. diff --git a/docs/ru/getting-started/example-datasets/uk-price-paid.md b/docs/ru/getting-started/example-datasets/uk-price-paid.md index 1e5acde6ee3..1a0991015bd 100644 --- a/docs/ru/getting-started/example-datasets/uk-price-paid.md +++ b/docs/ru/getting-started/example-datasets/uk-price-paid.md @@ -57,7 +57,7 @@ CREATE TABLE uk_price_paid - преобразование поля `time` в дату, поскольку оно содержит только время 00:00; - поле [UUid](../../sql-reference/data-types/uuid.md) игнорируется, потому что оно не будет использовано для анализа; - преобразование полей `type` и `duration` в более читаемые поля типа `Enum` с помощью функции [transform](../../sql-reference/functions/other-functions.md#transform); -- преобразование полей `is_new` и `category` из односимвольной строки (`Y`/`N` и `A`/`B`) в поле [UInt8](../../sql-reference/data-types/int-uint.md##uint8-uint16-uint32-uint64-uint256-int8-int16-int32-int64-int128-int256) со значениями 0 и 1 соответственно. +- преобразование полей `is_new` и `category` из односимвольной строки (`Y`/`N` и `A`/`B`) в поле [UInt8](../../sql-reference/data-types/int-uint.md#uint8-uint16-uint32-uint64-uint256-int8-int16-int32-int64-int128-int256) со значениями 0 и 1 соответственно. Обработанные данные передаются в `clickhouse-client` и импортируются в таблицу ClickHouse потоковым способом. From efbd8e4911e2d404c71bb29046ccfb2bc7635ca9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 26 Sep 2021 17:54:59 +0300 Subject: [PATCH 126/142] Remove ExceptionHandlingSink. --- src/Core/ExternalTable.cpp | 1 - .../MySQL/MaterializedMySQLSyncThread.cpp | 1 - src/Interpreters/GlobalSubqueriesVisitor.h | 1 - src/Interpreters/InterpreterInsertQuery.cpp | 4 +- src/Interpreters/executeQuery.cpp | 1 - .../PushingAsyncPipelineExecutor.cpp | 1 - src/Processors/QueryPipeline.cpp | 6 +- src/Processors/Sinks/ExceptionHandlingSink.h | 51 -------- src/Server/GRPCServer.cpp | 114 +++++++++++------- src/Storages/LiveView/StorageLiveView.cpp | 4 +- .../MaterializedPostgreSQLConsumer.cpp | 1 - .../PostgreSQLReplicationHandler.cpp | 1 - 12 files changed, 79 insertions(+), 107 deletions(-) delete mode 100644 src/Processors/Sinks/ExceptionHandlingSink.h diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index ccef23cc833..0820846b982 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 772fb10f914..cd667903113 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -12,7 +12,6 @@ # include # include # include -# include # include # include # include diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 769f35ab17c..70f7c0c0359 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -18,7 +18,6 @@ #include #include #include -#include #include namespace DB diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 31a156f0850..85a9ccb376e 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -16,7 +16,7 @@ #include #include #include -#include +#include #include #include #include @@ -404,7 +404,7 @@ BlockIO InterpreterInsertQuery::execute() pipeline.setSinks([&](const Block & cur_header, QueryPipelineBuilder::StreamType) -> ProcessorPtr { - return std::make_shared(cur_header); + return std::make_shared(cur_header); }); /// Don't use more threads for insert then for select to reduce memory consumption. diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index b13ac688d2a..077bd09f814 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -57,7 +57,6 @@ #include #include #include -#include #include #include diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index ee4b73c3222..08ff6d4d96c 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 1b3e790133b..4c46bed1093 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -6,9 +6,9 @@ #include #include #include -#include #include #include +#include #include #include #include @@ -300,7 +300,7 @@ QueryPipeline::QueryPipeline(Chain chain) for (auto processor : chain.getProcessors()) processors.emplace_back(std::move(processor)); - auto sink = std::make_shared(chain.getOutputPort().getHeader()); + auto sink = std::make_shared(chain.getOutputPort().getHeader()); connect(chain.getOutputPort(), sink->getPort()); processors.emplace_back(std::move(sink)); @@ -380,7 +380,7 @@ void QueryPipeline::complete(Chain chain) for (auto processor : chain.getProcessors()) processors.emplace_back(std::move(processor)); - auto sink = std::make_shared(chain.getOutputPort().getHeader()); + auto sink = std::make_shared(chain.getOutputPort().getHeader()); connect(*output, chain.getInputPort()); connect(chain.getOutputPort(), sink->getPort()); processors.emplace_back(std::move(sink)); diff --git a/src/Processors/Sinks/ExceptionHandlingSink.h b/src/Processors/Sinks/ExceptionHandlingSink.h deleted file mode 100644 index a16574dfbd8..00000000000 --- a/src/Processors/Sinks/ExceptionHandlingSink.h +++ /dev/null @@ -1,51 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -/// For now, this sink throws only last exception if any. -class ExceptionHandlingSink final : public IProcessor -{ -public: - explicit ExceptionHandlingSink(Block header) - : IProcessor({std::move(header)}, {}) - , input(inputs.front()) - { - } - - String getName() const override { return "ExceptionHandlingSink"; } - - Status prepare() override - { - while (!input.isFinished()) - { - input.setNeeded(); - if (!input.hasData()) - return Status::NeedData; - - auto data = input.pullData(); - if (data.exception) - last_exception = std::move(data.exception); - } - - if (last_exception) - return Status::Ready; - - return Status::Finished; - } - - void work() override - { - if (last_exception) - std::rethrow_exception(std::move(last_exception)); - } - - InputPort & getPort() { return input; } - -private: - InputPort & input; - std::exception_ptr last_exception; -}; - -} diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index bb1499429c5..3d97f7099e8 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -24,9 +24,10 @@ #include #include #include +#include #include #include -#include +#include #include #include #include @@ -1037,7 +1038,7 @@ namespace cur_pipeline.addTransform(std::move(sink)); cur_pipeline.setSinks([&](const Block & header, Pipe::StreamType) { - return std::make_shared(header); + return std::make_shared(header); }); auto executor = cur_pipeline.execute(); @@ -1076,12 +1077,15 @@ namespace void Call::generateOutput() { - if (!io.pipeline.pulling()) + if (!io.pipeline.initialized()) return; - auto executor = std::make_shared(io.pipeline); + Block header; + if (io.pipeline.pulling()) + header = io.pipeline.getHeader(); + write_buffer.emplace(*result.mutable_output()); - block_output_stream = query_context->getOutputStream(output_format, *write_buffer, executor->getHeader()); + block_output_stream = query_context->getOutputStream(output_format, *write_buffer, header); block_output_stream->writePrefix(); Stopwatch after_send_progress; @@ -1089,54 +1093,80 @@ namespace if (!input_function_is_used) check_query_info_contains_cancel_only = true; - auto check_for_cancel = [&] + if (io.pipeline.pulling()) { - if (isQueryCancelled()) + auto executor = std::make_shared(io.pipeline); + auto check_for_cancel = [&] { - executor->cancel(); - return false; + if (isQueryCancelled()) + { + executor->cancel(); + return false; + } + return true; + }; + + Block block; + while (check_for_cancel()) + { + if (!executor->pull(block, interactive_delay / 1000)) + break; + + throwIfFailedToSendResult(); + if (!check_for_cancel()) + break; + + if (block && !io.null_format) + block_output_stream->write(block); + + if (after_send_progress.elapsedMicroseconds() >= interactive_delay) + { + addProgressToResult(); + after_send_progress.restart(); + } + + addLogsToResult(); + + bool has_output = write_buffer->offset(); + if (has_output || result.has_progress() || result.logs_size()) + sendResult(); + + throwIfFailedToSendResult(); + if (!check_for_cancel()) + break; } - return true; - }; - Block block; - while (check_for_cancel()) - { - if (!executor->pull(block, interactive_delay / 1000)) - break; - - throwIfFailedToSendResult(); - if (!check_for_cancel()) - break; - - if (block && !io.null_format) - block_output_stream->write(block); - - if (after_send_progress.elapsedMicroseconds() >= interactive_delay) + if (!isQueryCancelled()) { + addTotalsToResult(executor->getTotalsBlock()); + addExtremesToResult(executor->getExtremesBlock()); + addProfileInfoToResult(executor->getProfileInfo()); + } + } + else + { + auto executor = std::make_shared(io.pipeline); + auto callback = [&]() -> bool + { + if (isQueryCancelled()) + return true; + + throwIfFailedToSendResult(); addProgressToResult(); - after_send_progress.restart(); - } + addLogsToResult(); - addLogsToResult(); + bool has_output = write_buffer->offset(); + if (has_output || result.has_progress() || result.logs_size()) + sendResult(); - bool has_output = write_buffer->offset(); - if (has_output || result.has_progress() || result.logs_size()) - sendResult(); - - throwIfFailedToSendResult(); - if (!check_for_cancel()) - break; + throwIfFailedToSendResult(); + return false; + }; + executor->setCancelCallback(std::move(callback), interactive_delay / 1000); + executor->execute(); } block_output_stream->writeSuffix(); - - if (!isQueryCancelled()) - { - addTotalsToResult(executor->getTotalsBlock()); - addExtremesToResult(executor->getExtremesBlock()); - addProfileInfoToResult(executor->getProfileInfo()); - } } void Call::finishQuery() diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index dd8b10ff1fd..335765405dd 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -16,7 +16,7 @@ limitations under the License. */ #include #include #include -#include +#include #include #include #include @@ -267,7 +267,7 @@ void StorageLiveView::writeIntoLiveView( pipeline.addChain(Chain(std::move(output))); pipeline.setSinks([&](const Block & cur_header, Pipe::StreamType) { - return std::make_shared(cur_header); + return std::make_shared(cur_header); }); auto executor = pipeline.execute(); diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index a5314096606..fdc30919ee7 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -9,7 +9,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 53dc7d01728..7fda31b33bd 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,7 +1,6 @@ #include "PostgreSQLReplicationHandler.h" #include -#include #include #include #include From 3c9e6a04bf3f31ce5544015955dc601d9933d159 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 26 Sep 2021 18:06:32 +0300 Subject: [PATCH 127/142] A little bit better --- tests/integration/helpers/cluster.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 42e406036e9..b4e017b6228 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2427,6 +2427,8 @@ class ClickHouseInstance: if self.stay_alive: entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND.replace("{main_config_file}", self.main_config_name) + else: + entrypoint_cmd = '[' + ', '.join(map(lambda x: '"' + x + '"', entrypoint_cmd.split())) + ']' logging.debug("Entrypoint cmd: {}".format(entrypoint_cmd)) From 13db65f47c30dade3f3c2ba898c3d90b960f78db Mon Sep 17 00:00:00 2001 From: Kevin Michel Date: Sun, 26 Sep 2021 17:15:11 +0200 Subject: [PATCH 128/142] Fix ReplicatedAccessStorage shutdown without startup If ReplicatedAccessStorage startup was not executing or if it failed before completing (for instance when ZooKeeper was not configured), its destructor would call shutdown and try to join a missing thread. --- src/Access/ReplicatedAccessStorage.cpp | 9 ++-- .../tests/gtest_replicated_access_storage.cpp | 46 +++++++++++++++++++ 2 files changed, 52 insertions(+), 3 deletions(-) create mode 100644 src/Access/tests/gtest_replicated_access_storage.cpp diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index f91b7c8fb06..7c71e264cd0 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -63,9 +63,12 @@ void ReplicatedAccessStorage::shutdown() bool prev_stop_flag = stop_flag.exchange(true); if (!prev_stop_flag) { - /// Notify the worker thread to stop waiting for new queue items - refresh_queue.push(UUIDHelpers::Nil); - worker_thread.join(); + if (worker_thread.joinable()) + { + /// Notify the worker thread to stop waiting for new queue items + refresh_queue.push(UUIDHelpers::Nil); + worker_thread.join(); + } } } diff --git a/src/Access/tests/gtest_replicated_access_storage.cpp b/src/Access/tests/gtest_replicated_access_storage.cpp new file mode 100644 index 00000000000..f2052e91749 --- /dev/null +++ b/src/Access/tests/gtest_replicated_access_storage.cpp @@ -0,0 +1,46 @@ +#include +#include + +using namespace DB; + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NO_ZOOKEEPER; +} +} + + +TEST(ReplicatedAccessStorage, ShutdownWithoutStartup) +{ + auto get_zk = []() + { + return std::shared_ptr(); + }; + + auto storage = ReplicatedAccessStorage("replicated", "/clickhouse/access", get_zk); + storage.shutdown(); +} + + +TEST(ReplicatedAccessStorage, ShutdownWithFailedStartup) +{ + auto get_zk = []() + { + return std::shared_ptr(); + }; + + auto storage = ReplicatedAccessStorage("replicated", "/clickhouse/access", get_zk); + try + { + storage.startup(); + } + catch (Exception & e) + { + if (e.code() != ErrorCodes::NO_ZOOKEEPER) + throw; + } + storage.shutdown(); +} + From e06115d77f2020315c2252e717edc46da5335d97 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sun, 26 Sep 2021 17:24:05 -0300 Subject: [PATCH 129/142] test for #23634 --- ..._nullable_key_and_index_negate_cond.reference | 3 +++ .../01410_nullable_key_and_index_negate_cond.sql | 16 ++++++++++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/01410_nullable_key_and_index_negate_cond.reference create mode 100644 tests/queries/0_stateless/01410_nullable_key_and_index_negate_cond.sql diff --git a/tests/queries/0_stateless/01410_nullable_key_and_index_negate_cond.reference b/tests/queries/0_stateless/01410_nullable_key_and_index_negate_cond.reference new file mode 100644 index 00000000000..f735f1378b6 --- /dev/null +++ b/tests/queries/0_stateless/01410_nullable_key_and_index_negate_cond.reference @@ -0,0 +1,3 @@ +s s s +s s s +s s s diff --git a/tests/queries/0_stateless/01410_nullable_key_and_index_negate_cond.sql b/tests/queries/0_stateless/01410_nullable_key_and_index_negate_cond.sql new file mode 100644 index 00000000000..3d1e3177173 --- /dev/null +++ b/tests/queries/0_stateless/01410_nullable_key_and_index_negate_cond.sql @@ -0,0 +1,16 @@ +drop table if exists test_23634; + +set force_primary_key=1; + +CREATE TABLE test_23634 (id Nullable(String), s Nullable(String), s1 Nullable(String)) +ENGINE = MergeTree() ORDER BY (id,s) SETTINGS allow_nullable_key = 1; + +INSERT into test_23634 values ('s','s','s'), (null,'s1','s1'), (null,null,'s2'), (null,null,null); + +select * from test_23634 where id !=''; + +select * from test_23634 where id !='' and s != ''; + +select * from test_23634 where id !='' and s != '' and s1 != ''; + +drop table test_23634; From 42af45b2bae6cc3a9f107b635fc406a3ea35f094 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 26 Sep 2021 23:16:16 +0300 Subject: [PATCH 130/142] rocksdb: fix race condition during multiple DB opening This should fix the following SIGSEGV, that was found on CI [1]: BaseDaemon: Address: NULL pointer. Access: read. Unknown si_code. BaseDaemon: 4.4. inlined from ../contrib/rocksdb/utilities/object_registry.cc:19: rocksdb::ObjectLibrary::FindEntry() const ... BaseDaemon: 7.3. inlined from ../contrib/rocksdb/options/cf_options.cc:678: rocksdb::$_7::operator()() [1]: https://clickhouse-test-reports.s3.yandex.net/29341/2b2bec3679df7965af908ce3f1e8e17e39bd12fe/integration_tests_flaky_check_(asan).html#fail1 And also I checked manually with TSan binary, and here is a data race reported by TSan: WARNING: ThreadSanitizer: data race (pid=3356) Read of size 8 at 0x7b0c0008cca8 by thread T40: 2 rocksdb::ObjectLibrary::FindEntry() const obj-x86_64-linux-gnu/../contrib/rocksdb/utilities/object_registry.cc:18:27 (clickhouse-tsan+0x1b839a6c) ... 6 rocksdb::$_7::operator()() const obj-x86_64-linux-gnu/../contrib/rocksdb/options/cf_options.cc:676:32 (clickhouse-tsan+0x1b6bfa63) ... 28 rocksdb::GetColumnFamilyOptionsFromMap() obj-x86_64-linux-gnu/../contrib/rocksdb/options/options_helper.cc:727:10 (clickhouse-tsan+0x1b6fffd2) 29 DB::StorageEmbeddedRocksDB::initDb() obj-x86_64-linux-gnu/../src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp:359:26 (clickhouse-tsan+0x14195e31) ... Previous write of size 8 at 0x7b0c0008cca8 by thread T41: ... 9 rocksdb::ObjectLibrary::AddEntry() obj-x86_64-linux-gnu/../contrib/rocksdb/utilities/object_registry.cc:31:19 (clickhouse-tsan+0x1b8392fc) ... 11 rocksdb::RegisterTableFactories()::$_0::operator()() const obj-x86_64-linux-gnu/../contrib/rocksdb/table/table_factory.cc:23:14 (clickhouse-tsan+0x1b7ea94c) ... 43 rocksdb::GetColumnFamilyOptionsFromMap() obj-x86_64-linux-gnu/../contrib/rocksdb/options/options_helper.cc:727:10 (clickhouse-tsan+0x1b6fffd2) 44 DB::StorageEmbeddedRocksDB::initDb() obj-x86_64-linux-gnu/../src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp:359:26 (clickhouse-tsan+0x14195e31) Refs: https://github.com/ClickHouse-Extras/rocksdb/pull/13 Fixes: #29341 --- contrib/rocksdb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/rocksdb b/contrib/rocksdb index 5ea892c8673..296c1b8b95f 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 5ea892c8673e6c5a052887653673b967d44cc59b +Subproject commit 296c1b8b95fd448b8097a1b2cc9f704ff4a73a2c From b3d1bfc67a41c5bd5ecb0a5faa791abd33c1d5fc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 21 Sep 2021 21:07:52 +0300 Subject: [PATCH 131/142] Send UNKNOWN_DATABASE to the client (via TCP) Before this patch: $ clickhouse-client --database foo -q 'select 1' Code: 32. DB::Exception: Attempt to read after eof: while receiving packet from localhost:9000. (ATTEMPT_TO_READ_AFTER_EOF) After: $ clickhouse-client --database foo -q 'select 1' Received exception from server (version 21.11.1): Code: 81. DB::Exception: Received from localhost:9000. DB::Exception: Database foo doesn't exist. (UNKNOWN_DATABASE) (query: select 1) Fixes: #26864 (cc @vitlibar) --- src/Server/TCPHandler.cpp | 29 +++++++++---------- .../02030_client_unknown_database.reference | 0 .../02030_client_unknown_database.sh | 9 ++++++ 3 files changed, 23 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/02030_client_unknown_database.reference create mode 100755 tests/queries/0_stateless/02030_client_unknown_database.sh diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index eb756a47156..e2e41c1fb6f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -115,6 +115,20 @@ void TCPHandler::runImpl() try { receiveHello(); + sendHello(); + + if (!is_interserver_mode) /// In interserver mode queries are executed without a session context. + { + session->makeSessionContext(); + + /// If session created, then settings in session context has been updated. + /// So it's better to update the connection settings for flexibility. + extractConnectionSettingsFromContext(session->sessionContext()); + + /// When connecting, the default database could be specified. + if (!default_database.empty()) + session->sessionContext()->setCurrentDatabase(default_database); + } } catch (const Exception & e) /// Typical for an incorrect username, password, or address. { @@ -140,21 +154,6 @@ void TCPHandler::runImpl() throw; } - sendHello(); - - if (!is_interserver_mode) /// In interserver mode queries are executed without a session context. - { - session->makeSessionContext(); - - /// If session created, then settings in session context has been updated. - /// So it's better to update the connection settings for flexibility. - extractConnectionSettingsFromContext(session->sessionContext()); - - /// When connecting, the default database could be specified. - if (!default_database.empty()) - session->sessionContext()->setCurrentDatabase(default_database); - } - while (true) { /// We are waiting for a packet from the client. Thus, every `poll_interval` seconds check whether we need to shut down. diff --git a/tests/queries/0_stateless/02030_client_unknown_database.reference b/tests/queries/0_stateless/02030_client_unknown_database.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02030_client_unknown_database.sh b/tests/queries/0_stateless/02030_client_unknown_database.sh new file mode 100755 index 00000000000..28bd4895a23 --- /dev/null +++ b/tests/queries/0_stateless/02030_client_unknown_database.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash + +CLICKHOUSE_DATABASE=no_such_database_could_exist + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "SELECT 1" |& grep -q UNKNOWN_DATABASE From 8493b014cd7ea7472c396b37f6a93df7191c402f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 27 Sep 2021 02:16:58 +0300 Subject: [PATCH 132/142] Improve test --- tests/queries/0_stateless/2024_merge_regexp_assert.sql | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/2024_merge_regexp_assert.sql b/tests/queries/0_stateless/2024_merge_regexp_assert.sql index 3a3c2920f0b..8ea4a77bbd8 100644 --- a/tests/queries/0_stateless/2024_merge_regexp_assert.sql +++ b/tests/queries/0_stateless/2024_merge_regexp_assert.sql @@ -1 +1,7 @@ -SELECT a FROM merge(REGEXP('.'), 'query_log'); -- { serverError 47 } +DROP TABLE IF EXISTS t; +CREATE TABLE t (b UInt8) ENGINE = Memory; +SELECT a FROM merge(REGEXP('.'), '^t$'); -- { serverError 47 } +SELECT a FROM merge(REGEXP('\0'), '^t$'); -- { serverError 47 } +SELECT a FROM merge(REGEXP('\0a'), '^t$'); -- { serverError 47 } +SELECT a FROM merge(REGEXP('\0a'), '^$'); -- { serverError 36 } +DROP TABLE t; From 544cccb2fbb58032adc691fbfc45817456154864 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Mon, 27 Sep 2021 09:43:13 +0300 Subject: [PATCH 133/142] Forward-declared LOGICAL_ERROR to make style check happy --- src/Parsers/IAST.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index bdbf3b2b448..9a8838956f9 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -14,6 +14,7 @@ namespace ErrorCodes extern const int TOO_DEEP_AST; extern const int BAD_ARGUMENTS; extern const int UNKNOWN_ELEMENT_IN_AST; + extern const int LOGICAL_ERROR; } From 09232788c090c56fc488a5960e938b6354747e61 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 27 Sep 2021 10:55:33 +0300 Subject: [PATCH 134/142] Fix GRPC tests. --- src/Server/GRPCServer.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 3d97f7099e8..0fb9d82aca6 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1077,7 +1077,7 @@ namespace void Call::generateOutput() { - if (!io.pipeline.initialized()) + if (!io.pipeline.initialized() || io.pipeline.pushing()) return; Block header; @@ -1148,8 +1148,6 @@ namespace auto executor = std::make_shared(io.pipeline); auto callback = [&]() -> bool { - if (isQueryCancelled()) - return true; throwIfFailedToSendResult(); addProgressToResult(); @@ -1160,7 +1158,8 @@ namespace sendResult(); throwIfFailedToSendResult(); - return false; + + return isQueryCancelled(); }; executor->setCancelCallback(std::move(callback), interactive_delay / 1000); executor->execute(); From 1c9778603ff49563d1d3d0d357de0608167e504d Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 27 Sep 2021 11:43:00 +0300 Subject: [PATCH 135/142] Update clickhouse-test --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 470b88de574..7ac434e03cd 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -609,7 +609,7 @@ class TestCase: if skip_reason is not None: return TestResult(self.name, TestStatus.SKIPPED, skip_reason, 0., "") - if args.testname and self.send_test_name_failed(suite, self.case): + if args.testname and self.send_test_name_failed(suite.suite, self.case): description = "\nServer does not respond to health check\n" return TestResult(self.name, TestStatus.FAIL, FailureReason.SERVER_DIED, 0., description) From d0f53cfc317662b05214ecd0234842c21a071fd4 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 27 Sep 2021 12:17:00 +0300 Subject: [PATCH 136/142] Added setting use_skip_indexes --- docs/en/operations/settings/settings.md | 15 +++++++++++++-- src/Core/Settings.h | 1 + src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 3 files changed, 15 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 17317a13908..3a3eac0236a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -93,6 +93,17 @@ Works with tables in the MergeTree family. If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition reduces the amount of data to read. For more information about data ranges in MergeTree tables, see [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). +## use_skip_indexes {#settings-force_data_skipping_indices} + +Use data skipping indexes during query execution. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: 1. + ## force_data_skipping_indices {#settings-force_data_skipping_indices} Disables query execution if passed data skipping indices wasn't used. @@ -3630,7 +3641,7 @@ Default value: `enable`. ## max_hyperscan_regexp_length {#max-hyperscan-regexp-length} -Defines the maximum length for each regular expression in the [hyperscan multi-match functions](../../sql-reference/functions/string-search-functions.md#multimatchanyhaystack-pattern1-pattern2-patternn). +Defines the maximum length for each regular expression in the [hyperscan multi-match functions](../../sql-reference/functions/string-search-functions.md#multimatchanyhaystack-pattern1-pattern2-patternn). Possible values: @@ -3673,7 +3684,7 @@ Exception: Regexp length too large. ## max_hyperscan_regexp_total_length {#max-hyperscan-regexp-total-length} -Sets the maximum length total of all regular expressions in each [hyperscan multi-match function](../../sql-reference/functions/string-search-functions.md#multimatchanyhaystack-pattern1-pattern2-patternn). +Sets the maximum length total of all regular expressions in each [hyperscan multi-match function](../../sql-reference/functions/string-search-functions.md#multimatchanyhaystack-pattern1-pattern2-patternn). Possible values: diff --git a/src/Core/Settings.h b/src/Core/Settings.h index a728ba636ad..9dfa4d911b5 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -161,6 +161,7 @@ class IColumn; \ M(Bool, force_index_by_date, false, "Throw an exception if there is a partition key in a table, and it is not used.", 0) \ M(Bool, force_primary_key, false, "Throw an exception if there is primary key in a table, and it is not used.", 0) \ + M(Bool, use_skip_indexes, true, "Use data skipping indexes during query execution.", 0) \ M(String, force_data_skipping_indices, "", "Comma separated list of strings or literals with the name of the data skipping indices that should be used during query execution, otherwise an exception will be thrown.", 0) \ \ M(Float, max_streams_to_max_threads_ratio, 1, "Allows you to use more sources than the number of threads - to more evenly distribute work across threads. It is assumed that this is a temporary solution, since it will be possible in the future to make the number of sources equal to the number of threads, but for each source to dynamically select available work for itself.", 0) \ diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 86d4a7c93b1..b5af4a2195e 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -875,7 +875,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( log, num_streams, result.index_stats, - true /* use_skip_indexes */); + context->getSettings().use_skip_indexes); } catch (...) { From 8ce21a70bba0d3fc3958d16d054fc7980c404c04 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 27 Sep 2021 16:06:35 +0300 Subject: [PATCH 137/142] Fixed docs --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 3a3eac0236a..0e4e9d3b489 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -93,7 +93,7 @@ Works with tables in the MergeTree family. If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition reduces the amount of data to read. For more information about data ranges in MergeTree tables, see [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md). -## use_skip_indexes {#settings-force_data_skipping_indices} +## use_skip_indexes {#settings-use_skip_indexes} Use data skipping indexes during query execution. From cd9c796e6fe1c8aa1a7760fda337a343b80123dd Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 27 Sep 2021 17:23:56 +0300 Subject: [PATCH 138/142] Map bloom filter index mapValues equals function support --- .../MergeTreeIndexConditionBloomFilter.cpp | 45 ++++++++++++------- .../2021_map_bloom_filter_index.reference | 12 +++++ .../2021_map_bloom_filter_index.sql | 12 +++++ 3 files changed, 54 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index cb617b0ef22..a11eaa9a4be 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -511,6 +511,8 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( RPNElement & out, const ASTPtr & parent) { + std::cerr << "MergeTreeIndexConditionBloomFilter::traverseASTEquals " << function_name << " ast " << key_ast->formatForErrorMessage() << std::endl; + if (header.has(key_ast->getColumnName())) { size_t position = header.getPositionByName(key_ast->getColumnName()); @@ -622,7 +624,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( return match_with_subtype; } - if (function->name == "arrayElement") + if (function->name == "arrayElement" && (function_name == "equals" || function_name == "notEquals")) { /** Try to parse arrayElement for mapKeys index. * It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map @@ -637,25 +639,38 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( const auto & col_name = assert_cast(function->arguments.get()->children[0].get())->name(); auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name); + auto map_values_index_column_name = fmt::format("mapValues({})", col_name); - if (!header.has(map_keys_index_column_name)) - return false; + size_t position = 0; + Field const_value = value_field; - size_t position = header.getPositionByName(map_keys_index_column_name); - const DataTypePtr & index_type = header.getByPosition(position).type; - out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS; - - auto & argument = function->arguments.get()->children[1]; - - if (const auto * literal = argument->as()) + if (header.has(map_keys_index_column_name)) { - auto element_key = literal->value; - const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type); - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), element_key))); - return true; + position = header.getPositionByName(map_keys_index_column_name); + + auto & argument = function->arguments.get()->children[1]; + + if (const auto * literal = argument->as()) + const_value = literal->value; + else + return false; + } + else if (header.has(map_values_index_column_name)) + { + position = header.getPositionByName(map_values_index_column_name); + } + else + { + return false; } - return false; + out.function = function_name == "equals" ? RPNElement::FUNCTION_EQUALS : RPNElement::FUNCTION_NOT_EQUALS; + + const auto & index_type = header.getByPosition(position).type; + const auto actual_type = BloomFilter::getPrimitiveType(index_type); + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), const_value))); + + return true; } } diff --git a/tests/queries/0_stateless/2021_map_bloom_filter_index.reference b/tests/queries/0_stateless/2021_map_bloom_filter_index.reference index 4f0a04073ae..ccfd5f93695 100644 --- a/tests/queries/0_stateless/2021_map_bloom_filter_index.reference +++ b/tests/queries/0_stateless/2021_map_bloom_filter_index.reference @@ -32,6 +32,18 @@ Has with existing key Has with non existing key Has with non existing key and default value Map bloom filter mapValues +Equals with existing key +0 {'K0':'V0'} +Equals with non existing key +Equals with non existing key and default value +0 {'K0':'V0'} +1 {'K1':'V1'} +Not equals with existing key +1 {'K1':'V1'} +Not equals with non existing key +0 {'K0':'V0'} +1 {'K1':'V1'} +Not equals with non existing key and default value IN with existing key 0 {'K0':'V0'} IN with non existing key diff --git a/tests/queries/0_stateless/2021_map_bloom_filter_index.sql b/tests/queries/0_stateless/2021_map_bloom_filter_index.sql index 6e0c4f4a360..c7e522db631 100644 --- a/tests/queries/0_stateless/2021_map_bloom_filter_index.sql +++ b/tests/queries/0_stateless/2021_map_bloom_filter_index.sql @@ -64,6 +64,18 @@ INSERT INTO map_test_index_map_values VALUES (0, {'K0':'V0'}), (1, {'K1':'V1'}); SELECT 'Map bloom filter mapValues'; +SELECT 'Equals with existing key'; +SELECT * FROM map_test_index_map_values WHERE map['K0'] = 'V0'; +SELECT 'Equals with non existing key'; +SELECT * FROM map_test_index_map_values WHERE map['K2'] = 'V2'; +SELECT 'Equals with non existing key and default value'; +SELECT * FROM map_test_index_map_values WHERE map['K3'] = ''; +SELECT 'Not equals with existing key'; +SELECT * FROM map_test_index_map_values WHERE map['K0'] != 'V0'; +SELECT 'Not equals with non existing key'; +SELECT * FROM map_test_index_map_values WHERE map['K2'] != 'V2'; +SELECT 'Not equals with non existing key and default value'; +SELECT * FROM map_test_index_map_values WHERE map['K3'] != ''; SELECT 'IN with existing key'; SELECT * FROM map_test_index_map_values WHERE map['K0'] IN 'V0'; SELECT 'IN with non existing key'; From bd499e5b7dc1c8ff4fbcfff8e957281ce0a1be42 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 27 Sep 2021 17:44:34 +0300 Subject: [PATCH 139/142] Dictionary Array nested Map added test --- ...2025_dictionary_array_nested_map.reference | 1 + .../2025_dictionary_array_nested_map.sql | 25 +++++++++++++++++++ 2 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/2025_dictionary_array_nested_map.reference create mode 100644 tests/queries/0_stateless/2025_dictionary_array_nested_map.sql diff --git a/tests/queries/0_stateless/2025_dictionary_array_nested_map.reference b/tests/queries/0_stateless/2025_dictionary_array_nested_map.reference new file mode 100644 index 00000000000..5e1ee2e8faa --- /dev/null +++ b/tests/queries/0_stateless/2025_dictionary_array_nested_map.reference @@ -0,0 +1 @@ +[{'l':0,'h':10000,'t':0.1},{'l':10001,'h':100000000000000,'t':0.2}] diff --git a/tests/queries/0_stateless/2025_dictionary_array_nested_map.sql b/tests/queries/0_stateless/2025_dictionary_array_nested_map.sql new file mode 100644 index 00000000000..1a0dd05c83d --- /dev/null +++ b/tests/queries/0_stateless/2025_dictionary_array_nested_map.sql @@ -0,0 +1,25 @@ +CREATE TABLE dict_nested_map_test_table +( + test_id UInt32, + type String, + test_config Array(Map(String, Decimal(28,12))), + ncp UInt8 +) +ENGINE=MergeTree() +ORDER BY test_id; + +INSERT INTO dict_nested_map_test_table VALUES (3, 't', [{'l': 0.0, 'h': 10000.0, 't': 0.1}, {'l': 10001.0, 'h': 100000000000000.0, 't': 0.2}], 0); + +CREATE DICTIONARY dict_nested_map_dictionary +( + test_id UInt32, + type String, + test_config Array(Map(String, Decimal(28,12))), + ncp UInt8 +) +PRIMARY KEY test_id +SOURCE(CLICKHOUSE(TABLE 'dict_nested_map_test_table')) +LAYOUT(HASHED(PREALLOCATE 1)) +LIFETIME(MIN 1 MAX 1000000); + +SELECT dictGet('dict_nested_map_dictionary', 'test_config', toUInt64(3)); From 2b8931523f496679f9813e6c0dabf1c63005c421 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 27 Sep 2021 19:19:38 +0300 Subject: [PATCH 140/142] Dictionary view different database added test --- ...025_dictionary_view_different_db.reference | 1 + .../2025_dictionary_view_different_db.sql | 43 +++++++++++++++++++ 2 files changed, 44 insertions(+) create mode 100644 tests/queries/0_stateless/2025_dictionary_view_different_db.reference create mode 100644 tests/queries/0_stateless/2025_dictionary_view_different_db.sql diff --git a/tests/queries/0_stateless/2025_dictionary_view_different_db.reference b/tests/queries/0_stateless/2025_dictionary_view_different_db.reference new file mode 100644 index 00000000000..1aa2c68151a --- /dev/null +++ b/tests/queries/0_stateless/2025_dictionary_view_different_db.reference @@ -0,0 +1 @@ +0 ViewValue Value diff --git a/tests/queries/0_stateless/2025_dictionary_view_different_db.sql b/tests/queries/0_stateless/2025_dictionary_view_different_db.sql new file mode 100644 index 00000000000..0019d278392 --- /dev/null +++ b/tests/queries/0_stateless/2025_dictionary_view_different_db.sql @@ -0,0 +1,43 @@ +-- Tags: no-parallel + +DROP DATABASE IF EXISTS 2025_test_db; +CREATE DATABASE 2025_test_db; + +DROP TABLE IF EXISTS 2025_test_db.test_table; +CREATE TABLE 2025_test_db.test_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO 2025_test_db.test_table VALUES (0, 'Value'); + +CREATE DICTIONARY 2025_test_db.test_dictionary +( + id UInt64, + value String +) +PRIMARY KEY id +LAYOUT(DIRECT()) +SOURCE(CLICKHOUSE(TABLE 'test_table' DB '2025_test_db')); + +DROP TABLE IF EXISTS 2025_test_db.view_table; +CREATE TABLE 2025_test_db.view_table +( + id UInt64, + value String +) ENGINE=TinyLog; + +INSERT INTO 2025_test_db.view_table VALUES (0, 'ViewValue'); + +DROP VIEW IF EXISTS test_view_different_db; +CREATE VIEW test_view_different_db AS SELECT id, value, dictGet('2025_test_db.test_dictionary', 'value', id) FROM 2025_test_db.view_table; +SELECT * FROM test_view_different_db; + +DROP TABLE 2025_test_db.test_table; +DROP DICTIONARY 2025_test_db.test_dictionary; +DROP TABLE 2025_test_db.view_table; + +DROP VIEW test_view_different_db; + +DROP DATABASE 2025_test_db; \ No newline at end of file From 17f1a3910bdb0d6a7d20857d317e56abe87068b9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 27 Sep 2021 19:20:58 +0300 Subject: [PATCH 141/142] Fixed test --- tests/queries/0_stateless/2025_dictionary_view_different_db.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/2025_dictionary_view_different_db.sql b/tests/queries/0_stateless/2025_dictionary_view_different_db.sql index 0019d278392..b06285b683f 100644 --- a/tests/queries/0_stateless/2025_dictionary_view_different_db.sql +++ b/tests/queries/0_stateless/2025_dictionary_view_different_db.sql @@ -40,4 +40,4 @@ DROP TABLE 2025_test_db.view_table; DROP VIEW test_view_different_db; -DROP DATABASE 2025_test_db; \ No newline at end of file +DROP DATABASE 2025_test_db; From 23d899fb604430f92accf7d55ecc9ccd0f4b0f5e Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 27 Sep 2021 19:54:59 +0300 Subject: [PATCH 142/142] Update 01158_zookeeper_log_long.sql --- tests/queries/0_stateless/01158_zookeeper_log_long.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.sql b/tests/queries/0_stateless/01158_zookeeper_log_long.sql index f067ce15079..61a36df68d8 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.sql +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.sql @@ -4,7 +4,7 @@ drop table if exists rmt; -- cleanup code will perform extra Exists -- (so the .reference will not match) -create table rmt (n int) engine=ReplicatedMergeTree('/test/01158/{database}/rmt', '1') order by n settings cleanup_delay_period=86400; +create table rmt (n int) engine=ReplicatedMergeTree('/test/01158/{database}/rmt', '1') order by n settings cleanup_delay_period=86400, replicated_can_become_leader=0; system sync replica rmt; insert into rmt values (1); insert into rmt values (1);