From 6b1030c9b879154e65b3fc456466ea1e36ce6aeb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 13 Aug 2021 19:32:29 +0300 Subject: [PATCH 001/317] 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/317] 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/317] 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/317] 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 33b95bfd7e81b95091a6f33d89eb27f2c38d2b02 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 30 Aug 2021 16:06:47 +0300 Subject: [PATCH 005/317] wip --- docs/en/development/contrib.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index 9daf6148324..e94a9f42b0d 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -98,7 +98,9 @@ SELECT library_name, license_type, license_path FROM system.licenses ORDER BY li 3. Branch from the branch you want to integrate, e.g., `master` -> `clickhouse/master`, or `release/vX.Y.Z` -> `clickhouse/release/vX.Y.Z`. 4. All forks in [Clickhouse-extras](https://github.com/ClickHouse-Extras) can be automatically synchronized with upstreams. `clickhouse/...` branches will remain unaffected, since virtually nobody is going to use that naming pattern in their upstream repos. 5. Add submodules under `contrib` of ClickHouse repo that refer the above forks/mirrors. Set the submodules to track the corresponding `clickhouse/...` branches. -6. Every time the custom changes have to be made in the library code, a dedicated branch should be created, like `clickhouse/my-fix`. Then this branch should be merged into the branch, that is tracked by the submodule, e.g., `clickhouse/master` or `clickhouse/release/vX.Y.Z`. +6. Main branch(develop/master) is meant to be synced with upstream. If a patch has to be commited to a fork it should remain on top of main branch. Main branch update process is done via git rebase and allow a) to remove patches that were pushed to upstream and b) to maintain simple list of changes over upstream version. +7. Every time the custom changes have to be made in the library code, a dedicated branch should be created, like `clickhouse/my-fix`. Then this branch should be merged into the main branch in a fork when it is ready. +8. ClickHouse main repository shouls reference only commit in `clickhouse/`branches 7. No code should be pushed in any branch of the forks in [Clickhouse-extras](https://github.com/ClickHouse-Extras), whose names do not follow `clickhouse/...` pattern. 8. Always write the custom changes with the official repo in mind. Once the PR is merged from (a feature/fix branch in) your personal fork into the fork in [Clickhouse-extras](https://github.com/ClickHouse-Extras), and the submodule is bumped in ClickHouse repo, consider opening another PR from (a feature/fix branch in) the fork in [Clickhouse-extras](https://github.com/ClickHouse-Extras) to the official repo of the library. This will make sure, that 1) the contribution has more than a single use case and importance, 2) others will also benefit from it, 3) the change will not remain a maintenance burden solely on ClickHouse developers. 9. When a submodule needs to start using a newer code from the original branch (e.g., `master`), and since the custom changes might be merged in the branch it is tracking (e.g., `clickhouse/master`) and so it may diverge from its original counterpart (i.e., `master`), a careful merge should be carried out first, i.e., `master` -> `clickhouse/master`, and only then the submodule can be bumped in ClickHouse. From 5be05c2ef995395167c86ed5ae1ea831c96a21c8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 31 Aug 2021 16:50:07 +0300 Subject: [PATCH 006/317] 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 007/317] 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 008/317] 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 1ea2dd45642437119e9c5418a472ebf380ab8643 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 2 Sep 2021 15:43:15 +0300 Subject: [PATCH 009/317] Improve work with iptables and services listing. --- tests/integration/helpers/cluster.py | 3 ++- tests/integration/helpers/network.py | 1 + tests/integration/runner | 1 + 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6dcff59a252..06423859132 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1049,7 +1049,7 @@ class ClickHouseCluster: errors += [str(ex)] time.sleep(interval) - run_and_check(['docker-compose', 'ps', '--services', '--all']) + run_and_check(['docker', 'ps', '--all']) logging.error("Can't connect to URL:{}".format(errors)) raise Exception("Cannot wait URL {}(interval={}, timeout={}, attempts={})".format( url, interval, timeout, attempts)) @@ -1757,6 +1757,7 @@ services: - {db_dir}:/var/lib/clickhouse/ - {logs_dir}:/var/log/clickhouse-server/ - /etc/passwd:/etc/passwd:ro + - /run/xtables.lock:/run/xtables.lock:ro {binary_volume} {odbc_bridge_volume} {library_bridge_volume} diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 7d9906ae663..5d4ef722803 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -212,6 +212,7 @@ class _NetworkManager: self._container = self._docker_client.containers.run('yandex/clickhouse-integration-helper', auto_remove=True, command=('sleep %s' % self.container_exit_timeout), + volumes={'/run/xtables.lock': {'bind': '/run/xtables.lock', 'mode': 'rw' }}, detach=True, network_mode='host') container_id = self._container.id self._container_expire_time = time.time() + self.container_expire_timeout diff --git a/tests/integration/runner b/tests/integration/runner index 2143d7ebf29..66f3e31e948 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -276,6 +276,7 @@ if __name__ == "__main__": --volume={library_bridge_bin}:/clickhouse-library-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ + --volume=/run/xtables.lock:/run/xtables.lock \ {dockerd_internal_volume} -e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 \ {env_tags} {env_cleanup} -e PYTEST_OPTS='{parallel} {opts} {tests_list} -vvv' {img} {command}".format( net=net, From 66a76ab70f32d3ef1ee99e8f2d74ced0a1b2f402 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 3 Sep 2021 20:29:36 +0300 Subject: [PATCH 010/317] 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 011/317] 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 012/317] 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 013/317] 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 014/317] 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 21ba2c6596a12a764d69922bc49611225700d867 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 4 Sep 2021 15:37:08 -0700 Subject: [PATCH 015/317] Docs - S2 functions --- docs/en/sql-reference/functions/geo/s2.md | 120 ++++++++++++++++++++++ 1 file changed, 120 insertions(+) create mode 100644 docs/en/sql-reference/functions/geo/s2.md diff --git a/docs/en/sql-reference/functions/geo/s2.md b/docs/en/sql-reference/functions/geo/s2.md new file mode 100644 index 00000000000..ebe4cd66d8a --- /dev/null +++ b/docs/en/sql-reference/functions/geo/s2.md @@ -0,0 +1,120 @@ +--- +toc_title: S2 Geometry +--- + +# Functions for Working with S2 Index {#s2Index} + +[S2](https://s2geometry.io/) is a geographical indexing system where all geographical data is represented on a three-dimensional sphere (similar to a globe). + +## geoToS2 {#geoToS2} + +Returns [S2](#s2index) point index corresponding to the provided coordinates `(longitude, latitude)`. + +**Syntax** + +``` sql +geoToS2(lon, lat) +``` + +**Arguments** + +- `lon` — Longitude. [Float64](../../../sql-reference/data-types/float.md). +- `lat` — Latitude. [Float64](../../../sql-reference/data-types/float.md). + +**Returned values** + +- S2 point index. + +Type: [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +``` sql +SELECT geoToS2(37.79506683, 55.71290588) as s2Index; +``` + +Result: + +``` text +┌─────────────s2Index─┐ +│ 4704772434919038107 │ +└─────────────────────┘ +``` + +## s2ToGeo {#s2ToGeo} + +Returns geo coordinates `(longitude, latitude)` corresponding to the provided [S2](#s2index) point index. + +**Syntax** + +``` sql +s2ToGeo(s2index) +``` + +**Arguments** + +- `s2Index` — S2 Index. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Returned values** + +- A tuple consisting of two values: `tuple(lon,lat)`. + +Type: `lon` - [Float64](../../../sql-reference/data-types/float.md). `lat` — [Float64](../../../sql-reference/data-types/float.md). + + +**Example** + +Query: + +``` sql +SELECT s2ToGeo(4704772434919038107) as s2Coodrinates; +``` + +Result: + +``` text +┌─s2Coodrinates────────────────────────┐ +│ (37.79506681471008,55.7129059052841) │ +└──────────────────────────────────────┘ +``` + + + +## s2GetNeighbors {#s2GetNeighbors} + +Returns S2 neighbor indices corresponding to the provided [S2](#s2index)). Each cell in the S2 system is a quadrilateral bounded by four geodesics. So, each cell has 4 neighbors. + +**Syntax** + +``` sql +s2ToGeo(s2index) +``` + +**Arguments** + +- `s2Index` — S2 Index. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Returned values** + +- An array consisting of the 4 neighbor indices: `array[s2index1, s2index3, s2index2, s2index4]`. + +Type: Each S2 index is [UInt64](../../../sql-reference/data-types/int-uint.md). + + +**Example** + +Query: + +``` sql + select s2GetNeighbors(5074766849661468672) AS s2Neighbors; +``` + +Result: + +``` text +┌─s2Neighbors───────────────────────────────────────────────────────────────────────┐ +│ [5074766987100422144,5074766712222515200,5074767536856236032,5074767261978329088] │ +└───────────────────────────────────────────────────────────────────────────────────┘ +``` From 5bbf761a684dbf059ac2bd81e56d5e2a777252e8 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 4 Sep 2021 15:38:04 -0700 Subject: [PATCH 016/317] Docs - S2 Cap,Union,Contains,Intersect functions --- docs/en/sql-reference/functions/geo/s2.md | 121 +++++++++++++++++++++- 1 file changed, 120 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/geo/s2.md b/docs/en/sql-reference/functions/geo/s2.md index ebe4cd66d8a..ed065bf2282 100644 --- a/docs/en/sql-reference/functions/geo/s2.md +++ b/docs/en/sql-reference/functions/geo/s2.md @@ -94,7 +94,7 @@ s2ToGeo(s2index) **Arguments** -- `s2Index` — S2 Index. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2index` — S2 Index. [UInt64](../../../sql-reference/data-types/int-uint.md). **Returned values** @@ -118,3 +118,122 @@ Result: │ [5074766987100422144,5074766712222515200,5074767536856236032,5074767261978329088] │ └───────────────────────────────────────────────────────────────────────────────────┘ ``` + +## s2CellsIntersect {#s2CellsIntersect} + +Determines if the two provided [S2](#s2index)) cell indices intersect or not. + +**Syntax** + +``` sql +s2CellsIntersect(s2index1, s2index2) +``` + +**Arguments** + +- `siIndex1`, `s2index2` — S2 Index. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Returned values** + +- 1 — If the S2 cell indices intersect. +- 0 — If the S2 cell indices don't intersect. + +Type: [UInt8](../../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +``` sql + select s2CellsIntersect(9926595209846587392, 9926594385212866560) as intersect; +``` + +Result: + +``` text +┌─intersect─┐ +│ 1 │ +└───────────┘ +``` + + +## s2CapContains {#s2CapContains} + +A cap represents a portion of the sphere that has been cut off by a plane. It is defined by a point on a sphere and a radius in degrees. Imagine that we draw a line through the center of the sphere and our point. An infinite number of planes pass through this line, but any plane will intersect the cap in two points. Thus the angle is defined by one of this points and the entire line. So, the radius of Pi/2 defines a hemisphere and the radius of Pi defines a whole sphere. + +Determines if a cap contains a s2 point index. + +**Syntax** + +``` sql +s2CellsIntersect(center, degrees, point) +``` + +**Arguments** + +- `center` - S2 point index corresponding to the cap. [UInt64](../../../sql-reference/data-types/int-uint.md). + - `degrees` - Radius of the cap in degrees. [Float64](../../../sql-reference/data-types/float.md). + - `point` - S2 point index. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Returned values** + +- 1 — If the cap contains the S2 point index. +- 0 — If the cap doesn't contain the S2 point index. + +Type: [UInt8](../../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + +``` sql +select s2CapContains(1157339245694594829, 1.0, 1157347770437378819) as capContains; +``` + +Result: + +``` text +┌─capContains─┐ +│ 1 │ +└─────────────┘ +``` + + +## s2CapUnion {#s2CapUnion} + +A cap represents a portion of the sphere that has been cut off by a plane. It is defined by a point on a sphere and a radius in degrees. Imagine that we draw a line through the center of the sphere and our point. An infinite number of planes pass through this line, but any plane will intersect the cap in two points. Thus the angle is defined by one of this points and the entire line. So, the radius of Pi/2 defines a hemisphere and the radius of Pi defines a whole sphere. + +Determines the smallest cap that contains the given two input caps. + +**Syntax** + +``` sql +s2CapUnion(center1, radius1, center2, radius2) +``` + +**Arguments** + +- `center1`, `center2` - S2 point indices corresponding to the two input caps. [UInt64](../../../sql-reference/data-types/int-uint.md). + - `radius1`, `radius2` - Radii of the two input caps in degrees. [Float64](../../../sql-reference/data-types/float.md). + +**Returned values** + +- `center` - S2 point index corresponding the center of the smallest cap containing the two input caps. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). + - `radius` - Radius of the smallest cap containing the two input caps. Type: [Float64](../../../sql-reference/data-types/float.md). + + +**Example** + +Query: + +``` sql +SELECT s2CapUnion(3814912406305146967, 1.0, 1157347770437378819, 1.0) AS capUnion; +``` + +Result: + +``` text +┌─capUnion───────────────────────────────┐ +│ (4534655147792050737,60.2088283994957) │ +└────────────────────────────────────────┘ +``` From 1180d33ecfee347aaa400345f6ca17dc89a24150 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Mon, 6 Sep 2021 15:00:15 -0700 Subject: [PATCH 017/317] Docs - S2Rect functions --- docs/en/sql-reference/functions/geo/s2.md | 157 ++++++++++++++++++++++ 1 file changed, 157 insertions(+) diff --git a/docs/en/sql-reference/functions/geo/s2.md b/docs/en/sql-reference/functions/geo/s2.md index ed065bf2282..dba2c232f64 100644 --- a/docs/en/sql-reference/functions/geo/s2.md +++ b/docs/en/sql-reference/functions/geo/s2.md @@ -237,3 +237,160 @@ Result: │ (4534655147792050737,60.2088283994957) │ └────────────────────────────────────────┘ ``` + +## s2RectAdd{#s2RectAdd} + +In the S2 system, a rectangle is represented by a type of S2Region called a S2LatLngRect that represents a rectangle in latitude-longitude space. + +Increases the size of the bounding rectangle to include the given S2 point index. + + +**Syntax** + +``` sql +s2RectAdd(s2pointLow, s2pointHigh, s2Point) +``` + +**Arguments** + +- `s2PointLow` - Low S2 point index corresponding to the rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2PointHigh` - High S2 point index corresponding to the rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Point` - Target S2 point index that the bound rectangle should be grown to include. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Returned values** + +- `s2PointLow` - Low S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). + - `s2PointHigh` - Hight S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/float.md). + + +**Example** + +Query: + +``` sql +SELECT s2RectAdd(5178914411069187297, 5177056748191934217, 5179056748191934217) as rectAdd; +``` + +Result: + +``` text +┌─rectAdd───────────────────────────────────┐ +│ (5179062030687166815,5177056748191934217) │ +└───────────────────────────────────────────┘ +``` + + +## s2RectContains{#s2RectContains} + +In the S2 system, a rectangle is represented by a type of S2Region called a S2LatLngRect that represents a rectangle in latitude-longitude space. + +Determines if a given rectangle contains a S2 point index. + +**Syntax** + +``` sql +s2RectContains(s2PointLow, s2PointHi, s2Point) +``` + +**Arguments** + +- `s2PointLow` - Low S2 point index corresponding to the rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2PointHigh` - High S2 point index corresponding to the rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Point` - Target S2 point index. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Returned values** + +- 1 — If the rectangle contains the given S2 point. +- 0 — If the rectangle doesn't contain the given S2 point. + +**Example** + +Query: + +``` sql +SELECT s2RectContains(5179062030687166815, 5177056748191934217, 5177914411069187297) AS rectContains +``` + +Result: + +``` text +┌─rectContains─┐ +│ 0 │ +└──────────────┘ +``` + +## s2RectUinion{#s2RectUnion} + +In the S2 system, a rectangle is represented by a type of S2Region called a S2LatLngRect that represents a rectangle in latitude-longitude space. + +Returns the smallest rectangle containing the union of this rectangle and the given rectangle. + +**Syntax** + +``` sql +s2RectUnion(s2Rect1PointLow, s2Rect1PointHi, s2Rect2PointLow, s2Rect2PointHi) +``` + +**Arguments** + +- `s2Rect1PointLow`, `s2Rect1PointHi` - Low and High S2 point indices corresponding to the first rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Rect2PointLow`, `s2Rect2PointHi` - Low and High S2 point indices corresponding to the second rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Returned values** + +- `s2UnionRect2PointLow` - Low S2 cell id corresponding to the union rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2UnionRect2PointHi` - High S2 cell id corresponding to the union rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). + + +**Example** + +Query: + +``` sql +SELECT s2RectUnion(5178914411069187297, 5177056748191934217, 5179062030687166815, 5177056748191934217) AS rectUnion +``` + +Result: + +``` text +┌─rectUnion─────────────────────────────────┐ +│ (5179062030687166815,5177056748191934217) │ +└───────────────────────────────────────────┘ +``` + +## s2RectIntersection{#s2RectIntersection} + +Returns the smallest Rectangle containing the intersection of this rectangle and the given rectangle. + +**Syntax** + +``` sql +s2RectIntersection(s2Rect1PointLow, s2Rect1PointHi, s2Rect2PointLow, s2Rect2PointHi) +``` + +**Arguments** + +- `s2Rect1PointLow`, `s2Rect1PointHi` - Low and High S2 point indices corresponding to the first rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2Rect2PointLow`, `s2Rect2PointHi` - Low and High S2 point indices corresponding to the second rectangle. [UInt64](../../../sql-reference/data-types/int-uint.md). + +**Returned values** + +- `s2UnionRect2PointLow` - Low S2 cell id corresponding to the rectangle containing the intersection of the given rectangles. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). +- `s2UnionRect2PointHi` - Hi S2 cell id corresponding to the rectangle containing the intersection of the given rectangles. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). + + +**Example** + +Query: + +``` sql +SELECT s2RectIntersection(5178914411069187297, 5177056748191934217, 5179062030687166815, 5177056748191934217) AS rectIntersection +``` + +Result: + +``` text +┌─rectIntersection──────────────────────────┐ +│ (5178914411069187297,5177056748191934217) │ +└───────────────────────────────────────────┘ +``` From be0a2c03337f569393622b8f467f5847ec2f06ce Mon Sep 17 00:00:00 2001 From: bharatnc Date: Tue, 7 Sep 2021 22:25:50 -0700 Subject: [PATCH 018/317] Docs - minor corrections --- docs/en/sql-reference/functions/geo/s2.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/s2.md b/docs/en/sql-reference/functions/geo/s2.md index dba2c232f64..532266ea983 100644 --- a/docs/en/sql-reference/functions/geo/s2.md +++ b/docs/en/sql-reference/functions/geo/s2.md @@ -159,7 +159,7 @@ Result: ## s2CapContains {#s2CapContains} -A cap represents a portion of the sphere that has been cut off by a plane. It is defined by a point on a sphere and a radius in degrees. Imagine that we draw a line through the center of the sphere and our point. An infinite number of planes pass through this line, but any plane will intersect the cap in two points. Thus the angle is defined by one of this points and the entire line. So, the radius of Pi/2 defines a hemisphere and the radius of Pi defines a whole sphere. +A cap represents a portion of the sphere that has been cut off by a plane. It is defined by a point on a sphere and a radius in degrees. Determines if a cap contains a s2 point index. @@ -201,7 +201,7 @@ Result: ## s2CapUnion {#s2CapUnion} -A cap represents a portion of the sphere that has been cut off by a plane. It is defined by a point on a sphere and a radius in degrees. Imagine that we draw a line through the center of the sphere and our point. An infinite number of planes pass through this line, but any plane will intersect the cap in two points. Thus the angle is defined by one of this points and the entire line. So, the radius of Pi/2 defines a hemisphere and the radius of Pi defines a whole sphere. +A cap represents a portion of the sphere that has been cut off by a plane. It is defined by a point on a sphere and a radius in degrees. Determines the smallest cap that contains the given two input caps. From 999a4fe83129427c2dab6808321848b5203247e9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 8 Sep 2021 21:29:38 +0300 Subject: [PATCH 019/317] 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 020/317] 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 021/317] 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 022/317] 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 023/317] 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 024/317] 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 025/317] 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 026/317] 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 027/317] 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 028/317] 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 029/317] 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 030/317] 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 031/317] 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 032/317] 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 033/317] 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 034/317] 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 035/317] 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 036/317] 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 037/317] 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 038/317] 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 039/317] 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 040/317] 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 041/317] 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 042/317] 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 bc8dd77d0fc09ba84422f4dbaae20a2e9140a9d3 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 14 Sep 2021 15:36:35 +0300 Subject: [PATCH 043/317] Update menus dataset - Update menus dataset. - Add some crosslinks. --- .../getting-started/example-datasets/menus.md | 91 ++++++++++++------- docs/en/interfaces/formats.md | 2 +- .../getting-started/example-datasets/index.md | 3 +- .../getting-started/example-datasets/menus.md | 14 +++ docs/ru/interfaces/formats.md | 2 +- 5 files changed, 78 insertions(+), 34 deletions(-) create mode 100644 docs/ru/getting-started/example-datasets/menus.md diff --git a/docs/en/getting-started/example-datasets/menus.md b/docs/en/getting-started/example-datasets/menus.md index 8f330f39226..388b1bc53b6 100644 --- a/docs/en/getting-started/example-datasets/menus.md +++ b/docs/en/getting-started/example-datasets/menus.md @@ -15,7 +15,7 @@ The size is just 1.3 million records about dishes in the menus (a very small dat ## Download the Dataset -``` +```bash wget https://s3.amazonaws.com/menusdata.nypl.org/gzips/2021_08_01_07_01_17_data.tgz ``` @@ -24,21 +24,21 @@ Download size is about 35 MB. ## Unpack the Dataset -``` +```bash tar xvf 2021_08_01_07_01_17_data.tgz ``` Uncompressed size is about 150 MB. The data is normalized consisted of four tables: -- Menu: information about menus: the name of the restaurant, the date when menu was seen, etc; -- Dish: information about dishes: the name of the dish along with some characteristic; -- MenuPage: information about the pages in the menus; every page belongs to some menu; -- MenuItem: an item of the menu - a dish along with its price on some menu page: links to dish and menu page. +- Menu — information about menus: the name of the restaurant, the date when menu was seen, etc. +- Dish — information about dishes: the name of the dish along with some characteristic. +- MenuPage — information about the pages in the menus, because every page belongs to some menu. +- MenuItem — an item of the menu. A dish along with its price on some menu page: links to dish and menu page. ## Create the Tables -``` +```sql CREATE TABLE dish ( id UInt32, @@ -101,26 +101,26 @@ CREATE TABLE menu_item ) ENGINE = MergeTree ORDER BY id; ``` -We use `Decimal` data type to store prices. Everything else is quite straightforward. +We use [Decimal](../../sql-reference/data-types/decimal.md) data type to store prices. Everything else is quite straightforward. ## Import Data Upload data into ClickHouse: -``` +```bash clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO dish FORMAT CSVWithNames" < Dish.csv clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO menu FORMAT CSVWithNames" < Menu.csv clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO menu_page FORMAT CSVWithNames" < MenuPage.csv clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --date_time_input_format best_effort --query "INSERT INTO menu_item FORMAT CSVWithNames" < MenuItem.csv ``` -We use `CSVWithNames` format as the data is represented by CSV with header. +We use [CSVWithNames](../../interfaces/formats.md#csvwithnames) format as the data is represented by CSV with header. We disable `format_csv_allow_single_quotes` as only double quotes are used for data fields and single quotes can be inside the values and should not confuse the CSV parser. We disable `input_format_null_as_default` as our data does not have NULLs. Otherwise ClickHouse will try to parse `\N` sequences and can be confused with `\` in data. -The setting `--date_time_input_format best_effort` allows to parse `DateTime` fields in wide variety of formats. For example, ISO-8601 without seconds like '2000-01-01 01:02' will be recognized. Without this setting only fixed DateTime format is allowed. +The setting `--date_time_input_format best_effort` allows to parse `DateTime` fields in wide variety of formats. For example, ISO-8601 without seconds like '2000-01-01 01:02' will be recognized. Without this setting only fixed [DateTime](../../sql-reference/data-types/datetime.md) format is allowed. ## Denormalize the Data @@ -129,7 +129,7 @@ For typical analytical tasks it is way more efficient to deal with pre-JOINed da We will create a table that will contain all the data JOINed together: -``` +```sql CREATE TABLE menu_item_denorm ENGINE = MergeTree ORDER BY (dish_name, created_at) AS SELECT @@ -171,21 +171,32 @@ AS SELECT FROM menu_item JOIN dish ON menu_item.dish_id = dish.id JOIN menu_page ON menu_item.menu_page_id = menu_page.id - JOIN menu ON menu_page.menu_id = menu.id + JOIN menu ON menu_page.menu_id = menu.id; ``` ## Validate the Data +Query: + +```sql +SELECT count() FROM menu_item_denorm; ``` -SELECT count() FROM menu_item_denorm -1329175 + +Result: + +```text +┌─count()─┐ +│ 1329175 │ +└─────────┘ ``` ## Run Some Queries -Averaged historical prices of dishes: +### Averaged historical prices of dishes -``` +Query: + +```sql SELECT round(toUInt32OrZero(extract(menu_date, '^\\d{4}')), -1) AS d, count(), @@ -194,8 +205,12 @@ SELECT FROM menu_item_denorm WHERE (menu_currency = 'Dollars') AND (d > 0) AND (d < 2022) GROUP BY d -ORDER BY d ASC +ORDER BY d ASC; +``` +Result: + +```text ┌────d─┬─count()─┬─round(avg(price), 2)─┬─bar(avg(price), 0, 100, 100)─┐ │ 1850 │ 618 │ 1.5 │ █▍ │ │ 1860 │ 1634 │ 1.29 │ █▎ │ @@ -215,15 +230,15 @@ ORDER BY d ASC │ 2000 │ 2467 │ 11.85 │ ███████████▋ │ │ 2010 │ 597 │ 25.66 │ █████████████████████████▋ │ └──────┴─────────┴──────────────────────┴──────────────────────────────┘ - -17 rows in set. Elapsed: 0.044 sec. Processed 1.33 million rows, 54.62 MB (30.00 million rows/s., 1.23 GB/s.) ``` Take it with a grain of salt. -### Burger Prices: +### Burger Prices -``` +Query: + +```sql SELECT round(toUInt32OrZero(extract(menu_date, '^\\d{4}')), -1) AS d, count(), @@ -232,8 +247,12 @@ SELECT FROM menu_item_denorm WHERE (menu_currency = 'Dollars') AND (d > 0) AND (d < 2022) AND (dish_name ILIKE '%burger%') GROUP BY d -ORDER BY d ASC +ORDER BY d ASC; +``` +Result: + +```text ┌────d─┬─count()─┬─round(avg(price), 2)─┬─bar(avg(price), 0, 50, 100)───────────┐ │ 1880 │ 2 │ 0.42 │ ▋ │ │ 1890 │ 7 │ 0.85 │ █▋ │ @@ -250,13 +269,13 @@ ORDER BY d ASC │ 2000 │ 21 │ 7.14 │ ██████████████▎ │ │ 2010 │ 6 │ 18.42 │ ████████████████████████████████████▋ │ └──────┴─────────┴──────────────────────┴───────────────────────────────────────┘ - -14 rows in set. Elapsed: 0.052 sec. Processed 1.33 million rows, 94.15 MB (25.48 million rows/s., 1.80 GB/s.) ``` -### Vodka: +### Vodka -``` +Query: + +```sql SELECT round(toUInt32OrZero(extract(menu_date, '^\\d{4}')), -1) AS d, count(), @@ -265,8 +284,12 @@ SELECT FROM menu_item_denorm WHERE (menu_currency IN ('Dollars', '')) AND (d > 0) AND (d < 2022) AND (dish_name ILIKE '%vodka%') GROUP BY d -ORDER BY d ASC +ORDER BY d ASC; +``` +Result: + +```text ┌────d─┬─count()─┬─round(avg(price), 2)─┬─bar(avg(price), 0, 50, 100)─┐ │ 1910 │ 2 │ 0 │ │ │ 1920 │ 1 │ 0.3 │ ▌ │ @@ -282,11 +305,13 @@ ORDER BY d ASC To get vodka we have to write `ILIKE '%vodka%'` and this definitely makes a statement. -### Caviar: +### Caviar Let's print caviar prices. Also let's print a name of any dish with caviar. -``` +Query: + +```sql SELECT round(toUInt32OrZero(extract(menu_date, '^\\d{4}')), -1) AS d, count(), @@ -296,8 +321,12 @@ SELECT FROM menu_item_denorm WHERE (menu_currency IN ('Dollars', '')) AND (d > 0) AND (d < 2022) AND (dish_name ILIKE '%caviar%') GROUP BY d -ORDER BY d ASC +ORDER BY d ASC; +``` +Result: + +```text ┌────d─┬─count()─┬─round(avg(price), 2)─┬─bar(avg(price), 0, 50, 100)──────┬─any(dish_name)──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ │ 1090 │ 1 │ 0 │ │ Caviar │ │ 1880 │ 3 │ 0 │ │ Caviar │ diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index f353f0b1d43..a04bcbd46f4 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -387,7 +387,7 @@ The CSV format supports the output of totals and extremes the same way as `TabSe ## CSVWithNames {#csvwithnames} -Also prints the header row, similar to `TabSeparatedWithNames`. +Also prints the header row, similar to [TabSeparatedWithNames](#tabseparatedwithnames). ## CustomSeparated {#format-customseparated} diff --git a/docs/ru/getting-started/example-datasets/index.md b/docs/ru/getting-started/example-datasets/index.md index 756b3a75dee..fe49e9241a7 100644 --- a/docs/ru/getting-started/example-datasets/index.md +++ b/docs/ru/getting-started/example-datasets/index.md @@ -15,6 +15,7 @@ toc_title: "Введение" - [Терабайт логов кликов от Criteo](criteo.md) - [AMPLab Big Data Benchmark](amplab-benchmark.md) - [Данные о такси в Нью-Йорке](nyc-taxi.md) +- [What's on the Menu?](menus.md) - [OnTime](ontime.md) -- [Вышки сотовой связи](../../getting-started/example-datasets/cell-towers.md) +- [Вышки сотовой связи](cell-towers.md) diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md new file mode 100644 index 00000000000..b86525476db --- /dev/null +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -0,0 +1,14 @@ +--- +toc_priority: 21 +toc_title: Menus +--- + +# New York Public Library "What's on the Menu?" Dataset + +The dataset is created by the New York Public Library. It contains historical data on the menus of hotels, restaurants and cafes with the dishes along with their prices. + +Source: http://menus.nypl.org/data +The data is in public domain. + +The data is from library's archive and it may be incomplete and difficult for statistical analysis. Nevertheless it is also very yummy. +The size is just 1.3 million records about dishes in the menus (a very small data volume for ClickHouse, but it's still a good example). diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index a02e1436d36..1b686602159 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -365,7 +365,7 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR ## CSVWithNames {#csvwithnames} -Выводит также заголовок, аналогично `TabSeparatedWithNames`. +Выводит также заголовок, аналогично [TabSeparatedWithNames](#tabseparatedwithnames). ## CustomSeparated {#format-customseparated} From 6a32a949e6d23b87295e4f59202cfeca752376ed Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Sep 2021 17:22:26 +0300 Subject: [PATCH 044/317] 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 045/317] 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 046/317] 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 1be67e00cb65514806a487730689d20851b9c8c6 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 14 Sep 2021 21:03:05 +0300 Subject: [PATCH 047/317] Update translation. --- .../getting-started/example-datasets/menus.md | 4 +- .../getting-started/example-datasets/menus.md | 353 +++++++++++++++++- 2 files changed, 351 insertions(+), 6 deletions(-) diff --git a/docs/en/getting-started/example-datasets/menus.md b/docs/en/getting-started/example-datasets/menus.md index 388b1bc53b6..7cb8b2da203 100644 --- a/docs/en/getting-started/example-datasets/menus.md +++ b/docs/en/getting-started/example-datasets/menus.md @@ -11,10 +11,12 @@ Source: http://menus.nypl.org/data The data is in public domain. The data is from library's archive and it may be incomplete and difficult for statistical analysis. Nevertheless it is also very yummy. -The size is just 1.3 million records about dishes in the menus (a very small data volume for ClickHouse, but it's still a good example). +The size is just 1.3 million records about dishes in the menus — it's a very small data volume for ClickHouse, but it's still a good example. ## Download the Dataset +Run the command: + ```bash wget https://s3.amazonaws.com/menusdata.nypl.org/gzips/2021_08_01_07_01_17_data.tgz ``` diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index b86525476db..add7ca94683 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -5,10 +5,353 @@ toc_title: Menus # New York Public Library "What's on the Menu?" Dataset -The dataset is created by the New York Public Library. It contains historical data on the menus of hotels, restaurants and cafes with the dishes along with their prices. +Набор данных создан Нью-Йоркской публичной библиотекой. Он содержит исторические данные о меню отелей, ресторанов и кафе с блюдами, а также их ценами. -Source: http://menus.nypl.org/data -The data is in public domain. +Источник: http://menus.nypl.org/data +Эти данные находятся в открытом доступе. -The data is from library's archive and it may be incomplete and difficult for statistical analysis. Nevertheless it is also very yummy. -The size is just 1.3 million records about dishes in the menus (a very small data volume for ClickHouse, but it's still a good example). +Данные взяты из архива библиотеки, и они могут быть неполными и сложными для статистического анализа. Тем не менее это тоже очень интересно. +В наборе всего 1,3 миллиона записей о блюдах в меню — очень небольшой объем данных для ClickHouse, но это все равно хороший пример. + +## Загрузите набор данных + +Выполните команду: + +```bash +wget https://s3.amazonaws.com/menusdata.nypl.org/gzips/2021_08_01_07_01_17_data.tgz +``` + +Замените ссылку на актуальную ссылку с http://menus.nypl.org/data, если это необходимо. +Размер загрузки составляет около 35 МБ. + +## Распакуйте набор данных + +```bash +tar xvf 2021_08_01_07_01_17_data.tgz +``` + +Размер распакованных данных составляет около 150 МБ. + +The data is normalized consisted of four tables: +- Menu — information about menus: the name of the restaurant, the date when menu was seen, etc. +- Dish — information about dishes: the name of the dish along with some characteristic. +- MenuPage — information about the pages in the menus, because every page belongs to some menu. +- MenuItem — an item of the menu. A dish along with its price on some menu page: links to dish and menu page. + +## Создайте таблицы + +```sql +CREATE TABLE dish +( + id UInt32, + name String, + description String, + menus_appeared UInt32, + times_appeared Int32, + first_appeared UInt16, + last_appeared UInt16, + lowest_price Decimal64(3), + highest_price Decimal64(3) +) ENGINE = MergeTree ORDER BY id; + +CREATE TABLE menu +( + id UInt32, + name String, + sponsor String, + event String, + venue String, + place String, + physical_description String, + occasion String, + notes String, + call_number String, + keywords String, + language String, + date String, + location String, + location_type String, + currency String, + currency_symbol String, + status String, + page_count UInt16, + dish_count UInt16 +) ENGINE = MergeTree ORDER BY id; + +CREATE TABLE menu_page +( + id UInt32, + menu_id UInt32, + page_number UInt16, + image_id String, + full_height UInt16, + full_width UInt16, + uuid UUID +) ENGINE = MergeTree ORDER BY id; + +CREATE TABLE menu_item +( + id UInt32, + menu_page_id UInt32, + price Decimal64(3), + high_price Decimal64(3), + dish_id UInt32, + created_at DateTime, + updated_at DateTime, + xpos Float64, + ypos Float64 +) ENGINE = MergeTree ORDER BY id; +``` + +We use [Decimal](../../sql-reference/data-types/decimal.md) data type to store prices. Everything else is quite straightforward. + +## Import Data + +Upload data into ClickHouse: + +```bash +clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO dish FORMAT CSVWithNames" < Dish.csv +clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO menu FORMAT CSVWithNames" < Menu.csv +clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO menu_page FORMAT CSVWithNames" < MenuPage.csv +clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --date_time_input_format best_effort --query "INSERT INTO menu_item FORMAT CSVWithNames" < MenuItem.csv +``` + +We use [CSVWithNames](../../interfaces/formats.md#csvwithnames) format as the data is represented by CSV with header. + +We disable `format_csv_allow_single_quotes` as only double quotes are used for data fields and single quotes can be inside the values and should not confuse the CSV parser. + +We disable `input_format_null_as_default` as our data does not have NULLs. Otherwise ClickHouse will try to parse `\N` sequences and can be confused with `\` in data. + +The setting `--date_time_input_format best_effort` allows to parse `DateTime` fields in wide variety of formats. For example, ISO-8601 without seconds like '2000-01-01 01:02' will be recognized. Without this setting only fixed [DateTime](../../sql-reference/data-types/datetime.md) format is allowed. + + +## Denormalize the Data + +Data is presented in multiple tables in normalized form. It means you have to perform JOINs if you want to query, e.g. dish names from menu items. +For typical analytical tasks it is way more efficient to deal with pre-JOINed data to avoid doing JOIN every time. It is called "denormalized" data. + +We will create a table that will contain all the data JOINed together: + +```sql +CREATE TABLE menu_item_denorm +ENGINE = MergeTree ORDER BY (dish_name, created_at) +AS SELECT + price, + high_price, + created_at, + updated_at, + xpos, + ypos, + dish.id AS dish_id, + dish.name AS dish_name, + dish.description AS dish_description, + dish.menus_appeared AS dish_menus_appeared, + dish.times_appeared AS dish_times_appeared, + dish.first_appeared AS dish_first_appeared, + dish.last_appeared AS dish_last_appeared, + dish.lowest_price AS dish_lowest_price, + dish.highest_price AS dish_highest_price, + menu.id AS menu_id, + menu.name AS menu_name, + menu.sponsor AS menu_sponsor, + menu.event AS menu_event, + menu.venue AS menu_venue, + menu.place AS menu_place, + menu.physical_description AS menu_physical_description, + menu.occasion AS menu_occasion, + menu.notes AS menu_notes, + menu.call_number AS menu_call_number, + menu.keywords AS menu_keywords, + menu.language AS menu_language, + menu.date AS menu_date, + menu.location AS menu_location, + menu.location_type AS menu_location_type, + menu.currency AS menu_currency, + menu.currency_symbol AS menu_currency_symbol, + menu.status AS menu_status, + menu.page_count AS menu_page_count, + menu.dish_count AS menu_dish_count +FROM menu_item + JOIN dish ON menu_item.dish_id = dish.id + JOIN menu_page ON menu_item.menu_page_id = menu_page.id + JOIN menu ON menu_page.menu_id = menu.id; +``` + + +## Validate the Data + +Запрос: + +```sql +SELECT count() FROM menu_item_denorm; +``` + +Результат: + +```text +┌─count()─┐ +│ 1329175 │ +└─────────┘ +``` + +## Примеры + +### Усредненные исторические цены на блюда + +Запрос: + +```sql +SELECT + round(toUInt32OrZero(extract(menu_date, '^\\d{4}')), -1) AS d, + count(), + round(avg(price), 2), + bar(avg(price), 0, 100, 100) +FROM menu_item_denorm +WHERE (menu_currency = 'Dollars') AND (d > 0) AND (d < 2022) +GROUP BY d +ORDER BY d ASC; +``` + +Результат: + +```text +┌────d─┬─count()─┬─round(avg(price), 2)─┬─bar(avg(price), 0, 100, 100)─┐ +│ 1850 │ 618 │ 1.5 │ █▍ │ +│ 1860 │ 1634 │ 1.29 │ █▎ │ +│ 1870 │ 2215 │ 1.36 │ █▎ │ +│ 1880 │ 3909 │ 1.01 │ █ │ +│ 1890 │ 8837 │ 1.4 │ █▍ │ +│ 1900 │ 176292 │ 0.68 │ ▋ │ +│ 1910 │ 212196 │ 0.88 │ ▊ │ +│ 1920 │ 179590 │ 0.74 │ ▋ │ +│ 1930 │ 73707 │ 0.6 │ ▌ │ +│ 1940 │ 58795 │ 0.57 │ ▌ │ +│ 1950 │ 41407 │ 0.95 │ ▊ │ +│ 1960 │ 51179 │ 1.32 │ █▎ │ +│ 1970 │ 12914 │ 1.86 │ █▋ │ +│ 1980 │ 7268 │ 4.35 │ ████▎ │ +│ 1990 │ 11055 │ 6.03 │ ██████ │ +│ 2000 │ 2467 │ 11.85 │ ███████████▋ │ +│ 2010 │ 597 │ 25.66 │ █████████████████████████▋ │ +└──────┴─────────┴──────────────────────┴──────────────────────────────┘ +``` + +Просто не принимайте это всерьез. + +### Цены на бургеры + +Запрос: + +```sql +SELECT + round(toUInt32OrZero(extract(menu_date, '^\\d{4}')), -1) AS d, + count(), + round(avg(price), 2), + bar(avg(price), 0, 50, 100) +FROM menu_item_denorm +WHERE (menu_currency = 'Dollars') AND (d > 0) AND (d < 2022) AND (dish_name ILIKE '%burger%') +GROUP BY d +ORDER BY d ASC; +``` + +Результат: + +```text +┌────d─┬─count()─┬─round(avg(price), 2)─┬─bar(avg(price), 0, 50, 100)───────────┐ +│ 1880 │ 2 │ 0.42 │ ▋ │ +│ 1890 │ 7 │ 0.85 │ █▋ │ +│ 1900 │ 399 │ 0.49 │ ▊ │ +│ 1910 │ 589 │ 0.68 │ █▎ │ +│ 1920 │ 280 │ 0.56 │ █ │ +│ 1930 │ 74 │ 0.42 │ ▋ │ +│ 1940 │ 119 │ 0.59 │ █▏ │ +│ 1950 │ 134 │ 1.09 │ ██▏ │ +│ 1960 │ 272 │ 0.92 │ █▋ │ +│ 1970 │ 108 │ 1.18 │ ██▎ │ +│ 1980 │ 88 │ 2.82 │ █████▋ │ +│ 1990 │ 184 │ 3.68 │ ███████▎ │ +│ 2000 │ 21 │ 7.14 │ ██████████████▎ │ +│ 2010 │ 6 │ 18.42 │ ████████████████████████████████████▋ │ +└──────┴─────────┴──────────────────────┴───────────────────────────────────────┘ +``` + +### Водка + +Запрос: + +```sql +SELECT + round(toUInt32OrZero(extract(menu_date, '^\\d{4}')), -1) AS d, + count(), + round(avg(price), 2), + bar(avg(price), 0, 50, 100) +FROM menu_item_denorm +WHERE (menu_currency IN ('Dollars', '')) AND (d > 0) AND (d < 2022) AND (dish_name ILIKE '%vodka%') +GROUP BY d +ORDER BY d ASC; +``` + +Результат: + +```text +┌────d─┬─count()─┬─round(avg(price), 2)─┬─bar(avg(price), 0, 50, 100)─┐ +│ 1910 │ 2 │ 0 │ │ +│ 1920 │ 1 │ 0.3 │ ▌ │ +│ 1940 │ 21 │ 0.42 │ ▋ │ +│ 1950 │ 14 │ 0.59 │ █▏ │ +│ 1960 │ 113 │ 2.17 │ ████▎ │ +│ 1970 │ 37 │ 0.68 │ █▎ │ +│ 1980 │ 19 │ 2.55 │ █████ │ +│ 1990 │ 86 │ 3.6 │ ███████▏ │ +│ 2000 │ 2 │ 3.98 │ ███████▊ │ +└──────┴─────────┴──────────────────────┴─────────────────────────────┘ +``` + +To get vodka we have to write `ILIKE '%vodka%'` and this definitely makes a statement. + +### Икра + +Давайте выведем цены на икру. Также давайте выведем название любого блюда с икрой. + +Запрос: + +```sql +SELECT + round(toUInt32OrZero(extract(menu_date, '^\\d{4}')), -1) AS d, + count(), + round(avg(price), 2), + bar(avg(price), 0, 50, 100), + any(dish_name) +FROM menu_item_denorm +WHERE (menu_currency IN ('Dollars', '')) AND (d > 0) AND (d < 2022) AND (dish_name ILIKE '%caviar%') +GROUP BY d +ORDER BY d ASC; +``` + +Результат: + +```text +┌────d─┬─count()─┬─round(avg(price), 2)─┬─bar(avg(price), 0, 50, 100)──────┬─any(dish_name)──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ 1090 │ 1 │ 0 │ │ Caviar │ +│ 1880 │ 3 │ 0 │ │ Caviar │ +│ 1890 │ 39 │ 0.59 │ █▏ │ Butter and caviar │ +│ 1900 │ 1014 │ 0.34 │ ▋ │ Anchovy Caviar on Toast │ +│ 1910 │ 1588 │ 1.35 │ ██▋ │ 1/1 Brötchen Caviar │ +│ 1920 │ 927 │ 1.37 │ ██▋ │ ASTRAKAN CAVIAR │ +│ 1930 │ 289 │ 1.91 │ ███▋ │ Astrachan caviar │ +│ 1940 │ 201 │ 0.83 │ █▋ │ (SPECIAL) Domestic Caviar Sandwich │ +│ 1950 │ 81 │ 2.27 │ ████▌ │ Beluga Caviar │ +│ 1960 │ 126 │ 2.21 │ ████▍ │ Beluga Caviar │ +│ 1970 │ 105 │ 0.95 │ █▊ │ BELUGA MALOSSOL CAVIAR AMERICAN DRESSING │ +│ 1980 │ 12 │ 7.22 │ ██████████████▍ │ Authentic Iranian Beluga Caviar the world's finest black caviar presented in ice garni and a sampling of chilled 100° Russian vodka │ +│ 1990 │ 74 │ 14.42 │ ████████████████████████████▋ │ Avocado Salad, Fresh cut avocado with caviare │ +│ 2000 │ 3 │ 7.82 │ ███████████████▋ │ Aufgeschlagenes Kartoffelsueppchen mit Forellencaviar │ +│ 2010 │ 6 │ 15.58 │ ███████████████████████████████▏ │ "OYSTERS AND PEARLS" "Sabayon" of Pearl Tapioca with Island Creek Oysters and Russian Sevruga Caviar │ +└──────┴─────────┴──────────────────────┴──────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +По крайней мере, есть икра с водкой. Очень мило. + +### Test it in Playground + +The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICByb3VuZCh0b1VJbnQzMk9yWmVybyhleHRyYWN0KG1lbnVfZGF0ZSwgJ15cXGR7NH0nKSksIC0xKSBBUyBkLAogICAgY291bnQoKSwKICAgIHJvdW5kKGF2ZyhwcmljZSksIDIpLAogICAgYmFyKGF2ZyhwcmljZSksIDAsIDUwLCAxMDApLAogICAgYW55KGRpc2hfbmFtZSkKRlJPTSBtZW51X2l0ZW1fZGVub3JtCldIRVJFIChtZW51X2N1cnJlbmN5IElOICgnRG9sbGFycycsICcnKSkgQU5EIChkID4gMCkgQU5EIChkIDwgMjAyMikgQU5EIChkaXNoX25hbWUgSUxJS0UgJyVjYXZpYXIlJykKR1JPVVAgQlkgZApPUkRFUiBCWSBkIEFTQw==). From 69c896df668ce9fbdd83c17f8d2196d000249ebb Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Wed, 15 Sep 2021 09:45:11 +0300 Subject: [PATCH 048/317] 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 049/317] 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 050/317] 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 051/317] 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 052/317] 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 053/317] 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 054/317] 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 055/317] 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 056/317] 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 057/317] 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 058/317] 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 059/317] 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 060/317] 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 061/317] 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 062/317] 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 063/317] 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 064/317] 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 065/317] 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 d3e6b9044a73e0758acf73a8fae99705becd2ed7 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 16 Sep 2021 17:02:36 +0300 Subject: [PATCH 066/317] Add anchors to all the headings. --- .../getting-started/example-datasets/menus.md | 32 +++++++++---------- .../getting-started/example-datasets/menus.md | 27 ++++++++-------- 2 files changed, 29 insertions(+), 30 deletions(-) diff --git a/docs/en/getting-started/example-datasets/menus.md b/docs/en/getting-started/example-datasets/menus.md index 7cb8b2da203..6cf4a5ecd4f 100644 --- a/docs/en/getting-started/example-datasets/menus.md +++ b/docs/en/getting-started/example-datasets/menus.md @@ -3,7 +3,7 @@ toc_priority: 21 toc_title: Menus --- -# New York Public Library "What's on the Menu?" Dataset +# New York Public Library "What's on the Menu?" Dataset {#menus-dataset} The dataset is created by the New York Public Library. It contains historical data on the menus of hotels, restaurants and cafes with the dishes along with their prices. @@ -13,7 +13,7 @@ The data is in public domain. The data is from library's archive and it may be incomplete and difficult for statistical analysis. Nevertheless it is also very yummy. The size is just 1.3 million records about dishes in the menus — it's a very small data volume for ClickHouse, but it's still a good example. -## Download the Dataset +## Download the Dataset {#download-dataset} Run the command: @@ -24,7 +24,7 @@ wget https://s3.amazonaws.com/menusdata.nypl.org/gzips/2021_08_01_07_01_17_data. Replace the link to the up to date link from http://menus.nypl.org/data if needed. Download size is about 35 MB. -## Unpack the Dataset +## Unpack the Dataset {#unpack-dataset} ```bash tar xvf 2021_08_01_07_01_17_data.tgz @@ -38,7 +38,7 @@ The data is normalized consisted of four tables: - MenuPage — information about the pages in the menus, because every page belongs to some menu. - MenuItem — an item of the menu. A dish along with its price on some menu page: links to dish and menu page. -## Create the Tables +## Create the Tables {#create-tables} ```sql CREATE TABLE dish @@ -105,7 +105,7 @@ CREATE TABLE menu_item We use [Decimal](../../sql-reference/data-types/decimal.md) data type to store prices. Everything else is quite straightforward. -## Import Data +## Import the Data {#import-data} Upload data into ClickHouse: @@ -120,13 +120,13 @@ We use [CSVWithNames](../../interfaces/formats.md#csvwithnames) format as the da We disable `format_csv_allow_single_quotes` as only double quotes are used for data fields and single quotes can be inside the values and should not confuse the CSV parser. -We disable `input_format_null_as_default` as our data does not have NULLs. Otherwise ClickHouse will try to parse `\N` sequences and can be confused with `\` in data. +We disable [input_format_null_as_default](../../operations/settings/settings.mdsettings-input-format-null-as-default) as our data does not have [NULL](../../sql-reference/syntax.md#null-literal). Otherwise ClickHouse will try to parse `\N` sequences and can be confused with `\` in data. -The setting `--date_time_input_format best_effort` allows to parse `DateTime` fields in wide variety of formats. For example, ISO-8601 without seconds like '2000-01-01 01:02' will be recognized. Without this setting only fixed [DateTime](../../sql-reference/data-types/datetime.md) format is allowed. +The setting [--date_time_input_format best_effort](../../operations/settings/settings.md#settings-date_time_input_format) allows to parse [DateTime](../../sql-reference/data-types/datetime.md) fields in wide variety of formats. For example, ISO-8601 without seconds like '2000-01-01 01:02' will be recognized. Without this setting only fixed DateTime format is allowed. -## Denormalize the Data +## Denormalize the Data {#denormalize-data} -Data is presented in multiple tables in normalized form. It means you have to perform JOINs if you want to query, e.g. dish names from menu items. +Data is presented in multiple tables in normalized form. It means you have to perform [JOIN](../../sql-reference/statements/select/join.md#select-join) if you want to query, e.g. dish names from menu items. For typical analytical tasks it is way more efficient to deal with pre-JOINed data to avoid doing JOIN every time. It is called "denormalized" data. We will create a table that will contain all the data JOINed together: @@ -176,7 +176,7 @@ FROM menu_item JOIN menu ON menu_page.menu_id = menu.id; ``` -## Validate the Data +## Validate the Data {#validate-data} Query: @@ -192,9 +192,9 @@ Result: └─────────┘ ``` -## Run Some Queries +## Run Some Queries {#run-queries} -### Averaged historical prices of dishes +### Averaged historical prices of dishes {#query-averaged-historical-prices} Query: @@ -236,7 +236,7 @@ Result: Take it with a grain of salt. -### Burger Prices +### Burger Prices {#query-burger-prices} Query: @@ -273,7 +273,7 @@ Result: └──────┴─────────┴──────────────────────┴───────────────────────────────────────┘ ``` -### Vodka +### Vodka {#query-vodka} Query: @@ -307,7 +307,7 @@ Result: To get vodka we have to write `ILIKE '%vodka%'` and this definitely makes a statement. -### Caviar +### Caviar {#query-caviar} Let's print caviar prices. Also let's print a name of any dish with caviar. @@ -350,6 +350,6 @@ Result: At least they have caviar with vodka. Very nice. -### Test it in Playground +### Test it in Playground {#playground} The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICByb3VuZCh0b1VJbnQzMk9yWmVybyhleHRyYWN0KG1lbnVfZGF0ZSwgJ15cXGR7NH0nKSksIC0xKSBBUyBkLAogICAgY291bnQoKSwKICAgIHJvdW5kKGF2ZyhwcmljZSksIDIpLAogICAgYmFyKGF2ZyhwcmljZSksIDAsIDUwLCAxMDApLAogICAgYW55KGRpc2hfbmFtZSkKRlJPTSBtZW51X2l0ZW1fZGVub3JtCldIRVJFIChtZW51X2N1cnJlbmN5IElOICgnRG9sbGFycycsICcnKSkgQU5EIChkID4gMCkgQU5EIChkIDwgMjAyMikgQU5EIChkaXNoX25hbWUgSUxJS0UgJyVjYXZpYXIlJykKR1JPVVAgQlkgZApPUkRFUiBCWSBkIEFTQw==). diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index add7ca94683..7b4004e86df 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -3,7 +3,7 @@ toc_priority: 21 toc_title: Menus --- -# New York Public Library "What's on the Menu?" Dataset +# New York Public Library "What's on the Menu?" Dataset {#menus-dataset} Набор данных создан Нью-Йоркской публичной библиотекой. Он содержит исторические данные о меню отелей, ресторанов и кафе с блюдами, а также их ценами. @@ -13,7 +13,7 @@ toc_title: Menus Данные взяты из архива библиотеки, и они могут быть неполными и сложными для статистического анализа. Тем не менее это тоже очень интересно. В наборе всего 1,3 миллиона записей о блюдах в меню — очень небольшой объем данных для ClickHouse, но это все равно хороший пример. -## Загрузите набор данных +## Загрузите набор данных {#download-dataset} Выполните команду: @@ -24,7 +24,7 @@ wget https://s3.amazonaws.com/menusdata.nypl.org/gzips/2021_08_01_07_01_17_data. Замените ссылку на актуальную ссылку с http://menus.nypl.org/data, если это необходимо. Размер загрузки составляет около 35 МБ. -## Распакуйте набор данных +## Распакуйте набор данных {#unpack-dataset} ```bash tar xvf 2021_08_01_07_01_17_data.tgz @@ -38,7 +38,7 @@ The data is normalized consisted of four tables: - MenuPage — information about the pages in the menus, because every page belongs to some menu. - MenuItem — an item of the menu. A dish along with its price on some menu page: links to dish and menu page. -## Создайте таблицы +## Создайте таблицы {#create-tables} ```sql CREATE TABLE dish @@ -105,7 +105,7 @@ CREATE TABLE menu_item We use [Decimal](../../sql-reference/data-types/decimal.md) data type to store prices. Everything else is quite straightforward. -## Import Data +## Import Data {#import-data} Upload data into ClickHouse: @@ -125,7 +125,7 @@ We disable `input_format_null_as_default` as our data does not have NULLs. Other The setting `--date_time_input_format best_effort` allows to parse `DateTime` fields in wide variety of formats. For example, ISO-8601 without seconds like '2000-01-01 01:02' will be recognized. Without this setting only fixed [DateTime](../../sql-reference/data-types/datetime.md) format is allowed. -## Denormalize the Data +## Denormalize the Data {#denormalize-data} Data is presented in multiple tables in normalized form. It means you have to perform JOINs if you want to query, e.g. dish names from menu items. For typical analytical tasks it is way more efficient to deal with pre-JOINed data to avoid doing JOIN every time. It is called "denormalized" data. @@ -177,8 +177,7 @@ FROM menu_item JOIN menu ON menu_page.menu_id = menu.id; ``` - -## Validate the Data +## Validate the Data {#validate-data} Запрос: @@ -194,9 +193,9 @@ SELECT count() FROM menu_item_denorm; └─────────┘ ``` -## Примеры +## Примеры запросов {#run-queries} -### Усредненные исторические цены на блюда +### Усредненные исторические цены на блюда {#query-averaged-historical-prices} Запрос: @@ -238,7 +237,7 @@ ORDER BY d ASC; Просто не принимайте это всерьез. -### Цены на бургеры +### Цены на бургеры {#query-burger-prices} Запрос: @@ -275,7 +274,7 @@ ORDER BY d ASC; └──────┴─────────┴──────────────────────┴───────────────────────────────────────┘ ``` -### Водка +### Водка {#query-vodka} Запрос: @@ -309,7 +308,7 @@ ORDER BY d ASC; To get vodka we have to write `ILIKE '%vodka%'` and this definitely makes a statement. -### Икра +### Икра {#query-caviar} Давайте выведем цены на икру. Также давайте выведем название любого блюда с икрой. @@ -352,6 +351,6 @@ ORDER BY d ASC; По крайней мере, есть икра с водкой. Очень мило. -### Test it in Playground +### Test it in Playground {#playground} The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICByb3VuZCh0b1VJbnQzMk9yWmVybyhleHRyYWN0KG1lbnVfZGF0ZSwgJ15cXGR7NH0nKSksIC0xKSBBUyBkLAogICAgY291bnQoKSwKICAgIHJvdW5kKGF2ZyhwcmljZSksIDIpLAogICAgYmFyKGF2ZyhwcmljZSksIDAsIDUwLCAxMDApLAogICAgYW55KGRpc2hfbmFtZSkKRlJPTSBtZW51X2l0ZW1fZGVub3JtCldIRVJFIChtZW51X2N1cnJlbmN5IElOICgnRG9sbGFycycsICcnKSkgQU5EIChkID4gMCkgQU5EIChkIDwgMjAyMikgQU5EIChkaXNoX25hbWUgSUxJS0UgJyVjYXZpYXIlJykKR1JPVVAgQlkgZApPUkRFUiBCWSBkIEFTQw==). From 327c3aa2fa6ffc88ea7cd5a61f907a2911bdfb81 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 16 Sep 2021 18:01:14 +0300 Subject: [PATCH 067/317] Update translation --- .../getting-started/example-datasets/menus.md | 20 ++++---- .../getting-started/example-datasets/menus.md | 48 +++++++++++-------- 2 files changed, 37 insertions(+), 31 deletions(-) diff --git a/docs/en/getting-started/example-datasets/menus.md b/docs/en/getting-started/example-datasets/menus.md index 6cf4a5ecd4f..9ee017ba86c 100644 --- a/docs/en/getting-started/example-datasets/menus.md +++ b/docs/en/getting-started/example-datasets/menus.md @@ -33,13 +33,15 @@ tar xvf 2021_08_01_07_01_17_data.tgz Uncompressed size is about 150 MB. The data is normalized consisted of four tables: -- Menu — information about menus: the name of the restaurant, the date when menu was seen, etc. -- Dish — information about dishes: the name of the dish along with some characteristic. -- MenuPage — information about the pages in the menus, because every page belongs to some menu. -- MenuItem — an item of the menu. A dish along with its price on some menu page: links to dish and menu page. +- `Menu` — Information about menus: the name of the restaurant, the date when menu was seen, etc. +- `Dish` — Information about dishes: the name of the dish along with some characteristic. +- `MenuPage` — Information about the pages in the menus, because every page belongs to some menu. +- `MenuItem` — An item of the menu. A dish along with its price on some menu page: links to dish and menu page. ## Create the Tables {#create-tables} +We use [Decimal](../../sql-reference/data-types/decimal.md) data type to store prices. + ```sql CREATE TABLE dish ( @@ -103,11 +105,9 @@ CREATE TABLE menu_item ) ENGINE = MergeTree ORDER BY id; ``` -We use [Decimal](../../sql-reference/data-types/decimal.md) data type to store prices. Everything else is quite straightforward. - ## Import the Data {#import-data} -Upload data into ClickHouse: +Upload data into ClickHouse, run: ```bash clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO dish FORMAT CSVWithNames" < Dish.csv @@ -122,14 +122,14 @@ We disable `format_csv_allow_single_quotes` as only double quotes are used for d We disable [input_format_null_as_default](../../operations/settings/settings.mdsettings-input-format-null-as-default) as our data does not have [NULL](../../sql-reference/syntax.md#null-literal). Otherwise ClickHouse will try to parse `\N` sequences and can be confused with `\` in data. -The setting [--date_time_input_format best_effort](../../operations/settings/settings.md#settings-date_time_input_format) allows to parse [DateTime](../../sql-reference/data-types/datetime.md) fields in wide variety of formats. For example, ISO-8601 without seconds like '2000-01-01 01:02' will be recognized. Without this setting only fixed DateTime format is allowed. +The setting [date_time_input_format best_effort](../../operations/settings/settings.md#settings-date_time_input_format) allows to parse [DateTime](../../sql-reference/data-types/datetime.md) fields in wide variety of formats. For example, ISO-8601 without seconds like '2000-01-01 01:02' will be recognized. Without this setting only fixed DateTime format is allowed. ## Denormalize the Data {#denormalize-data} -Data is presented in multiple tables in normalized form. It means you have to perform [JOIN](../../sql-reference/statements/select/join.md#select-join) if you want to query, e.g. dish names from menu items. +Data is presented in multiple tables in [normalized form](https://en.wikipedia.org/wiki/Database_normalization#Normal_forms). It means you have to perform [JOIN](../../sql-reference/statements/select/join.md#select-join) if you want to query, e.g. dish names from menu items. For typical analytical tasks it is way more efficient to deal with pre-JOINed data to avoid doing JOIN every time. It is called "denormalized" data. -We will create a table that will contain all the data JOINed together: +We will create a table `menu_item_denorm` where will contain all the data JOINed together: ```sql CREATE TABLE menu_item_denorm diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index 7b4004e86df..27a7c3f5f8e 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -21,7 +21,7 @@ toc_title: Menus wget https://s3.amazonaws.com/menusdata.nypl.org/gzips/2021_08_01_07_01_17_data.tgz ``` -Замените ссылку на актуальную ссылку с http://menus.nypl.org/data, если это необходимо. +При необходимости, замените ссылку на актуальную ссылку с http://menus.nypl.org/data. Размер загрузки составляет около 35 МБ. ## Распакуйте набор данных {#unpack-dataset} @@ -32,14 +32,16 @@ tar xvf 2021_08_01_07_01_17_data.tgz Размер распакованных данных составляет около 150 МБ. -The data is normalized consisted of four tables: -- Menu — information about menus: the name of the restaurant, the date when menu was seen, etc. -- Dish — information about dishes: the name of the dish along with some characteristic. -- MenuPage — information about the pages in the menus, because every page belongs to some menu. -- MenuItem — an item of the menu. A dish along with its price on some menu page: links to dish and menu page. +Данные нормализованы и состоят из четырех таблиц: +- `Menu` — информация о меню: название ресторана, дата, когда было просмотрено меню, и т.д. +- `Dish` — информация о блюдах: название блюда вместе с некоторыми характеристиками. +- `MenuPage` — информация о страницах в меню, потому что каждая страница принадлежит какому-либо меню. +- `MenuItem` — один из пунктов меню. Блюдо вместе с его ценой на какой-либо странице меню: ссылки на блюдо и страницу меню. ## Создайте таблицы {#create-tables} +Для хранения цен используется тип данных [Decimal](../../sql-reference/data-types/decimal.md). + ```sql CREATE TABLE dish ( @@ -103,11 +105,9 @@ CREATE TABLE menu_item ) ENGINE = MergeTree ORDER BY id; ``` -We use [Decimal](../../sql-reference/data-types/decimal.md) data type to store prices. Everything else is quite straightforward. +## Импортируйте данные {#import-data} -## Import Data {#import-data} - -Upload data into ClickHouse: +Импортируйте данные в ClickHouse, выполните: ```bash clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO dish FORMAT CSVWithNames" < Dish.csv @@ -116,21 +116,27 @@ clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_defa clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --date_time_input_format best_effort --query "INSERT INTO menu_item FORMAT CSVWithNames" < MenuItem.csv ``` -We use [CSVWithNames](../../interfaces/formats.md#csvwithnames) format as the data is represented by CSV with header. +Поскольку данные представлены в формате CSV с заголовком указывается параметр [CSVWithNames](../../interfaces/formats.md#csvwithnames). We disable `format_csv_allow_single_quotes` as only double quotes are used for data fields and single quotes can be inside the values and should not confuse the CSV parser. -We disable `input_format_null_as_default` as our data does not have NULLs. Otherwise ClickHouse will try to parse `\N` sequences and can be confused with `\` in data. +Отключите `format_csv_allow_single_quotes`, так как для данных используются только двойные кавычки, а одинарные кавычки могут находиться внутри значений и не должны сбивать с толку CSV-парсер. -The setting `--date_time_input_format best_effort` allows to parse `DateTime` fields in wide variety of formats. For example, ISO-8601 without seconds like '2000-01-01 01:02' will be recognized. Without this setting only fixed [DateTime](../../sql-reference/data-types/datetime.md) format is allowed. +Отключите [input_format_null_as_default](../../operations/settings/settings.mdsettings-input-format-null-as-default), поскольку в данных нет значений [NULL](../../sql-reference/syntax.md#null-literal). +В противном случае ClickHouse попытается проанализировать последовательности `\N` и может перепутать с `\` в данных. -## Denormalize the Data {#denormalize-data} +Настройка [date_time_input_format best_effort](../../operations/settings/settings.md#settings-date_time_input_format) позволяет анализировать поля [DateTime](../../sql-reference/data-types/datetime.md) в самых разных форматах. К примеру, будет распознан ISO-8601 без секунд: '2000-01-01 01:02'. Без этой настройки допускается только фиксированный формат даты и времени. -Data is presented in multiple tables in normalized form. It means you have to perform JOINs if you want to query, e.g. dish names from menu items. -For typical analytical tasks it is way more efficient to deal with pre-JOINed data to avoid doing JOIN every time. It is called "denormalized" data. +## Денормализуйте данные {#denormalize-data} -We will create a table that will contain all the data JOINed together: +Данные представлены в нескольких таблицах в [нормализованном виде](https://ru.wikipedia.org/wiki/%D0%9D%D0%BE%D1%80%D0%BC%D0%B0%D0%BB%D1%8C%D0%BD%D0%B0%D1%8F_%D1%84%D0%BE%D1%80%D0%BC%D0%B0). + +Это означает, что вам нужно использовать условие объединения [JOIN](../../sql-reference/statements/select/join.md#select-join), если вы хотите получить, например, названия блюд из пунктов меню. + +Для типовых аналитических задач гораздо эффективнее работать с предварительно объединенными данными, чтобы не использовать JOIN каждый раз. Такие данные называются "денормализованными". + +Создайте таблицу `menu_item_denorm`, которая будет содержать все данные, объединенные вместе: ```sql CREATE TABLE menu_item_denorm @@ -177,7 +183,7 @@ FROM menu_item JOIN menu ON menu_page.menu_id = menu.id; ``` -## Validate the Data {#validate-data} +## Проверьте загруженные данные {#validate-data} Запрос: @@ -306,11 +312,11 @@ ORDER BY d ASC; └──────┴─────────┴──────────────────────┴─────────────────────────────┘ ``` -To get vodka we have to write `ILIKE '%vodka%'` and this definitely makes a statement. +Чтобы получить водку, мы должны написать `ILIKE '%vodka%'`, и это хорошая идея. ### Икра {#query-caviar} -Давайте выведем цены на икру. Также давайте выведем название любого блюда с икрой. +Посмотрите цены на икру. Получите название любого блюда с икрой. Запрос: @@ -353,4 +359,4 @@ ORDER BY d ASC; ### Test it in Playground {#playground} -The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICByb3VuZCh0b1VJbnQzMk9yWmVybyhleHRyYWN0KG1lbnVfZGF0ZSwgJ15cXGR7NH0nKSksIC0xKSBBUyBkLAogICAgY291bnQoKSwKICAgIHJvdW5kKGF2ZyhwcmljZSksIDIpLAogICAgYmFyKGF2ZyhwcmljZSksIDAsIDUwLCAxMDApLAogICAgYW55KGRpc2hfbmFtZSkKRlJPTSBtZW51X2l0ZW1fZGVub3JtCldIRVJFIChtZW51X2N1cnJlbmN5IElOICgnRG9sbGFycycsICcnKSkgQU5EIChkID4gMCkgQU5EIChkIDwgMjAyMikgQU5EIChkaXNoX25hbWUgSUxJS0UgJyVjYXZpYXIlJykKR1JPVVAgQlkgZApPUkRFUiBCWSBkIEFTQw==). +Этот набор данных доступен в интерактивном ресурсе [Online Playground](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICByb3VuZCh0b1VJbnQzMk9yWmVybyhleHRyYWN0KG1lbnVfZGF0ZSwgJ15cXGR7NH0nKSksIC0xKSBBUyBkLAogICAgY291bnQoKSwKICAgIHJvdW5kKGF2ZyhwcmljZSksIDIpLAogICAgYmFyKGF2ZyhwcmljZSksIDAsIDUwLCAxMDApLAogICAgYW55KGRpc2hfbmFtZSkKRlJPTSBtZW51X2l0ZW1fZGVub3JtCldIRVJFIChtZW51X2N1cnJlbmN5IElOICgnRG9sbGFycycsICcnKSkgQU5EIChkID4gMCkgQU5EIChkIDwgMjAyMikgQU5EIChkaXNoX25hbWUgSUxJS0UgJyVjYXZpYXIlJykKR1JPVVAgQlkgZApPUkRFUiBCWSBkIEFTQw==). From 4d788cee8c4fe775807a073223bc5454302a2898 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 16 Sep 2021 18:03:35 +0300 Subject: [PATCH 068/317] Fix link. --- docs/en/getting-started/example-datasets/menus.md | 2 +- docs/ru/getting-started/example-datasets/menus.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/menus.md b/docs/en/getting-started/example-datasets/menus.md index 9ee017ba86c..ea25ab1e6a7 100644 --- a/docs/en/getting-started/example-datasets/menus.md +++ b/docs/en/getting-started/example-datasets/menus.md @@ -120,7 +120,7 @@ We use [CSVWithNames](../../interfaces/formats.md#csvwithnames) format as the da We disable `format_csv_allow_single_quotes` as only double quotes are used for data fields and single quotes can be inside the values and should not confuse the CSV parser. -We disable [input_format_null_as_default](../../operations/settings/settings.mdsettings-input-format-null-as-default) as our data does not have [NULL](../../sql-reference/syntax.md#null-literal). Otherwise ClickHouse will try to parse `\N` sequences and can be confused with `\` in data. +We disable [input_format_null_as_default](../../operations/settings/settings.md#settings-input-format-null-as-default) as our data does not have [NULL](../../sql-reference/syntax.md#null-literal). Otherwise ClickHouse will try to parse `\N` sequences and can be confused with `\` in data. The setting [date_time_input_format best_effort](../../operations/settings/settings.md#settings-date_time_input_format) allows to parse [DateTime](../../sql-reference/data-types/datetime.md) fields in wide variety of formats. For example, ISO-8601 without seconds like '2000-01-01 01:02' will be recognized. Without this setting only fixed DateTime format is allowed. diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index 27a7c3f5f8e..cc92622bb60 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -122,7 +122,7 @@ We disable `format_csv_allow_single_quotes` as only double quotes are used for d Отключите `format_csv_allow_single_quotes`, так как для данных используются только двойные кавычки, а одинарные кавычки могут находиться внутри значений и не должны сбивать с толку CSV-парсер. -Отключите [input_format_null_as_default](../../operations/settings/settings.mdsettings-input-format-null-as-default), поскольку в данных нет значений [NULL](../../sql-reference/syntax.md#null-literal). +Отключите [input_format_null_as_default](../../operations/settings/settings.md#settings-input-format-null-as-default), поскольку в данных нет значений [NULL](../../sql-reference/syntax.md#null-literal). В противном случае ClickHouse попытается проанализировать последовательности `\N` и может перепутать с `\` в данных. From e4683f016450d8d9983ece2eb55d41ab6d2f11cf Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 16 Sep 2021 18:14:50 +0300 Subject: [PATCH 069/317] Section name fixed. --- docs/ru/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index cc92622bb60..12e908f82a0 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -357,6 +357,6 @@ ORDER BY d ASC; По крайней мере, есть икра с водкой. Очень мило. -### Test it in Playground {#playground} +### Online Playground {#playground} Этот набор данных доступен в интерактивном ресурсе [Online Playground](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICByb3VuZCh0b1VJbnQzMk9yWmVybyhleHRyYWN0KG1lbnVfZGF0ZSwgJ15cXGR7NH0nKSksIC0xKSBBUyBkLAogICAgY291bnQoKSwKICAgIHJvdW5kKGF2ZyhwcmljZSksIDIpLAogICAgYmFyKGF2ZyhwcmljZSksIDAsIDUwLCAxMDApLAogICAgYW55KGRpc2hfbmFtZSkKRlJPTSBtZW51X2l0ZW1fZGVub3JtCldIRVJFIChtZW51X2N1cnJlbmN5IElOICgnRG9sbGFycycsICcnKSkgQU5EIChkID4gMCkgQU5EIChkIDwgMjAyMikgQU5EIChkaXNoX25hbWUgSUxJS0UgJyVjYXZpYXIlJykKR1JPVVAgQlkgZApPUkRFUiBCWSBkIEFTQw==). From b1f18eff945e2eeb3b8f3a0dcd3f9094c07ef900 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Thu, 16 Sep 2021 18:35:25 +0300 Subject: [PATCH 070/317] 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 071/317] 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 072/317] 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 073/317] 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 074/317] 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 075/317] 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 076/317] 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 077/317] 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 9bed03faf5f968b1865d9e7a7a875ffe5190491e Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sun, 19 Sep 2021 16:14:42 +0300 Subject: [PATCH 078/317] Update docs/ru/getting-started/example-datasets/menus.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index 12e908f82a0..e9677a95978 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -1,6 +1,6 @@ --- toc_priority: 21 -toc_title: Menus +toc_title: Меню --- # New York Public Library "What's on the Menu?" Dataset {#menus-dataset} From b15571694a900f80db919542e4fd7ac8ad7dbbda Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sun, 19 Sep 2021 16:14:49 +0300 Subject: [PATCH 079/317] Update docs/ru/getting-started/example-datasets/menus.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index e9677a95978..1b57b13ee42 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -3,7 +3,7 @@ toc_priority: 21 toc_title: Меню --- -# New York Public Library "What's on the Menu?" Dataset {#menus-dataset} +# Набор данных публичной библиотеки Нью-Йорка "Что в меню?" {#menus-dataset} Набор данных создан Нью-Йоркской публичной библиотекой. Он содержит исторические данные о меню отелей, ресторанов и кафе с блюдами, а также их ценами. From 2ab40907d3ce5279fd52dc8767fbc800ea2b1749 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sun, 19 Sep 2021 16:19:05 +0300 Subject: [PATCH 080/317] Update docs/ru/getting-started/example-datasets/menus.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index 1b57b13ee42..fc869793e6e 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -10,7 +10,7 @@ toc_title: Меню Источник: http://menus.nypl.org/data Эти данные находятся в открытом доступе. -Данные взяты из архива библиотеки, и они могут быть неполными и сложными для статистического анализа. Тем не менее это тоже очень интересно. +Данные взяты из архива библиотеки, и они могут быть неполными и сложными для статистического анализа. Тем не менее, это тоже очень интересно. В наборе всего 1,3 миллиона записей о блюдах в меню — очень небольшой объем данных для ClickHouse, но это все равно хороший пример. ## Загрузите набор данных {#download-dataset} From 2c9499c2ff8372d7ac4018fc603d464138ca269c Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sun, 19 Sep 2021 16:19:11 +0300 Subject: [PATCH 081/317] Update docs/ru/getting-started/example-datasets/menus.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index fc869793e6e..fe0efabf343 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -21,7 +21,7 @@ toc_title: Меню wget https://s3.amazonaws.com/menusdata.nypl.org/gzips/2021_08_01_07_01_17_data.tgz ``` -При необходимости, замените ссылку на актуальную ссылку с http://menus.nypl.org/data. +При необходимости замените ссылку на актуальную ссылку с http://menus.nypl.org/data. Размер загрузки составляет около 35 МБ. ## Распакуйте набор данных {#unpack-dataset} From 310f82504deb523ebbe8120b4ca347cf4404d1f5 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sun, 19 Sep 2021 16:19:17 +0300 Subject: [PATCH 082/317] Update docs/ru/getting-started/example-datasets/menus.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index fe0efabf343..f698e0e6084 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -22,7 +22,7 @@ wget https://s3.amazonaws.com/menusdata.nypl.org/gzips/2021_08_01_07_01_17_data. ``` При необходимости замените ссылку на актуальную ссылку с http://menus.nypl.org/data. -Размер загрузки составляет около 35 МБ. +Размер архива составляет около 35 МБ. ## Распакуйте набор данных {#unpack-dataset} From 3fa4e683a8e8db91ba004e9995ffbdb4eff842e5 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sun, 19 Sep 2021 16:19:23 +0300 Subject: [PATCH 083/317] Update docs/ru/getting-started/example-datasets/menus.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index f698e0e6084..539561d1fbf 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -107,7 +107,7 @@ CREATE TABLE menu_item ## Импортируйте данные {#import-data} -Импортируйте данные в ClickHouse, выполните: +Импортируйте данные в ClickHouse, выполните команды: ```bash clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --query "INSERT INTO dish FORMAT CSVWithNames" < Dish.csv From 41b05b3a49917627d816f7f40557bc8ab8edc828 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sun, 19 Sep 2021 16:19:30 +0300 Subject: [PATCH 084/317] Update docs/ru/getting-started/example-datasets/menus.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index 539561d1fbf..d0a2a20d181 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -116,7 +116,7 @@ clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_defa clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_default 0 --date_time_input_format best_effort --query "INSERT INTO menu_item FORMAT CSVWithNames" < MenuItem.csv ``` -Поскольку данные представлены в формате CSV с заголовком указывается параметр [CSVWithNames](../../interfaces/formats.md#csvwithnames). +Поскольку данные представлены в формате CSV с заголовком, используется формат [CSVWithNames](../../interfaces/formats.md#csvwithnames). We disable `format_csv_allow_single_quotes` as only double quotes are used for data fields and single quotes can be inside the values and should not confuse the CSV parser. From 0919568bb6ec237114595e14bad759e4347aafd6 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sun, 19 Sep 2021 16:21:01 +0300 Subject: [PATCH 085/317] Update docs/ru/getting-started/example-datasets/menus.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/menus.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index d0a2a20d181..46251ac0285 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -118,8 +118,6 @@ clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_defa Поскольку данные представлены в формате CSV с заголовком, используется формат [CSVWithNames](../../interfaces/formats.md#csvwithnames). -We disable `format_csv_allow_single_quotes` as only double quotes are used for data fields and single quotes can be inside the values and should not confuse the CSV parser. - Отключите `format_csv_allow_single_quotes`, так как для данных используются только двойные кавычки, а одинарные кавычки могут находиться внутри значений и не должны сбивать с толку CSV-парсер. Отключите [input_format_null_as_default](../../operations/settings/settings.md#settings-input-format-null-as-default), поскольку в данных нет значений [NULL](../../sql-reference/syntax.md#null-literal). From bb1a8933a69a964dd90d811d5e10265109167e9a Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sun, 19 Sep 2021 16:25:08 +0300 Subject: [PATCH 086/317] Update docs/en/getting-started/example-datasets/menus.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/menus.md b/docs/en/getting-started/example-datasets/menus.md index ea25ab1e6a7..08a745a1d72 100644 --- a/docs/en/getting-started/example-datasets/menus.md +++ b/docs/en/getting-started/example-datasets/menus.md @@ -127,7 +127,7 @@ The setting [date_time_input_format best_effort](../../operations/settings/setti ## Denormalize the Data {#denormalize-data} Data is presented in multiple tables in [normalized form](https://en.wikipedia.org/wiki/Database_normalization#Normal_forms). It means you have to perform [JOIN](../../sql-reference/statements/select/join.md#select-join) if you want to query, e.g. dish names from menu items. -For typical analytical tasks it is way more efficient to deal with pre-JOINed data to avoid doing JOIN every time. It is called "denormalized" data. +For typical analytical tasks it is way more efficient to deal with pre-JOINed data to avoid doing `JOIN` every time. It is called "denormalized" data. We will create a table `menu_item_denorm` where will contain all the data JOINed together: From f22ebe847aca0b6bbcfd36dbd9143337ee6307df Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sun, 19 Sep 2021 16:25:14 +0300 Subject: [PATCH 087/317] Update docs/ru/getting-started/example-datasets/menus.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index 46251ac0285..b7e788e8629 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -355,6 +355,6 @@ ORDER BY d ASC; По крайней мере, есть икра с водкой. Очень мило. -### Online Playground {#playground} +## Online Playground {#playground} Этот набор данных доступен в интерактивном ресурсе [Online Playground](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICByb3VuZCh0b1VJbnQzMk9yWmVybyhleHRyYWN0KG1lbnVfZGF0ZSwgJ15cXGR7NH0nKSksIC0xKSBBUyBkLAogICAgY291bnQoKSwKICAgIHJvdW5kKGF2ZyhwcmljZSksIDIpLAogICAgYmFyKGF2ZyhwcmljZSksIDAsIDUwLCAxMDApLAogICAgYW55KGRpc2hfbmFtZSkKRlJPTSBtZW51X2l0ZW1fZGVub3JtCldIRVJFIChtZW51X2N1cnJlbmN5IElOICgnRG9sbGFycycsICcnKSkgQU5EIChkID4gMCkgQU5EIChkIDwgMjAyMikgQU5EIChkaXNoX25hbWUgSUxJS0UgJyVjYXZpYXIlJykKR1JPVVAgQlkgZApPUkRFUiBCWSBkIEFTQw==). From 391983d7ef7d19b928d9751629788f174220361a Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sun, 19 Sep 2021 16:25:32 +0300 Subject: [PATCH 088/317] Update docs/en/getting-started/example-datasets/menus.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/menus.md b/docs/en/getting-started/example-datasets/menus.md index 08a745a1d72..14825fcaa7b 100644 --- a/docs/en/getting-started/example-datasets/menus.md +++ b/docs/en/getting-started/example-datasets/menus.md @@ -350,6 +350,6 @@ Result: At least they have caviar with vodka. Very nice. -### Test it in Playground {#playground} +## Online Playground {#playground} The data is uploaded to ClickHouse Playground, [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUCiAgICByb3VuZCh0b1VJbnQzMk9yWmVybyhleHRyYWN0KG1lbnVfZGF0ZSwgJ15cXGR7NH0nKSksIC0xKSBBUyBkLAogICAgY291bnQoKSwKICAgIHJvdW5kKGF2ZyhwcmljZSksIDIpLAogICAgYmFyKGF2ZyhwcmljZSksIDAsIDUwLCAxMDApLAogICAgYW55KGRpc2hfbmFtZSkKRlJPTSBtZW51X2l0ZW1fZGVub3JtCldIRVJFIChtZW51X2N1cnJlbmN5IElOICgnRG9sbGFycycsICcnKSkgQU5EIChkID4gMCkgQU5EIChkIDwgMjAyMikgQU5EIChkaXNoX25hbWUgSUxJS0UgJyVjYXZpYXIlJykKR1JPVVAgQlkgZApPUkRFUiBCWSBkIEFTQw==). From 554fd69ccd3596a108a0f59186afc0e4535be01b Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sun, 19 Sep 2021 16:25:56 +0300 Subject: [PATCH 089/317] Update docs/ru/getting-started/example-datasets/menus.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/example-datasets/menus.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/menus.md b/docs/ru/getting-started/example-datasets/menus.md index b7e788e8629..5b549975b8f 100644 --- a/docs/ru/getting-started/example-datasets/menus.md +++ b/docs/ru/getting-started/example-datasets/menus.md @@ -132,7 +132,7 @@ clickhouse-client --format_csv_allow_single_quotes 0 --input_format_null_as_defa Это означает, что вам нужно использовать условие объединения [JOIN](../../sql-reference/statements/select/join.md#select-join), если вы хотите получить, например, названия блюд из пунктов меню. -Для типовых аналитических задач гораздо эффективнее работать с предварительно объединенными данными, чтобы не использовать JOIN каждый раз. Такие данные называются "денормализованными". +Для типовых аналитических задач гораздо эффективнее работать с предварительно объединенными данными, чтобы не использовать `JOIN` каждый раз. Такие данные называются денормализованными. Создайте таблицу `menu_item_denorm`, которая будет содержать все данные, объединенные вместе: From 651471078b3573f30ad1a132804107177685c9d6 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sun, 19 Sep 2021 17:08:53 +0300 Subject: [PATCH 090/317] Fix link (try 1). --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 8ad450738b7..4deaa96b793 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -45,7 +45,7 @@ Configuration template: - `min_part_size` – The minimum size of a data part. - `min_part_size_ratio` – The ratio of the data part size to the table size. - `method` – Compression method. Acceptable values: `lz4`, `lz4hc`, `zstd`. -- `level` – Compression level. See [Codecs](../../sql-reference/statements/create/table/#create-query-general-purpose-codecs). +- `level` – Compression level. See [Codecs](../../sql-reference/statements/create/table.md#create-query-general-purpose-codecs). You can configure multiple `` sections. From 53e946ec25e08903073705994bc7db855f6cc2ef Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sun, 19 Sep 2021 19:18:38 +0300 Subject: [PATCH 091/317] 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 092/317] 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 093/317] 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 094/317] 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 095/317] 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 096/317] 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 097/317] 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 098/317] 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 099/317] 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 100/317] 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 101/317] 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 102/317] 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 103/317] 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 104/317] 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 8f87d44958a49bf955e4bcc167ff09e65e4857b6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Thu, 2 Sep 2021 14:17:47 +0800 Subject: [PATCH 105/317] bloom filter skipping index support Map with key of String type --- .../MergeTree/MergeTreeIndexFullText.cpp | 65 +++++++++++++++++-- 1 file changed, 58 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 1c71d77b334..645a1d5d267 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -15,6 +15,8 @@ #include #include #include +#include +#include #include @@ -156,14 +158,42 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, for (size_t col = 0; col < index_columns.size(); ++col) { const auto & column = block.getByName(index_columns[col]).column; - for (size_t i = 0; i < rows_read; ++i) + + if (column->getDataType() == TypeIndex::Map) { - auto ref = column->getDataAt(*pos + i); - columnToBloomFilter(ref.data, ref.size, token_extractor, granule->bloom_filters[col]); + //update for the key of Map type + auto * column_map = assert_cast(const_cast(column.get())); + auto & column_array = assert_cast(column_map->getNestedColumn()); + auto & column_tuple = assert_cast(column_array.getData()); + auto & column_key = assert_cast(column_tuple.getColumn(0)); + + for (size_t i = 0; i < rows_read; ++i) + { + size_t element_start_row = *pos !=0 ? column_array.getOffsets()[*pos-1] : 0; + size_t elements_size = column_array.getOffsets()[*pos] - element_start_row; + + for (size_t row_num = 0; row_num < elements_size; row_num++) + { + auto ref = column_key.getDataAt(element_start_row+row_num); + columnToBloomFilter(ref.data, ref.size, token_extractor, granule->bloom_filters[col]); + } + + *pos += 1; + } } + else + { + for (size_t i = 0; i < rows_read; ++i) + { + auto ref = column->getDataAt(*pos + i); + columnToBloomFilter(ref.data, ref.size, token_extractor, granule->bloom_filters[col]); + } + + *pos += rows_read; + } + } granule->has_elems = true; - *pos += rows_read; } @@ -327,7 +357,15 @@ bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx bool MergeTreeConditionFullText::getKey(const ASTPtr & node, size_t & key_column_num) { - auto it = std::find(index_columns.begin(), index_columns.end(), node->getColumnName()); + String column_name = node->getColumnName(); + + //try to get map column name in arrayElement function + if (const auto * func = typeid_cast(node.get())) + { + column_name = assert_cast(func->arguments.get()->children[0].get())->name(); + } + + auto it = std::find(index_columns.begin(), index_columns.end(), column_name); if (it == index_columns.end()) return false; @@ -372,6 +410,12 @@ bool MergeTreeConditionFullText::atomFromAST( { return false; } + + //try to parse arrayElement function + if (const auto * func = typeid_cast(args[0].get())) + { + const_value = assert_cast(func->arguments.get()->children[1].get())->name(); + } if (key_arg_pos == 1 && (func_name != "equals" && func_name != "notEquals")) return false; @@ -837,10 +881,17 @@ MergeTreeIndexPtr bloomFilterIndexCreator( void bloomFilterIndexValidator(const IndexDescription & index, bool /*attach*/) { + auto map_validator = [](DataTypeMap & map_type){ + if (*map_type.getKeyType().get() == TypeIndex::String|| + *map_type.getKeyType().get() == TypeIndex::FixedString) + }; + for (const auto & data_type : index.data_types) { - if (data_type->getTypeId() != TypeIndex::String && data_type->getTypeId() != TypeIndex::FixedString) - throw Exception("Bloom filter index can be used only with `String` or `FixedString` column.", ErrorCodes::INCORRECT_QUERY); + if (data_type->getTypeId() != TypeIndex::String + && data_type->getTypeId() != TypeIndex::FixedString + && data_type->getTypeId() != TypeIndex::Map) + throw Exception("Bloom filter index can be used only with `String`,`FixedString` or `Map` column.", ErrorCodes::INCORRECT_QUERY); } if (index.type == NgramTokenExtractor::getName()) From a3e657778abfab2d3ba5ea6fb71c2520f3383a77 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Thu, 2 Sep 2021 15:27:54 +0800 Subject: [PATCH 106/317] bloom filter skipping index support Map with key of String type --- .../MergeTree/MergeTreeIndexFullText.cpp | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 645a1d5d267..ae6885ef9f0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -412,9 +412,9 @@ bool MergeTreeConditionFullText::atomFromAST( } //try to parse arrayElement function - if (const auto * func = typeid_cast(args[0].get())) + if (const auto * map_func = typeid_cast(args[0].get())) { - const_value = assert_cast(func->arguments.get()->children[1].get())->name(); + const_value = assert_cast(map_func->arguments.get()->children[1].get())->name(); } if (key_arg_pos == 1 && (func_name != "equals" && func_name != "notEquals")) @@ -881,17 +881,25 @@ MergeTreeIndexPtr bloomFilterIndexCreator( void bloomFilterIndexValidator(const IndexDescription & index, bool /*attach*/) { - auto map_validator = [](DataTypeMap & map_type){ - if (*map_type.getKeyType().get() == TypeIndex::String|| - *map_type.getKeyType().get() == TypeIndex::FixedString) + auto validate_map_type = [](DataTypePtr data_type){ + if (data_type->getTypeId() != TypeIndex::Map) + return false; + + DataTypeMap * map_type = assert_cast(const_cast(data_type.get())); + + if (map_type->getKeyType()->getTypeId() != TypeIndex::String + && map_type->getKeyType()->getTypeId() != TypeIndex::FixedString) + return false; + + return true; }; for (const auto & data_type : index.data_types) { if (data_type->getTypeId() != TypeIndex::String && data_type->getTypeId() != TypeIndex::FixedString - && data_type->getTypeId() != TypeIndex::Map) - throw Exception("Bloom filter index can be used only with `String`,`FixedString` or `Map` column.", ErrorCodes::INCORRECT_QUERY); + && !validate_map_type(data_type)) + throw Exception("Bloom filter index can be used only with `String`,`FixedString` or `Map` with key of String or fixedString type.", ErrorCodes::INCORRECT_QUERY); } if (index.type == NgramTokenExtractor::getName()) From 32df36abb414b0d3dec0f8516300c015aec845ad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Thu, 2 Sep 2021 17:05:24 +0800 Subject: [PATCH 107/317] add tests --- .../0_stateless/02000_bloom_filter_map.reference | 2 ++ .../0_stateless/02000_bloom_filter_map.sql | 16 ++++++++++++++++ 2 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/02000_bloom_filter_map.reference create mode 100644 tests/queries/0_stateless/02000_bloom_filter_map.sql diff --git a/tests/queries/0_stateless/02000_bloom_filter_map.reference b/tests/queries/0_stateless/02000_bloom_filter_map.reference new file mode 100644 index 00000000000..0fb260e569f --- /dev/null +++ b/tests/queries/0_stateless/02000_bloom_filter_map.reference @@ -0,0 +1,2 @@ +3 {'K3':'V3'} +2 {'K2':'V2'} diff --git a/tests/queries/0_stateless/02000_bloom_filter_map.sql b/tests/queries/0_stateless/02000_bloom_filter_map.sql new file mode 100644 index 00000000000..4b2a86b3682 --- /dev/null +++ b/tests/queries/0_stateless/02000_bloom_filter_map.sql @@ -0,0 +1,16 @@ +CREATE DATABASE IF NOT EXISTS test; + +DROP TABLE IF EXISTS test.bf_tokenbf_map_test; +DROP TABLE IF EXISTS test.bf_ngram_map_test; + +CREATE TABLE test.bf_tokenbf_map_test (row_id UInt32, map Map(String, String), INDEX map_tokenbf map TYPE tokenbf_v1(256,2,0) GRANULARITY 1) Engine=MergeTree() ORDER BY row_id settings index_granularity = 2; +CREATE TABLE test.bf_ngram_map_test (row_id UInt32, map Map(String, String), INDEX map_tokenbf map TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1) Engine=MergeTree() ORDER BY row_id settings index_granularity = 2; + +INSERT INTO test.bf_tokenbf_map_test VALUES (1, {'K1':'V1'}),(2,{'K2':'V2'}),(3,{'K3':'V3'}),(4,{'K4':'V4'}); +INSERT INTO test.bf_ngram_map_test VALUES (1, {'K1':'V1'}),(2,{'K2':'V2'}),(3,{'K3':'V3'}),(4,{'K4':'V4'}); + +SELECT * FROM test.bf_tokenbf_map_test WHERE map['K3']='V3'; +SELECT * FROM test.bf_tokenbf_map_test WHERE map['K2']='V2'; + +DROP TABLE test.bf_tokenbf_map_test; +DROP TABLE test.bf_ngram_map_test; \ No newline at end of file From e84f1000fb07b3048b55bf2a63119e0377fa7c4a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Mon, 6 Sep 2021 16:36:34 +0800 Subject: [PATCH 108/317] optimze code to make sure only map function will be processed --- CMakeLists.txt | 642 ------------------ .../MergeTree/MergeTreeIndexFullText.cpp | 6 +- 2 files changed, 4 insertions(+), 644 deletions(-) delete mode 100644 CMakeLists.txt diff --git a/CMakeLists.txt b/CMakeLists.txt deleted file mode 100644 index 9c8903d853c..00000000000 --- a/CMakeLists.txt +++ /dev/null @@ -1,642 +0,0 @@ -cmake_minimum_required(VERSION 3.3) - -foreach(policy - CMP0023 - CMP0048 # CMake 3.0 - CMP0074 # CMake 3.12 - CMP0077 - CMP0079 - ) - if(POLICY ${policy}) - cmake_policy(SET ${policy} NEW) - endif() -endforeach() - -# set default policy -foreach(default_policy_var_name - # make option() honor normal variables for BUILD_SHARED_LIBS: - # - re2 - # - snappy - CMAKE_POLICY_DEFAULT_CMP0077 - # Google Test from sources uses too old cmake, 2.6.x, and CMP0022 should - # set, to avoid using deprecated LINK_INTERFACE_LIBRARIES(_)? over - # INTERFACE_LINK_LIBRARIES. - CMAKE_POLICY_DEFAULT_CMP0022 - ) - set(${default_policy_var_name} NEW) -endforeach() - -project(ClickHouse) - -# If turned off: e.g. when ENABLE_FOO is ON, but FOO tool was not found, the CMake will continue. -option(FAIL_ON_UNSUPPORTED_OPTIONS_COMBINATION - "Stop/Fail CMake configuration if some ENABLE_XXX option is defined (either ON or OFF) - but is not possible to satisfy" ON) - -if(FAIL_ON_UNSUPPORTED_OPTIONS_COMBINATION) - set(RECONFIGURE_MESSAGE_LEVEL FATAL_ERROR) -else() - set(RECONFIGURE_MESSAGE_LEVEL WARNING) -endif() - -enable_language(C CXX ASM) - -include (cmake/arch.cmake) -include (cmake/target.cmake) -include (cmake/tools.cmake) -include (cmake/analysis.cmake) -include (cmake/git_status.cmake) - -# Ignore export() since we don't use it, -# but it gets broken with a global targets via link_libraries() -macro (export) -endmacro () - -set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${CMAKE_CURRENT_SOURCE_DIR}/cmake/Modules/") -set(CMAKE_EXPORT_COMPILE_COMMANDS 1) # Write compile_commands.json -set(CMAKE_LINK_DEPENDS_NO_SHARED 1) # Do not relink all depended targets on .so -set(CMAKE_CONFIGURATION_TYPES "RelWithDebInfo;Debug;Release;MinSizeRel" CACHE STRING "" FORCE) -set(CMAKE_DEBUG_POSTFIX "d" CACHE STRING "Generate debug library name with a postfix.") # To be consistent with CMakeLists from contrib libs. - -# Enable the ability to organize targets into hierarchies of "folders" for capable GUI-based IDEs. -# For more info see https://cmake.org/cmake/help/latest/prop_gbl/USE_FOLDERS.html -set_property(GLOBAL PROPERTY USE_FOLDERS ON) - -# Check that submodules are present only if source was downloaded with git -if (EXISTS "${CMAKE_CURRENT_SOURCE_DIR}/.git" AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/boost/boost") - message (FATAL_ERROR "Submodules are not initialized. Run\n\tgit submodule update --init --recursive") -endif () - -include (cmake/find/ccache.cmake) - -# Take care to add prlimit in command line before ccache, or else ccache thinks that -# prlimit is compiler, and clang++ is its input file, and refuses to work with -# multiple inputs, e.g in ccache log: -# [2021-03-31T18:06:32.655327 36900] Command line: /usr/bin/ccache prlimit --as=10000000000 --data=5000000000 --cpu=600 /usr/bin/clang++-11 - ...... std=gnu++2a -MD -MT src/CMakeFiles/dbms.dir/Storages/MergeTree/IMergeTreeDataPart.cpp.o -MF src/CMakeFiles/dbms.dir/Storages/MergeTree/IMergeTreeDataPart.cpp.o.d -o src/CMakeFiles/dbms.dir/Storages/MergeTree/IMergeTreeDataPart.cpp.o -c ../src/Storages/MergeTree/IMergeTreeDataPart.cpp -# -# [2021-03-31T18:06:32.656704 36900] Multiple input files: /usr/bin/clang++-11 and ../src/Storages/MergeTree/IMergeTreeDataPart.cpp -# -# Another way would be to use --ccache-skip option before clang++-11 to make -# ccache ignore it. -option(ENABLE_CHECK_HEAVY_BUILDS "Don't allow C++ translation units to compile too long or to take too much memory while compiling." OFF) -if (ENABLE_CHECK_HEAVY_BUILDS) - # set DATA (since RSS does not work since 2.6.x+) to 5G - set (RLIMIT_DATA 5000000000) - # set VIRT (RLIMIT_AS) to 10G (DATA*10) - set (RLIMIT_AS 10000000000) - # set CPU time limit to 1000 seconds - set (RLIMIT_CPU 1000) - - # gcc10/gcc10/clang -fsanitize=memory is too heavy - if (SANITIZE STREQUAL "memory" OR COMPILER_GCC) - set (RLIMIT_DATA 10000000000) # 10G - endif() - - set (CMAKE_CXX_COMPILER_LAUNCHER prlimit --as=${RLIMIT_AS} --data=${RLIMIT_DATA} --cpu=${RLIMIT_CPU} ${CMAKE_CXX_COMPILER_LAUNCHER}) -endif () - -if (NOT CMAKE_BUILD_TYPE OR CMAKE_BUILD_TYPE STREQUAL "None") - set (CMAKE_BUILD_TYPE "RelWithDebInfo") - message (STATUS "CMAKE_BUILD_TYPE is not set, set to default = ${CMAKE_BUILD_TYPE}") -endif () -message (STATUS "CMAKE_BUILD_TYPE: ${CMAKE_BUILD_TYPE}") - -string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC) - -option(USE_STATIC_LIBRARIES "Disable to use shared libraries" ON) -option(MAKE_STATIC_LIBRARIES "Disable to make shared libraries" ${USE_STATIC_LIBRARIES}) - -if (NOT MAKE_STATIC_LIBRARIES) - # DEVELOPER ONLY. - # Faster linking if turned on. - option(SPLIT_SHARED_LIBRARIES "Keep all internal libraries as separate .so files") - - option(CLICKHOUSE_SPLIT_BINARY - "Make several binaries (clickhouse-server, clickhouse-client etc.) instead of one bundled") -endif () - -if (MAKE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) - message(FATAL_ERROR "Defining SPLIT_SHARED_LIBRARIES=1 without MAKE_STATIC_LIBRARIES=0 has no effect.") -endif() - -if (NOT MAKE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) - set(BUILD_SHARED_LIBS 1 CACHE INTERNAL "") -endif () - -if (USE_STATIC_LIBRARIES) - list(REVERSE CMAKE_FIND_LIBRARY_SUFFIXES) -endif () - -option (ENABLE_FUZZING "Fuzzy testing using libfuzzer" OFF) - -if (ENABLE_FUZZING) - # Also set WITH_COVERAGE=1 for better fuzzing process - # By default this is disabled, because fuzzers are built in CI with the clickhouse itself. - # And we don't want to enable coverage for it. - message (STATUS "Fuzzing instrumentation enabled") - set (FUZZER "libfuzzer") -endif() - -# Global libraries -# See: -# - default_libs.cmake -# - sanitize.cmake -add_library(global-libs INTERFACE) - -include (cmake/fuzzer.cmake) -include (cmake/sanitize.cmake) - -if (CMAKE_GENERATOR STREQUAL "Ninja" AND NOT DISABLE_COLORED_BUILD) - # Turn on colored output. https://github.com/ninja-build/ninja/wiki/FAQ - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fdiagnostics-color=always") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fdiagnostics-color=always") -endif () - -include (cmake/check_flags.cmake) -include (cmake/add_warning.cmake) - -if (NOT MSVC) - set (COMMON_WARNING_FLAGS "${COMMON_WARNING_FLAGS} -Wall") # -Werror and many more is also added inside cmake/warnings.cmake -endif () - -if (COMPILER_CLANG) - # clang: warning: argument unused during compilation: '-specs=/usr/share/dpkg/no-pie-compile.specs' [-Wunused-command-line-argument] - set (COMMON_WARNING_FLAGS "${COMMON_WARNING_FLAGS} -Wno-unused-command-line-argument") - # generate ranges for fast "addr2line" search - if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") - set(COMPILER_FLAGS "${COMPILER_FLAGS} -gdwarf-aranges") - endif () - - if (HAS_USE_CTOR_HOMING) - # For more info see https://blog.llvm.org/posts/2021-04-05-constructor-homing-for-debug-info/ - if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO") - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Xclang -fuse-ctor-homing") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Xclang -fuse-ctor-homing") - endif() - endif() -endif () - -# If turned `ON`, assumes the user has either the system GTest library or the bundled one. -option(ENABLE_TESTS "Provide unit_test_dbms target with Google.Test unit tests" ON) -option(ENABLE_EXAMPLES "Build all example programs in 'examples' subdirectories" OFF) - -if (OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64) AND NOT UNBUNDLED AND MAKE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES AND CMAKE_VERSION VERSION_GREATER "3.9.0") - # Only for Linux, x86_64 or aarch64. - option(GLIBC_COMPATIBILITY "Enable compatibility with older glibc libraries." ON) -elseif(GLIBC_COMPATIBILITY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Glibc compatibility cannot be enabled in current configuration") -endif () - -if (NOT CMAKE_VERSION VERSION_GREATER "3.9.0") - message (WARNING "CMake version must be greater than 3.9.0 for production builds.") -endif () - -# Make sure the final executable has symbols exported -set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic") - -find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-13" "llvm-objcopy-12" "llvm-objcopy-11" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy") - -if (NOT OBJCOPY_PATH AND OS_DARWIN) - find_program (BREW_PATH NAMES "brew") - if (BREW_PATH) - execute_process (COMMAND ${BREW_PATH} --prefix llvm ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE OUTPUT_VARIABLE LLVM_PREFIX) - if (LLVM_PREFIX) - find_program (OBJCOPY_PATH NAMES "llvm-objcopy" PATHS "${LLVM_PREFIX}/bin" NO_DEFAULT_PATH) - endif () - if (NOT OBJCOPY_PATH) - execute_process (COMMAND ${BREW_PATH} --prefix binutils ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE OUTPUT_VARIABLE BINUTILS_PREFIX) - if (BINUTILS_PREFIX) - find_program (OBJCOPY_PATH NAMES "objcopy" PATHS "${BINUTILS_PREFIX}/bin" NO_DEFAULT_PATH) - endif () - endif () - endif () -endif () - -if (OBJCOPY_PATH) - message (STATUS "Using objcopy: ${OBJCOPY_PATH}") -else () - message (FATAL_ERROR "Cannot find objcopy.") -endif () - -if (OS_DARWIN) - # The `-all_load` flag forces loading of all symbols from all libraries, - # and leads to multiply-defined symbols. This flag allows force loading - # from a _specific_ library, which is what we need. - set(WHOLE_ARCHIVE -force_load) - # The `-noall_load` flag is the default and now obsolete. - set(NO_WHOLE_ARCHIVE "") -else () - set(WHOLE_ARCHIVE --whole-archive) - set(NO_WHOLE_ARCHIVE --no-whole-archive) -endif () - -# Ignored if `lld` is used -option(ADD_GDB_INDEX_FOR_GOLD "Add .gdb-index to resulting binaries for gold linker.") - -if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") - # Can be lld or ld-lld. - if (LINKER_NAME MATCHES "lld$") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--gdb-index") - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--gdb-index") - message (STATUS "Adding .gdb-index via --gdb-index linker option.") - # we use another tool for gdb-index, because gold linker removes section .debug_aranges, which used inside clickhouse stacktraces - # http://sourceware-org.1504.n7.nabble.com/gold-No-debug-aranges-section-when-linking-with-gdb-index-td540965.html#a556932 - elseif (LINKER_NAME MATCHES "gold$" AND ADD_GDB_INDEX_FOR_GOLD) - find_program (GDB_ADD_INDEX_EXE NAMES "gdb-add-index" DOC "Path to gdb-add-index executable") - if (NOT GDB_ADD_INDEX_EXE) - set (USE_GDB_ADD_INDEX 0) - message (WARNING "Cannot add gdb index to binaries, because gold linker is used, but gdb-add-index executable not found.") - else() - set (USE_GDB_ADD_INDEX 1) - message (STATUS "gdb-add-index found: ${GDB_ADD_INDEX_EXE}") - endif() - endif () -endif() - -# Create BuildID when using lld. For other linkers it is created by default. -if (LINKER_NAME MATCHES "lld$") - # SHA1 is not cryptographically secure but it is the best what lld is offering. - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--build-id=sha1") -endif () - -# Add a section with the hash of the compiled machine code for integrity checks. -# Only for official builds, because adding a section can be time consuming (rewrite of several GB). -# And cross compiled binaries are not supported (since you cannot execute clickhouse hash-binary) -if (OBJCOPY_PATH AND YANDEX_OFFICIAL_BUILD AND (NOT CMAKE_TOOLCHAIN_FILE)) - set (USE_BINARY_HASH 1) -endif () - -cmake_host_system_information(RESULT AVAILABLE_PHYSICAL_MEMORY QUERY AVAILABLE_PHYSICAL_MEMORY) # Not available under freebsd - - -if(NOT AVAILABLE_PHYSICAL_MEMORY OR AVAILABLE_PHYSICAL_MEMORY GREATER 8000) - # Less `/tmp` usage, more RAM usage. - option(COMPILER_PIPE "-pipe compiler option" ON) -endif() - -if(COMPILER_PIPE) - set(COMPILER_FLAGS "${COMPILER_FLAGS} -pipe") -else() - message(STATUS "Disabling compiler -pipe option (have only ${AVAILABLE_PHYSICAL_MEMORY} mb of memory)") -endif() - -include(cmake/cpu_features.cmake) - -# Asynchronous unwind tables are needed for Query Profiler. -# They are already by default on some platforms but possibly not on all platforms. -# Enable it explicitly. -set (COMPILER_FLAGS "${COMPILER_FLAGS} -fasynchronous-unwind-tables") - -if (${CMAKE_VERSION} VERSION_LESS "3.12.4") - # CMake < 3.12 doesn't support setting 20 as a C++ standard version. - # We will add C++ standard controlling flag in CMAKE_CXX_FLAGS manually for now. - - if (COMPILER_GCC OR COMPILER_CLANG) - # to make numeric_limits<__int128> works with GCC - set (_CXX_STANDARD "gnu++2a") - else () - set (_CXX_STANDARD "c++2a") - endif () - - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=${_CXX_STANDARD}") -else () - set (CMAKE_CXX_STANDARD 20) - set (CMAKE_CXX_EXTENSIONS ON) # Same as gnu++2a (ON) vs c++2a (OFF): https://cmake.org/cmake/help/latest/prop_tgt/CXX_EXTENSIONS.html - set (CMAKE_CXX_STANDARD_REQUIRED ON) -endif () - -set (CMAKE_C_STANDARD 11) -set (CMAKE_C_EXTENSIONS ON) -set (CMAKE_C_STANDARD_REQUIRED ON) - -if (COMPILER_GCC OR COMPILER_CLANG) - # Enable C++14 sized global deallocation functions. It should be enabled by setting -std=c++14 but I'm not sure. - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fsized-deallocation") -endif () - -# falign-functions=32 prevents from random performance regressions with the code change. Thus, providing more stable -# benchmarks. -if (COMPILER_GCC OR COMPILER_CLANG) - set(COMPILER_FLAGS "${COMPILER_FLAGS} -falign-functions=32") -endif () - -# Compiler-specific coverage flags e.g. -fcoverage-mapping for gcc -option(WITH_COVERAGE "Profile the resulting binary/binaries" OFF) - -if (WITH_COVERAGE AND COMPILER_CLANG) - set(COMPILER_FLAGS "${COMPILER_FLAGS} -fprofile-instr-generate -fcoverage-mapping") - # If we want to disable coverage for specific translation units - set(WITHOUT_COVERAGE "-fno-profile-instr-generate -fno-coverage-mapping") -endif() - -if (WITH_COVERAGE AND COMPILER_GCC) - set(COMPILER_FLAGS "${COMPILER_FLAGS} -fprofile-arcs -ftest-coverage") - set(COVERAGE_OPTION "-lgcov") - set(WITHOUT_COVERAGE "-fno-profile-arcs -fno-test-coverage") -endif() - -set(COMPILER_FLAGS "${COMPILER_FLAGS}") - -set (CMAKE_BUILD_COLOR_MAKEFILE ON) -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${COMPILER_FLAGS} ${PLATFORM_EXTRA_CXX_FLAG} ${COMMON_WARNING_FLAGS} ${CXX_WARNING_FLAGS}") -set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_CXX_FLAGS_ADD}") -set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline ${CMAKE_CXX_FLAGS_ADD}") - -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${COMPILER_FLAGS} ${COMMON_WARNING_FLAGS} ${CMAKE_C_FLAGS_ADD}") -set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_C_FLAGS_ADD}") -set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline ${CMAKE_C_FLAGS_ADD}") - -if (COMPILER_CLANG) - if (OS_DARWIN) - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") - set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-U,_inside_main") - endif() - - # Display absolute paths in error messages. Otherwise KDevelop fails to navigate to correct file and opens a new file instead. - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fdiagnostics-absolute-paths") - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fdiagnostics-absolute-paths") - - if (NOT ENABLE_TESTS AND NOT SANITIZE) - # https://clang.llvm.org/docs/ThinLTO.html - # Applies to clang only. - # Disabled when building with tests or sanitizers. - option(ENABLE_THINLTO "Clang-specific link time optimization" ON) - endif() - - # Set new experimental pass manager, it's a performance, build time and binary size win. - # Can be removed after https://reviews.llvm.org/D66490 merged and released to at least two versions of clang. - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fexperimental-new-pass-manager") - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fexperimental-new-pass-manager") - - # We cannot afford to use LTO when compiling unit tests, and it's not enough - # to only supply -fno-lto at the final linking stage. So we disable it - # completely. - if (ENABLE_THINLTO AND NOT ENABLE_TESTS AND NOT SANITIZE) - # Link time optimization - set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -flto=thin") - set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -flto=thin") - set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -flto=thin") - elseif (ENABLE_THINLTO) - message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable ThinLTO") - endif () - - # Always prefer llvm tools when using clang. For instance, we cannot use GNU ar when llvm LTO is enabled - find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") - - if (LLVM_AR_PATH) - message(STATUS "Using llvm-ar: ${LLVM_AR_PATH}.") - set (CMAKE_AR ${LLVM_AR_PATH}) - else () - message(WARNING "Cannot find llvm-ar. System ar will be used instead. It does not work with ThinLTO.") - endif () - - find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8") - - if (LLVM_RANLIB_PATH) - message(STATUS "Using llvm-ranlib: ${LLVM_RANLIB_PATH}.") - set (CMAKE_RANLIB ${LLVM_RANLIB_PATH}) - else () - message(WARNING "Cannot find llvm-ranlib. System ranlib will be used instead. It does not work with ThinLTO.") - endif () - -elseif (ENABLE_THINLTO) - message (${RECONFIGURE_MESSAGE_LEVEL} "ThinLTO is only available with CLang") -endif () - -# Turns on all external libs like s3, kafka, ODBC, ... -option(ENABLE_LIBRARIES "Enable all external libraries by default" ON) - -# We recommend avoiding this mode for production builds because we can't guarantee -# all needed libraries exist in your system. -# This mode exists for enthusiastic developers who are searching for trouble. -# The whole idea of using unknown version of libraries from the OS distribution is deeply flawed. -# Useful for maintainers of OS packages. -option (UNBUNDLED "Use system libraries instead of ones in contrib/" OFF) - -if (UNBUNDLED) - set(NOT_UNBUNDLED OFF) -else () - set(NOT_UNBUNDLED ON) -endif () - -if (UNBUNDLED OR NOT (OS_LINUX OR OS_DARWIN)) - # Using system libs can cause a lot of warnings in includes (on macro expansion). - option(WERROR "Enable -Werror compiler option" OFF) -else () - option(WERROR "Enable -Werror compiler option" ON) -endif () - -if (WERROR) - add_warning(error) -endif () - -# Make this extra-checks for correct library dependencies. -if (OS_LINUX AND NOT SANITIZE) - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-undefined") - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined") -endif () - -include(cmake/dbms_glob_sources.cmake) - -if (OS_LINUX OR OS_ANDROID) - include(cmake/linux/default_libs.cmake) -elseif (OS_DARWIN) - include(cmake/darwin/default_libs.cmake) -elseif (OS_FREEBSD) - include(cmake/freebsd/default_libs.cmake) -endif () - -###################################### -### Add targets below this comment ### -###################################### - -set (CMAKE_POSTFIX_VARIABLE "CMAKE_${CMAKE_BUILD_TYPE_UC}_POSTFIX") - -if (MAKE_STATIC_LIBRARIES) - set (CMAKE_POSITION_INDEPENDENT_CODE OFF) - if (OS_LINUX AND NOT ARCH_ARM) - # Slightly more efficient code can be generated - # It's disabled for ARM because otherwise ClickHouse cannot run on Android. - set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -fno-pie") - set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -fno-pie") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-no-pie") - endif () -else () - set (CMAKE_POSITION_INDEPENDENT_CODE ON) -endif () - -# https://github.com/include-what-you-use/include-what-you-use -option (USE_INCLUDE_WHAT_YOU_USE "Automatically reduce unneeded includes in source code (external tool)" OFF) - -if (USE_INCLUDE_WHAT_YOU_USE) - find_program(IWYU_PATH NAMES include-what-you-use iwyu) - if (NOT IWYU_PATH) - message(FATAL_ERROR "Could not find the program include-what-you-use") - endif() - if (${CMAKE_VERSION} VERSION_LESS "3.3.0") - message(FATAL_ERROR "include-what-you-use requires CMake version at least 3.3.") - endif() -endif () - -if (ENABLE_TESTS) - message (STATUS "Unit tests are enabled") -else() - message(STATUS "Unit tests are disabled") -endif () - -enable_testing() # Enable for tests without binary - -# when installing to /usr - place configs to /etc but for /usr/local place to /usr/local/etc -if (CMAKE_INSTALL_PREFIX STREQUAL "/usr") - set (CLICKHOUSE_ETC_DIR "/etc") -else () - set (CLICKHOUSE_ETC_DIR "${CMAKE_INSTALL_PREFIX}/etc") -endif () - -message (STATUS - "Building for: ${CMAKE_SYSTEM} ${CMAKE_SYSTEM_PROCESSOR} ${CMAKE_LIBRARY_ARCHITECTURE} ; - USE_STATIC_LIBRARIES=${USE_STATIC_LIBRARIES} - MAKE_STATIC_LIBRARIES=${MAKE_STATIC_LIBRARIES} - SPLIT_SHARED=${SPLIT_SHARED_LIBRARIES} - UNBUNDLED=${UNBUNDLED} - CCACHE=${CCACHE_FOUND} ${CCACHE_VERSION}") - -include (GNUInstallDirs) -include (cmake/contrib_finder.cmake) - -find_contrib_lib(double-conversion) # Must be before parquet -include (cmake/find/ssl.cmake) -include (cmake/find/ldap.cmake) # after ssl -include (cmake/find/icu.cmake) -include (cmake/find/xz.cmake) -include (cmake/find/zlib.cmake) -include (cmake/find/zstd.cmake) -include (cmake/find/ltdl.cmake) # for odbc -# openssl, zlib before poco -include (cmake/find/sparsehash.cmake) -include (cmake/find/re2.cmake) -include (cmake/find/krb5.cmake) -include (cmake/find/libgsasl.cmake) -include (cmake/find/cyrus-sasl.cmake) -include (cmake/find/rdkafka.cmake) -include (cmake/find/libuv.cmake) # for amqpcpp and cassandra -include (cmake/find/amqpcpp.cmake) -include (cmake/find/capnp.cmake) -include (cmake/find/llvm.cmake) -include (cmake/find/h3.cmake) -include (cmake/find/libxml2.cmake) -include (cmake/find/brotli.cmake) -include (cmake/find/protobuf.cmake) -include (cmake/find/grpc.cmake) -include (cmake/find/pdqsort.cmake) -include (cmake/find/miniselect.cmake) -include (cmake/find/hdfs3.cmake) # uses protobuf -include (cmake/find/poco.cmake) -include (cmake/find/curl.cmake) -include (cmake/find/s3.cmake) -include (cmake/find/base64.cmake) -include (cmake/find/parquet.cmake) -include (cmake/find/simdjson.cmake) -include (cmake/find/fast_float.cmake) -include (cmake/find/rapidjson.cmake) -include (cmake/find/fastops.cmake) -include (cmake/find/odbc.cmake) -include (cmake/find/nanodbc.cmake) -include (cmake/find/sqlite.cmake) -include (cmake/find/rocksdb.cmake) -include (cmake/find/libpqxx.cmake) -include (cmake/find/nuraft.cmake) -include (cmake/find/yaml-cpp.cmake) -include (cmake/find/s2geometry.cmake) -include (cmake/find/nlp.cmake) -include (cmake/find/bzip2.cmake) - -if(NOT USE_INTERNAL_PARQUET_LIBRARY) - set (ENABLE_ORC OFF CACHE INTERNAL "") -endif() -include (cmake/find/orc.cmake) - -include (cmake/find/avro.cmake) -include (cmake/find/msgpack.cmake) -include (cmake/find/cassandra.cmake) -include (cmake/find/sentry.cmake) -include (cmake/find/stats.cmake) -include (cmake/find/datasketches.cmake) - -set (USE_INTERNAL_CITYHASH_LIBRARY ON CACHE INTERNAL "") -find_contrib_lib(cityhash) - -find_contrib_lib(farmhash) - -if (ENABLE_TESTS) - include (cmake/find/gtest.cmake) -endif () - -# Need to process before "contrib" dir: -include (cmake/find/mysqlclient.cmake) - -# When testing for memory leaks with Valgrind, don't link tcmalloc or jemalloc. - -include (cmake/print_flags.cmake) - -if (TARGET global-group) - install (EXPORT global DESTINATION cmake) -endif () - -add_subdirectory (contrib EXCLUDE_FROM_ALL) - -if (NOT ENABLE_JEMALLOC) - message (WARNING "Non default allocator is disabled. This is not recommended for production builds.") -endif () - -macro (add_executable target) - # invoke built-in add_executable - # explicitly acquire and interpose malloc symbols by clickhouse_malloc - # if GLIBC_COMPATIBILITY is ON and ENABLE_THINLTO is on than provide memcpy symbol explicitly to neutrialize thinlto's libcall generation. - if (GLIBC_COMPATIBILITY AND ENABLE_THINLTO) - _add_executable (${ARGV} $ $) - else () - _add_executable (${ARGV} $) - endif () - - get_target_property (type ${target} TYPE) - if (${type} STREQUAL EXECUTABLE) - # disabled for TSAN and gcc since libtsan.a provides overrides too - if (TARGET clickhouse_new_delete) - # operator::new/delete for executables (MemoryTracker stuff) - target_link_libraries (${target} PRIVATE clickhouse_new_delete) - endif() - - # In case of static jemalloc, because zone_register() is located in zone.c and - # is never used outside (it is declared as constructor) it is omitted - # by the linker, and so jemalloc will not be registered as system - # allocator under osx [1], and clickhouse will SIGSEGV. - # - # [1]: https://github.com/jemalloc/jemalloc/issues/708 - # - # About symbol name: - # - _zone_register not zone_register due to Mach-O binary format, - # - _je_zone_register due to JEMALLOC_PRIVATE_NAMESPACE=je_ under OS X. - # - but jemalloc-cmake does not run private_namespace.sh - # so symbol name should be _zone_register - if (ENABLE_JEMALLOC AND MAKE_STATIC_LIBRARIES AND OS_DARWIN) - set_property(TARGET ${target} APPEND PROPERTY LINK_OPTIONS -u_zone_register) - endif() - endif() -endmacro() - -set(ConfigIncludePath ${CMAKE_CURRENT_BINARY_DIR}/includes/configs CACHE INTERNAL "Path to generated configuration files.") -include_directories(${ConfigIncludePath}) - -# Add as many warnings as possible for our own code. -include (cmake/warnings.cmake) - -add_subdirectory (base) -add_subdirectory (src) -add_subdirectory (programs) -add_subdirectory (tests) -add_subdirectory (utils) - -include (cmake/print_include_directories.cmake) - -include (cmake/sanitize_target_link_libraries.cmake) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index ae6885ef9f0..f5460ec3f14 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -362,7 +362,8 @@ bool MergeTreeConditionFullText::getKey(const ASTPtr & node, size_t & key_column //try to get map column name in arrayElement function if (const auto * func = typeid_cast(node.get())) { - column_name = assert_cast(func->arguments.get()->children[0].get())->name(); + if (func->name == "arrayElement") + column_name = assert_cast(func->arguments.get()->children[0].get())->name(); } auto it = std::find(index_columns.begin(), index_columns.end(), column_name); @@ -414,7 +415,8 @@ bool MergeTreeConditionFullText::atomFromAST( //try to parse arrayElement function if (const auto * map_func = typeid_cast(args[0].get())) { - const_value = assert_cast(map_func->arguments.get()->children[1].get())->name(); + if (map_func->name == "arrayElement") + const_value = assert_cast(map_func->arguments.get()->children[1].get())->name(); } if (key_arg_pos == 1 && (func_name != "equals" && func_name != "notEquals")) From 400fdf6ead030de827f0744b665d5cb8d2461b81 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Mon, 6 Sep 2021 16:42:45 +0800 Subject: [PATCH 109/317] Rebase CMakeLists.txt --- CMakeLists.txt | 621 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 621 insertions(+) create mode 100644 CMakeLists.txt diff --git a/CMakeLists.txt b/CMakeLists.txt new file mode 100644 index 00000000000..875a6d1ab61 --- /dev/null +++ b/CMakeLists.txt @@ -0,0 +1,621 @@ +cmake_minimum_required(VERSION 3.3) + +foreach(policy + CMP0023 + CMP0048 # CMake 3.0 + CMP0074 # CMake 3.12 + CMP0077 + CMP0079 + ) + if(POLICY ${policy}) + cmake_policy(SET ${policy} NEW) + endif() +endforeach() + +# set default policy +foreach(default_policy_var_name + # make option() honor normal variables for BUILD_SHARED_LIBS: + # - re2 + # - snappy + CMAKE_POLICY_DEFAULT_CMP0077 + # Google Test from sources uses too old cmake, 2.6.x, and CMP0022 should + # set, to avoid using deprecated LINK_INTERFACE_LIBRARIES(_)? over + # INTERFACE_LINK_LIBRARIES. + CMAKE_POLICY_DEFAULT_CMP0022 + ) + set(${default_policy_var_name} NEW) +endforeach() + +project(ClickHouse) + +# If turned off: e.g. when ENABLE_FOO is ON, but FOO tool was not found, the CMake will continue. +option(FAIL_ON_UNSUPPORTED_OPTIONS_COMBINATION + "Stop/Fail CMake configuration if some ENABLE_XXX option is defined (either ON or OFF) + but is not possible to satisfy" ON) + +if(FAIL_ON_UNSUPPORTED_OPTIONS_COMBINATION) + set(RECONFIGURE_MESSAGE_LEVEL FATAL_ERROR) +else() + set(RECONFIGURE_MESSAGE_LEVEL WARNING) +endif() + +enable_language(C CXX ASM) + +include (cmake/arch.cmake) +include (cmake/target.cmake) +include (cmake/tools.cmake) +include (cmake/analysis.cmake) + +# Ignore export() since we don't use it, +# but it gets broken with a global targets via link_libraries() +macro (export) +endmacro () + +set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${CMAKE_CURRENT_SOURCE_DIR}/cmake/Modules/") +set(CMAKE_EXPORT_COMPILE_COMMANDS 1) # Write compile_commands.json +set(CMAKE_LINK_DEPENDS_NO_SHARED 1) # Do not relink all depended targets on .so +set(CMAKE_CONFIGURATION_TYPES "RelWithDebInfo;Debug;Release;MinSizeRel" CACHE STRING "" FORCE) +set(CMAKE_DEBUG_POSTFIX "d" CACHE STRING "Generate debug library name with a postfix.") # To be consistent with CMakeLists from contrib libs. + +# Enable the ability to organize targets into hierarchies of "folders" for capable GUI-based IDEs. +# For more info see https://cmake.org/cmake/help/latest/prop_gbl/USE_FOLDERS.html +set_property(GLOBAL PROPERTY USE_FOLDERS ON) + +# Check that submodules are present only if source was downloaded with git +if (EXISTS "${CMAKE_CURRENT_SOURCE_DIR}/.git" AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/boost/boost") + message (FATAL_ERROR "Submodules are not initialized. Run\n\tgit submodule update --init --recursive") +endif () + +include (cmake/find/ccache.cmake) + +# Take care to add prlimit in command line before ccache, or else ccache thinks that +# prlimit is compiler, and clang++ is its input file, and refuses to work with +# multiple inputs, e.g in ccache log: +# [2021-03-31T18:06:32.655327 36900] Command line: /usr/bin/ccache prlimit --as=10000000000 --data=5000000000 --cpu=600 /usr/bin/clang++-11 - ...... std=gnu++2a -MD -MT src/CMakeFiles/dbms.dir/Storages/MergeTree/IMergeTreeDataPart.cpp.o -MF src/CMakeFiles/dbms.dir/Storages/MergeTree/IMergeTreeDataPart.cpp.o.d -o src/CMakeFiles/dbms.dir/Storages/MergeTree/IMergeTreeDataPart.cpp.o -c ../src/Storages/MergeTree/IMergeTreeDataPart.cpp +# +# [2021-03-31T18:06:32.656704 36900] Multiple input files: /usr/bin/clang++-11 and ../src/Storages/MergeTree/IMergeTreeDataPart.cpp +# +# Another way would be to use --ccache-skip option before clang++-11 to make +# ccache ignore it. +option(ENABLE_CHECK_HEAVY_BUILDS "Don't allow C++ translation units to compile too long or to take too much memory while compiling." OFF) +if (ENABLE_CHECK_HEAVY_BUILDS) + # set DATA (since RSS does not work since 2.6.x+) to 2G + set (RLIMIT_DATA 5000000000) + # set VIRT (RLIMIT_AS) to 10G (DATA*10) + set (RLIMIT_AS 10000000000) + # set CPU time limit to 600 seconds + set (RLIMIT_CPU 600) + + # gcc10/gcc10/clang -fsanitize=memory is too heavy + if (SANITIZE STREQUAL "memory" OR COMPILER_GCC) + set (RLIMIT_DATA 10000000000) + endif() + + set (CMAKE_CXX_COMPILER_LAUNCHER prlimit --as=${RLIMIT_AS} --data=${RLIMIT_DATA} --cpu=${RLIMIT_CPU} ${CMAKE_CXX_COMPILER_LAUNCHER}) +endif () + +if (NOT CMAKE_BUILD_TYPE OR CMAKE_BUILD_TYPE STREQUAL "None") + set (CMAKE_BUILD_TYPE "RelWithDebInfo") + message (STATUS "CMAKE_BUILD_TYPE is not set, set to default = ${CMAKE_BUILD_TYPE}") +endif () +message (STATUS "CMAKE_BUILD_TYPE: ${CMAKE_BUILD_TYPE}") + +string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC) + +option(USE_STATIC_LIBRARIES "Disable to use shared libraries" ON) +option(MAKE_STATIC_LIBRARIES "Disable to make shared libraries" ${USE_STATIC_LIBRARIES}) + +if (NOT MAKE_STATIC_LIBRARIES) + # DEVELOPER ONLY. + # Faster linking if turned on. + option(SPLIT_SHARED_LIBRARIES "Keep all internal libraries as separate .so files") + + option(CLICKHOUSE_SPLIT_BINARY + "Make several binaries (clickhouse-server, clickhouse-client etc.) instead of one bundled") +endif () + +if (MAKE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) + message(FATAL_ERROR "Defining SPLIT_SHARED_LIBRARIES=1 without MAKE_STATIC_LIBRARIES=0 has no effect.") +endif() + +if (NOT MAKE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) + set(BUILD_SHARED_LIBS 1 CACHE INTERNAL "") +endif () + +if (USE_STATIC_LIBRARIES) + list(REVERSE CMAKE_FIND_LIBRARY_SUFFIXES) +endif () + +# Implies ${WITH_COVERAGE} +option (ENABLE_FUZZING "Fuzzy testing using libfuzzer" OFF) + +if (ENABLE_FUZZING) + message (STATUS "Fuzzing instrumentation enabled") + set (WITH_COVERAGE ON) + set (FUZZER "libfuzzer") +endif() + +# Global libraries +# See: +# - default_libs.cmake +# - sanitize.cmake +add_library(global-libs INTERFACE) + +include (cmake/fuzzer.cmake) +include (cmake/sanitize.cmake) + +if (CMAKE_GENERATOR STREQUAL "Ninja" AND NOT DISABLE_COLORED_BUILD) + # Turn on colored output. https://github.com/ninja-build/ninja/wiki/FAQ + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fdiagnostics-color=always") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fdiagnostics-color=always") +endif () + +include (cmake/add_warning.cmake) + +if (NOT MSVC) + set (COMMON_WARNING_FLAGS "${COMMON_WARNING_FLAGS} -Wall") # -Werror and many more is also added inside cmake/warnings.cmake +endif () + +if (COMPILER_CLANG) + # clang: warning: argument unused during compilation: '-specs=/usr/share/dpkg/no-pie-compile.specs' [-Wunused-command-line-argument] + set (COMMON_WARNING_FLAGS "${COMMON_WARNING_FLAGS} -Wno-unused-command-line-argument") + # generate ranges for fast "addr2line" search + if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") + set(COMPILER_FLAGS "${COMPILER_FLAGS} -gdwarf-aranges") + endif () +endif () + +# If turned `ON`, assumes the user has either the system GTest library or the bundled one. +option(ENABLE_TESTS "Provide unit_test_dbms target with Google.Test unit tests" ON) +option(ENABLE_EXAMPLES "Build all example programs in 'examples' subdirectories" OFF) + +if (OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64) AND NOT UNBUNDLED AND MAKE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES AND CMAKE_VERSION VERSION_GREATER "3.9.0") + # Only for Linux, x86_64 or aarch64. + option(GLIBC_COMPATIBILITY "Enable compatibility with older glibc libraries." ON) +elseif(GLIBC_COMPATIBILITY) + message (${RECONFIGURE_MESSAGE_LEVEL} "Glibc compatibility cannot be enabled in current configuration") +endif () + +if (NOT CMAKE_VERSION VERSION_GREATER "3.9.0") + message (WARNING "CMake version must be greater than 3.9.0 for production builds.") +endif () + +# Make sure the final executable has symbols exported +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic") + +find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-12" "llvm-objcopy-11" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy") + +if (NOT OBJCOPY_PATH AND OS_DARWIN) + find_program (BREW_PATH NAMES "brew") + if (BREW_PATH) + execute_process (COMMAND ${BREW_PATH} --prefix llvm ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE OUTPUT_VARIABLE LLVM_PREFIX) + if (LLVM_PREFIX) + find_program (OBJCOPY_PATH NAMES "llvm-objcopy" PATHS "${LLVM_PREFIX}/bin" NO_DEFAULT_PATH) + endif () + if (NOT OBJCOPY_PATH) + execute_process (COMMAND ${BREW_PATH} --prefix binutils ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE OUTPUT_VARIABLE BINUTILS_PREFIX) + if (BINUTILS_PREFIX) + find_program (OBJCOPY_PATH NAMES "objcopy" PATHS "${BINUTILS_PREFIX}/bin" NO_DEFAULT_PATH) + endif () + endif () + endif () +endif () + +if (OBJCOPY_PATH) + message (STATUS "Using objcopy: ${OBJCOPY_PATH}") +else () + message (FATAL_ERROR "Cannot find objcopy.") +endif () + +if (OS_DARWIN) + # The `-all_load` flag forces loading of all symbols from all libraries, + # and leads to multiply-defined symbols. This flag allows force loading + # from a _specific_ library, which is what we need. + set(WHOLE_ARCHIVE -force_load) + # The `-noall_load` flag is the default and now obsolete. + set(NO_WHOLE_ARCHIVE "") +else () + set(WHOLE_ARCHIVE --whole-archive) + set(NO_WHOLE_ARCHIVE --no-whole-archive) +endif () + +# Ignored if `lld` is used +option(ADD_GDB_INDEX_FOR_GOLD "Add .gdb-index to resulting binaries for gold linker.") + +if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") + # Can be lld or ld-lld. + if (LINKER_NAME MATCHES "lld$") + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--gdb-index") + set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--gdb-index") + message (STATUS "Adding .gdb-index via --gdb-index linker option.") + # we use another tool for gdb-index, because gold linker removes section .debug_aranges, which used inside clickhouse stacktraces + # http://sourceware-org.1504.n7.nabble.com/gold-No-debug-aranges-section-when-linking-with-gdb-index-td540965.html#a556932 + elseif (LINKER_NAME MATCHES "gold$" AND ADD_GDB_INDEX_FOR_GOLD) + find_program (GDB_ADD_INDEX_EXE NAMES "gdb-add-index" DOC "Path to gdb-add-index executable") + if (NOT GDB_ADD_INDEX_EXE) + set (USE_GDB_ADD_INDEX 0) + message (WARNING "Cannot add gdb index to binaries, because gold linker is used, but gdb-add-index executable not found.") + else() + set (USE_GDB_ADD_INDEX 1) + message (STATUS "gdb-add-index found: ${GDB_ADD_INDEX_EXE}") + endif() + endif () +endif() + +# Create BuildID when using lld. For other linkers it is created by default. +if (LINKER_NAME MATCHES "lld$") + # SHA1 is not cryptographically secure but it is the best what lld is offering. + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--build-id=sha1") +endif () + +# Add a section with the hash of the compiled machine code for integrity checks. +# Only for official builds, because adding a section can be time consuming (rewrite of several GB). +# And cross compiled binaries are not supported (since you cannot execute clickhouse hash-binary) +if (OBJCOPY_PATH AND YANDEX_OFFICIAL_BUILD AND (NOT CMAKE_TOOLCHAIN_FILE)) + set (USE_BINARY_HASH 1) +endif () + +cmake_host_system_information(RESULT AVAILABLE_PHYSICAL_MEMORY QUERY AVAILABLE_PHYSICAL_MEMORY) # Not available under freebsd + + +if(NOT AVAILABLE_PHYSICAL_MEMORY OR AVAILABLE_PHYSICAL_MEMORY GREATER 8000) + # Less `/tmp` usage, more RAM usage. + option(COMPILER_PIPE "-pipe compiler option" ON) +endif() + +if(COMPILER_PIPE) + set(COMPILER_FLAGS "${COMPILER_FLAGS} -pipe") +else() + message(STATUS "Disabling compiler -pipe option (have only ${AVAILABLE_PHYSICAL_MEMORY} mb of memory)") +endif() + +include(cmake/cpu_features.cmake) + +option(ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries non-portable but more performant code may be generated.") + +if (ARCH_NATIVE) + set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native") +endif () + +# Asynchronous unwind tables are needed for Query Profiler. +# They are already by default on some platforms but possibly not on all platforms. +# Enable it explicitly. +set (COMPILER_FLAGS "${COMPILER_FLAGS} -fasynchronous-unwind-tables") + +if (${CMAKE_VERSION} VERSION_LESS "3.12.4") + # CMake < 3.12 doesn't support setting 20 as a C++ standard version. + # We will add C++ standard controlling flag in CMAKE_CXX_FLAGS manually for now. + + if (COMPILER_GCC OR COMPILER_CLANG) + # to make numeric_limits<__int128> works with GCC + set (_CXX_STANDARD "gnu++2a") + else () + set (_CXX_STANDARD "c++2a") + endif () + + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -std=${_CXX_STANDARD}") +else () + set (CMAKE_CXX_STANDARD 20) + set (CMAKE_CXX_EXTENSIONS ON) # Same as gnu++2a (ON) vs c++2a (OFF): https://cmake.org/cmake/help/latest/prop_tgt/CXX_EXTENSIONS.html + set (CMAKE_CXX_STANDARD_REQUIRED ON) +endif () + +set (CMAKE_C_STANDARD 11) +set (CMAKE_C_EXTENSIONS ON) +set (CMAKE_C_STANDARD_REQUIRED ON) + +if (COMPILER_GCC OR COMPILER_CLANG) + # Enable C++14 sized global deallocation functions. It should be enabled by setting -std=c++14 but I'm not sure. + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fsized-deallocation") +endif () + +# falign-functions=32 prevents from random performance regressions with the code change. Thus, providing more stable +# benchmarks. +if (COMPILER_GCC OR COMPILER_CLANG) + set(COMPILER_FLAGS "${COMPILER_FLAGS} -falign-functions=32") +endif () + +# Compiler-specific coverage flags e.g. -fcoverage-mapping for gcc +option(WITH_COVERAGE "Profile the resulting binary/binaries" OFF) + +if (WITH_COVERAGE AND COMPILER_CLANG) + set(COMPILER_FLAGS "${COMPILER_FLAGS} -fprofile-instr-generate -fcoverage-mapping") + # If we want to disable coverage for specific translation units + set(WITHOUT_COVERAGE "-fno-profile-instr-generate -fno-coverage-mapping") +endif() + +if (WITH_COVERAGE AND COMPILER_GCC) + set(COMPILER_FLAGS "${COMPILER_FLAGS} -fprofile-arcs -ftest-coverage") + set(COVERAGE_OPTION "-lgcov") + set(WITHOUT_COVERAGE "-fno-profile-arcs -fno-test-coverage") +endif() + +set(COMPILER_FLAGS "${COMPILER_FLAGS}") + +set (CMAKE_BUILD_COLOR_MAKEFILE ON) +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${COMPILER_FLAGS} ${PLATFORM_EXTRA_CXX_FLAG} ${COMMON_WARNING_FLAGS} ${CXX_WARNING_FLAGS}") +set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_CXX_FLAGS_ADD}") +set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline ${CMAKE_CXX_FLAGS_ADD}") + +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${COMPILER_FLAGS} ${COMMON_WARNING_FLAGS} ${CMAKE_C_FLAGS_ADD}") +set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_C_FLAGS_ADD}") +set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline ${CMAKE_C_FLAGS_ADD}") + +if (COMPILER_CLANG) + if (OS_DARWIN) + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") + set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-U,_inside_main") + endif() + + # Display absolute paths in error messages. Otherwise KDevelop fails to navigate to correct file and opens a new file instead. + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fdiagnostics-absolute-paths") + set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fdiagnostics-absolute-paths") + + if (NOT ENABLE_TESTS AND NOT SANITIZE) + # https://clang.llvm.org/docs/ThinLTO.html + # Applies to clang only. + # Disabled when building with tests or sanitizers. + option(ENABLE_THINLTO "Clang-specific link time optimization" ON) + endif() + + # Set new experimental pass manager, it's a performance, build time and binary size win. + # Can be removed after https://reviews.llvm.org/D66490 merged and released to at least two versions of clang. + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fexperimental-new-pass-manager") + set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fexperimental-new-pass-manager") + + # We cannot afford to use LTO when compiling unit tests, and it's not enough + # to only supply -fno-lto at the final linking stage. So we disable it + # completely. + if (ENABLE_THINLTO AND NOT ENABLE_TESTS AND NOT SANITIZE) + # Link time optimization + set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -flto=thin") + set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -flto=thin") + set (CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO "${CMAKE_EXE_LINKER_FLAGS_RELWITHDEBINFO} -flto=thin") + elseif (ENABLE_THINLTO) + message (${RECONFIGURE_MESSAGE_LEVEL} "Cannot enable ThinLTO") + endif () + + # Always prefer llvm tools when using clang. For instance, we cannot use GNU ar when llvm LTO is enabled + find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") + + if (LLVM_AR_PATH) + message(STATUS "Using llvm-ar: ${LLVM_AR_PATH}.") + set (CMAKE_AR ${LLVM_AR_PATH}) + else () + message(WARNING "Cannot find llvm-ar. System ar will be used instead. It does not work with ThinLTO.") + endif () + + find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8") + + if (LLVM_RANLIB_PATH) + message(STATUS "Using llvm-ranlib: ${LLVM_RANLIB_PATH}.") + set (CMAKE_RANLIB ${LLVM_RANLIB_PATH}) + else () + message(WARNING "Cannot find llvm-ranlib. System ranlib will be used instead. It does not work with ThinLTO.") + endif () + +elseif (ENABLE_THINLTO) + message (${RECONFIGURE_MESSAGE_LEVEL} "ThinLTO is only available with CLang") +endif () + +# Turns on all external libs like s3, kafka, ODBC, ... +option(ENABLE_LIBRARIES "Enable all external libraries by default" ON) + +# We recommend avoiding this mode for production builds because we can't guarantee all needed libraries exist in your +# system. +# This mode exists for enthusiastic developers who are searching for trouble. +# Useful for maintainers of OS packages. +option (UNBUNDLED "Use system libraries instead of ones in contrib/" OFF) + +if (UNBUNDLED) + set(NOT_UNBUNDLED OFF) +else () + set(NOT_UNBUNDLED ON) +endif () + +if (UNBUNDLED OR NOT (OS_LINUX OR OS_DARWIN)) + # Using system libs can cause a lot of warnings in includes (on macro expansion). + option(WERROR "Enable -Werror compiler option" OFF) +else () + option(WERROR "Enable -Werror compiler option" ON) +endif () + +if (WERROR) + add_warning(error) +endif () + +# Make this extra-checks for correct library dependencies. +if (OS_LINUX AND NOT SANITIZE) + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-undefined") + set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined") +endif () + +include(cmake/dbms_glob_sources.cmake) + +if (OS_LINUX OR OS_ANDROID) + include(cmake/linux/default_libs.cmake) +elseif (OS_DARWIN) + include(cmake/darwin/default_libs.cmake) +elseif (OS_FREEBSD) + include(cmake/freebsd/default_libs.cmake) +endif () + +###################################### +### Add targets below this comment ### +###################################### + +set (CMAKE_POSTFIX_VARIABLE "CMAKE_${CMAKE_BUILD_TYPE_UC}_POSTFIX") + +if (MAKE_STATIC_LIBRARIES) + set (CMAKE_POSITION_INDEPENDENT_CODE OFF) + if (OS_LINUX AND NOT ARCH_ARM) + # Slightly more efficient code can be generated + # It's disabled for ARM because otherwise ClickHouse cannot run on Android. + set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -fno-pie") + set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -fno-pie") + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,-no-pie") + endif () +else () + set (CMAKE_POSITION_INDEPENDENT_CODE ON) +endif () + +# https://github.com/include-what-you-use/include-what-you-use +option (USE_INCLUDE_WHAT_YOU_USE "Automatically reduce unneeded includes in source code (external tool)" OFF) + +if (USE_INCLUDE_WHAT_YOU_USE) + find_program(IWYU_PATH NAMES include-what-you-use iwyu) + if (NOT IWYU_PATH) + message(FATAL_ERROR "Could not find the program include-what-you-use") + endif() + if (${CMAKE_VERSION} VERSION_LESS "3.3.0") + message(FATAL_ERROR "include-what-you-use requires CMake version at least 3.3.") + endif() +endif () + +if (ENABLE_TESTS) + message (STATUS "Unit tests are enabled") +else() + message(STATUS "Unit tests are disabled") +endif () + +enable_testing() # Enable for tests without binary + +# when installing to /usr - place configs to /etc but for /usr/local place to /usr/local/etc +if (CMAKE_INSTALL_PREFIX STREQUAL "/usr") + set (CLICKHOUSE_ETC_DIR "/etc") +else () + set (CLICKHOUSE_ETC_DIR "${CMAKE_INSTALL_PREFIX}/etc") +endif () + +message (STATUS + "Building for: ${CMAKE_SYSTEM} ${CMAKE_SYSTEM_PROCESSOR} ${CMAKE_LIBRARY_ARCHITECTURE} ; + USE_STATIC_LIBRARIES=${USE_STATIC_LIBRARIES} + MAKE_STATIC_LIBRARIES=${MAKE_STATIC_LIBRARIES} + SPLIT_SHARED=${SPLIT_SHARED_LIBRARIES} + UNBUNDLED=${UNBUNDLED} + CCACHE=${CCACHE_FOUND} ${CCACHE_VERSION}") + +include (GNUInstallDirs) +include (cmake/contrib_finder.cmake) + +find_contrib_lib(double-conversion) # Must be before parquet +include (cmake/find/ssl.cmake) +include (cmake/find/ldap.cmake) # after ssl +include (cmake/find/icu.cmake) +include (cmake/find/xz.cmake) +include (cmake/find/zlib.cmake) +include (cmake/find/zstd.cmake) +include (cmake/find/ltdl.cmake) # for odbc +# openssl, zlib before poco +include (cmake/find/sparsehash.cmake) +include (cmake/find/re2.cmake) +include (cmake/find/krb5.cmake) +include (cmake/find/libgsasl.cmake) +include (cmake/find/cyrus-sasl.cmake) +include (cmake/find/rdkafka.cmake) +include (cmake/find/libuv.cmake) # for amqpcpp and cassandra +include (cmake/find/amqpcpp.cmake) +include (cmake/find/capnp.cmake) +include (cmake/find/llvm.cmake) +include (cmake/find/h3.cmake) +include (cmake/find/libxml2.cmake) +include (cmake/find/brotli.cmake) +include (cmake/find/protobuf.cmake) +include (cmake/find/grpc.cmake) +include (cmake/find/pdqsort.cmake) +include (cmake/find/miniselect.cmake) +include (cmake/find/hdfs3.cmake) # uses protobuf +include (cmake/find/poco.cmake) +include (cmake/find/curl.cmake) +include (cmake/find/s3.cmake) +include (cmake/find/base64.cmake) +include (cmake/find/parquet.cmake) +include (cmake/find/simdjson.cmake) +include (cmake/find/fast_float.cmake) +include (cmake/find/rapidjson.cmake) +include (cmake/find/fastops.cmake) +include (cmake/find/odbc.cmake) +include (cmake/find/nanodbc.cmake) +include (cmake/find/sqlite.cmake) +include (cmake/find/rocksdb.cmake) +include (cmake/find/libpqxx.cmake) +include (cmake/find/nuraft.cmake) +include (cmake/find/yaml-cpp.cmake) +include (cmake/find/s2geometry.cmake) + +if(NOT USE_INTERNAL_PARQUET_LIBRARY) + set (ENABLE_ORC OFF CACHE INTERNAL "") +endif() +include (cmake/find/orc.cmake) + +include (cmake/find/avro.cmake) +include (cmake/find/msgpack.cmake) +include (cmake/find/cassandra.cmake) +include (cmake/find/sentry.cmake) +include (cmake/find/stats.cmake) +include (cmake/find/datasketches.cmake) + +set (USE_INTERNAL_CITYHASH_LIBRARY ON CACHE INTERNAL "") +find_contrib_lib(cityhash) + +find_contrib_lib(farmhash) + +if (ENABLE_TESTS) + include (cmake/find/gtest.cmake) +endif () + +# Need to process before "contrib" dir: +include (cmake/find/mysqlclient.cmake) + +# When testing for memory leaks with Valgrind, don't link tcmalloc or jemalloc. + +include (cmake/print_flags.cmake) + +if (TARGET global-group) + install (EXPORT global DESTINATION cmake) +endif () + +add_subdirectory (contrib EXCLUDE_FROM_ALL) + +if (NOT ENABLE_JEMALLOC) + message (WARNING "Non default allocator is disabled. This is not recommended for production builds.") +endif () + +macro (add_executable target) + # invoke built-in add_executable + # explicitly acquire and interpose malloc symbols by clickhouse_malloc + # if GLIBC_COMPATIBILITY is ON and ENABLE_THINLTO is on than provide memcpy symbol explicitly to neutrialize thinlto's libcall generation. + if (GLIBC_COMPATIBILITY AND ENABLE_THINLTO) + _add_executable (${ARGV} $ $) + else () + _add_executable (${ARGV} $) + endif () + + get_target_property (type ${target} TYPE) + if (${type} STREQUAL EXECUTABLE) + # disabled for TSAN and gcc since libtsan.a provides overrides too + if (TARGET clickhouse_new_delete) + # operator::new/delete for executables (MemoryTracker stuff) + target_link_libraries (${target} PRIVATE clickhouse_new_delete ${MALLOC_LIBRARIES}) + endif() + endif() +endmacro() + +set(ConfigIncludePath ${CMAKE_CURRENT_BINARY_DIR}/includes/configs CACHE INTERNAL "Path to generated configuration files.") +include_directories(${ConfigIncludePath}) + +# Add as many warnings as possible for our own code. +include (cmake/warnings.cmake) + +# Check if needed compiler flags are supported +include (cmake/check_flags.cmake) + +add_subdirectory (base) +add_subdirectory (src) +add_subdirectory (programs) +add_subdirectory (tests) +add_subdirectory (utils) + +include (cmake/print_include_directories.cmake) + +include (cmake/sanitize_target_link_libraries.cmake) From 9461d6dbf21d3d5d54bdb862ace5be988c0b2d94 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Mon, 6 Sep 2021 18:22:06 +0800 Subject: [PATCH 110/317] modify code style --- .../MergeTree/MergeTreeIndexFullText.cpp | 34 ++++++++----------- 1 file changed, 15 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index f5460ec3f14..4f6edfc66c2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -414,10 +414,8 @@ bool MergeTreeConditionFullText::atomFromAST( //try to parse arrayElement function if (const auto * map_func = typeid_cast(args[0].get())) - { if (map_func->name == "arrayElement") const_value = assert_cast(map_func->arguments.get()->children[1].get())->name(); - } if (key_arg_pos == 1 && (func_name != "equals" && func_name != "notEquals")) return false; @@ -883,25 +881,23 @@ MergeTreeIndexPtr bloomFilterIndexCreator( void bloomFilterIndexValidator(const IndexDescription & index, bool /*attach*/) { - auto validate_map_type = [](DataTypePtr data_type){ - if (data_type->getTypeId() != TypeIndex::Map) - return false; - - DataTypeMap * map_type = assert_cast(const_cast(data_type.get())); - - if (map_type->getKeyType()->getTypeId() != TypeIndex::String - && map_type->getKeyType()->getTypeId() != TypeIndex::FixedString) - return false; - - return true; - }; - for (const auto & data_type : index.data_types) { - if (data_type->getTypeId() != TypeIndex::String - && data_type->getTypeId() != TypeIndex::FixedString - && !validate_map_type(data_type)) - throw Exception("Bloom filter index can be used only with `String`,`FixedString` or `Map` with key of String or fixedString type.", ErrorCodes::INCORRECT_QUERY); + if (data_type->getTypeId() != TypeIndex::String + && data_type->getTypeId() != TypeIndex::FixedString) + { + + if (data_type->getTypeId() != TypeIndex::Map) + throw Exception("Bloom filter index can be used only with `String`,`FixedString` or `Map` with key of String or fixedString type.", ErrorCodes::INCORRECT_QUERY); + + else + { + DataTypeMap * map_type = assert_cast(const_cast(data_type.get())); + + if (map_type->getKeyType()->getTypeId() != TypeIndex::String&& map_type->getKeyType()->getTypeId() != TypeIndex::FixedString) + throw Exception("Bloom filter index can be used only with `String`,`FixedString` or `Map` with key of String or fixedString type.", ErrorCodes::INCORRECT_QUERY); + } + } } if (index.type == NgramTokenExtractor::getName()) From e3e3b84f63328c051cd8c0ade34afad9dc59a943 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Sat, 18 Sep 2021 19:37:35 +0800 Subject: [PATCH 111/317] fix tests --- src/Storages/MergeTree/MergeTreeIndexFullText.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 4f6edfc66c2..d3e3790bdc6 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -360,11 +360,9 @@ bool MergeTreeConditionFullText::getKey(const ASTPtr & node, size_t & key_column String column_name = node->getColumnName(); //try to get map column name in arrayElement function - if (const auto * func = typeid_cast(node.get())) - { + if (const auto func = node.get()->as()) if (func->name == "arrayElement") column_name = assert_cast(func->arguments.get()->children[0].get())->name(); - } auto it = std::find(index_columns.begin(), index_columns.end(), column_name); if (it == index_columns.end()) @@ -413,10 +411,9 @@ bool MergeTreeConditionFullText::atomFromAST( } //try to parse arrayElement function - if (const auto * map_func = typeid_cast(args[0].get())) + if (const auto map_func = args[0].get()->as()) if (map_func->name == "arrayElement") - const_value = assert_cast(map_func->arguments.get()->children[1].get())->name(); - + const_value = assert_cast(map_func->arguments->children[1].get())->name(); if (key_arg_pos == 1 && (func_name != "equals" && func_name != "notEquals")) return false; else if (!token_extractor->supportLike() && (func_name == "like" || func_name == "notLike")) From 6dadc521fb2be8dab181a06536dabf465b262855 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 21 Sep 2021 16:43:33 +0300 Subject: [PATCH 112/317] MergeTreeIndexFullText updated for data type map --- .../MergeTree/MergeTreeIndexFullText.cpp | 87 +++++++++++-------- .../MergeTree/MergeTreeIndexFullText.h | 2 +- .../02000_bloom_filter_map.reference | 6 +- .../0_stateless/02000_bloom_filter_map.sql | 38 +++++--- 4 files changed, 85 insertions(+), 48 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index d3e3790bdc6..0b073ba43b8 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -165,16 +165,16 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, auto * column_map = assert_cast(const_cast(column.get())); auto & column_array = assert_cast(column_map->getNestedColumn()); auto & column_tuple = assert_cast(column_array.getData()); - auto & column_key = assert_cast(column_tuple.getColumn(0)); + auto & column_key = column_tuple.getColumn(0); for (size_t i = 0; i < rows_read; ++i) { - size_t element_start_row = *pos !=0 ? column_array.getOffsets()[*pos-1] : 0; + size_t element_start_row = column_array.getOffsets()[*pos - 1]; size_t elements_size = column_array.getOffsets()[*pos] - element_start_row; for (size_t row_num = 0; row_num < elements_size; row_num++) { - auto ref = column_key.getDataAt(element_start_row+row_num); + auto ref = column_key.getDataAt(element_start_row + row_num); columnToBloomFilter(ref.data, ref.size, token_extractor, granule->bloom_filters[col]); } @@ -355,16 +355,9 @@ bool MergeTreeConditionFullText::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx return rpn_stack[0].can_be_true; } -bool MergeTreeConditionFullText::getKey(const ASTPtr & node, size_t & key_column_num) +bool MergeTreeConditionFullText::getKey(const std::string & key_column_name, size_t & key_column_num) { - String column_name = node->getColumnName(); - - //try to get map column name in arrayElement function - if (const auto func = node.get()->as()) - if (func->name == "arrayElement") - column_name = assert_cast(func->arguments.get()->children[0].get())->name(); - - auto it = std::find(index_columns.begin(), index_columns.end(), column_name); + auto it = std::find(index_columns.begin(), index_columns.end(), key_column_name); if (it == index_columns.end()) return false; @@ -392,16 +385,50 @@ bool MergeTreeConditionFullText::atomFromAST( { key_arg_pos = 0; } - else if (KeyCondition::getConstant(args[1], block_with_constants, const_value, const_type) && getKey(args[0], key_column_num)) + else if (KeyCondition::getConstant(args[1], block_with_constants, const_value, const_type) && getKey(args[0]->getColumnName(), key_column_num)) { key_arg_pos = 0; } - else if (KeyCondition::getConstant(args[0], block_with_constants, const_value, const_type) && getKey(args[1], key_column_num)) + else if (KeyCondition::getConstant(args[0], block_with_constants, const_value, const_type) && getKey(args[1]->getColumnName(), key_column_num)) { key_arg_pos = 1; } + else if (const auto * index_function = args[0].get()->as()) + { + if (index_function->name == "arrayElement") + { + auto column_name = assert_cast(index_function->arguments.get()->children[0].get())->name(); + + if (!getKey(column_name, key_column_num)) + return false; + + key_arg_pos = 0; + + auto & argument = index_function->arguments.get()->children[1]; + + if (const auto * literal = argument->as()) + { + const_value = literal->value; + + if (const_value.getType() != Field::Types::String) + return false; + + const_type = std::make_shared(); + } + else + { + return false; + } + } + else + { + return false; + } + } else + { return false; + } if (const_type && const_type->getTypeId() != TypeIndex::String && const_type->getTypeId() != TypeIndex::FixedString @@ -409,11 +436,7 @@ bool MergeTreeConditionFullText::atomFromAST( { return false; } - - //try to parse arrayElement function - if (const auto map_func = args[0].get()->as()) - if (map_func->name == "arrayElement") - const_value = assert_cast(map_func->arguments->children[1].get())->name(); + if (key_arg_pos == 1 && (func_name != "equals" && func_name != "notEquals")) return false; else if (!token_extractor->supportLike() && (func_name == "like" || func_name == "notLike")) @@ -536,7 +559,7 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( for (size_t i = 0; i < tuple_elements.size(); ++i) { size_t key = 0; - if (getKey(tuple_elements[i], key)) + if (getKey(tuple_elements[i]->getColumnName(), key)) { key_tuple_mapping.emplace_back(i, key); data_types.push_back(index_data_types[key]); @@ -546,7 +569,7 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( else { size_t key = 0; - if (getKey(left_arg, key)) + if (getKey(left_arg->getColumnName(), key)) { key_tuple_mapping.emplace_back(0, key); data_types.push_back(index_data_types[key]); @@ -880,21 +903,17 @@ void bloomFilterIndexValidator(const IndexDescription & index, bool /*attach*/) { for (const auto & data_type : index.data_types) { - if (data_type->getTypeId() != TypeIndex::String - && data_type->getTypeId() != TypeIndex::FixedString) - { + DataTypePtr index_key_data_type = data_type; - if (data_type->getTypeId() != TypeIndex::Map) - throw Exception("Bloom filter index can be used only with `String`,`FixedString` or `Map` with key of String or fixedString type.", ErrorCodes::INCORRECT_QUERY); + if (data_type->getTypeId() == TypeIndex::Map) + { + DataTypeMap * map_type = assert_cast(const_cast(data_type.get())); + index_key_data_type = map_type->getKeyType(); + } - else - { - DataTypeMap * map_type = assert_cast(const_cast(data_type.get())); - - if (map_type->getKeyType()->getTypeId() != TypeIndex::String&& map_type->getKeyType()->getTypeId() != TypeIndex::FixedString) - throw Exception("Bloom filter index can be used only with `String`,`FixedString` or `Map` with key of String or fixedString type.", ErrorCodes::INCORRECT_QUERY); - } - } + if (index_key_data_type->getTypeId() != TypeIndex::String && index_key_data_type->getTypeId() != TypeIndex::FixedString) + throw Exception(ErrorCodes::INCORRECT_QUERY, + "Bloom filter index can be used only with `String`,`FixedString` or `Map` with key of `String` or `FixedString` type."); } if (index.type == NgramTokenExtractor::getName()) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.h b/src/Storages/MergeTree/MergeTreeIndexFullText.h index d34cbc61da2..404e10f86af 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.h +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.h @@ -147,7 +147,7 @@ private: bool atomFromAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out); - bool getKey(const ASTPtr & node, size_t & key_column_num); + bool getKey(const std::string & key_column_name, size_t & key_column_num); bool tryPrepareSetBloomFilter(const ASTs & args, RPNElement & out); static bool createFunctionEqualsCondition( diff --git a/tests/queries/0_stateless/02000_bloom_filter_map.reference b/tests/queries/0_stateless/02000_bloom_filter_map.reference index 0fb260e569f..93ca7fc2ff9 100644 --- a/tests/queries/0_stateless/02000_bloom_filter_map.reference +++ b/tests/queries/0_stateless/02000_bloom_filter_map.reference @@ -1,2 +1,4 @@ -3 {'K3':'V3'} -2 {'K2':'V2'} +1 {'K1':'V1'} {'K1':'V1'} +2 {'K2':'V2'} {'K2':'V2'} +1 {'K1':'V1'} {'K1':'V1'} +2 {'K2':'V2'} {'K2':'V2'} diff --git a/tests/queries/0_stateless/02000_bloom_filter_map.sql b/tests/queries/0_stateless/02000_bloom_filter_map.sql index 4b2a86b3682..488c369ec85 100644 --- a/tests/queries/0_stateless/02000_bloom_filter_map.sql +++ b/tests/queries/0_stateless/02000_bloom_filter_map.sql @@ -1,16 +1,32 @@ -CREATE DATABASE IF NOT EXISTS test; +DROP TABLE IF EXISTS bf_tokenbf_map_test; +DROP TABLE IF EXISTS bf_ngram_map_test; -DROP TABLE IF EXISTS test.bf_tokenbf_map_test; -DROP TABLE IF EXISTS test.bf_ngram_map_test; +CREATE TABLE bf_tokenbf_map_test +( + row_id UInt32, + map Map(String, String), + map_fixed Map(FixedString(2), String), + INDEX map_tokenbf map TYPE tokenbf_v1(256,2,0) GRANULARITY 1, + INDEX map_fixed_tokenbf map_fixed TYPE tokenbf_v1(256,2,0) GRANULARITY 1 +) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 2; -CREATE TABLE test.bf_tokenbf_map_test (row_id UInt32, map Map(String, String), INDEX map_tokenbf map TYPE tokenbf_v1(256,2,0) GRANULARITY 1) Engine=MergeTree() ORDER BY row_id settings index_granularity = 2; -CREATE TABLE test.bf_ngram_map_test (row_id UInt32, map Map(String, String), INDEX map_tokenbf map TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1) Engine=MergeTree() ORDER BY row_id settings index_granularity = 2; +CREATE TABLE bf_ngram_map_test +( + row_id UInt32, + map Map(String, String), + map_fixed Map(FixedString(2), String), + INDEX map_ngram map TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1, + INDEX map_fixed_ngram map_fixed TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1 +) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 2; -INSERT INTO test.bf_tokenbf_map_test VALUES (1, {'K1':'V1'}),(2,{'K2':'V2'}),(3,{'K3':'V3'}),(4,{'K4':'V4'}); -INSERT INTO test.bf_ngram_map_test VALUES (1, {'K1':'V1'}),(2,{'K2':'V2'}),(3,{'K3':'V3'}),(4,{'K4':'V4'}); +INSERT INTO bf_tokenbf_map_test VALUES (1, {'K1':'V1'}, {'K1':'V1'}), (2, {'K2':'V2'}, {'K2':'V2'}); +INSERT INTO bf_ngram_map_test VALUES (1, {'K1':'V1'}, {'K1':'V1'}), (2, {'K2':'V2'}, {'K2':'V2'}); -SELECT * FROM test.bf_tokenbf_map_test WHERE map['K3']='V3'; -SELECT * FROM test.bf_tokenbf_map_test WHERE map['K2']='V2'; +SELECT * FROM bf_tokenbf_map_test WHERE map['K1']='V1'; +SELECT * FROM bf_ngram_map_test WHERE map['K2']='V2'; -DROP TABLE test.bf_tokenbf_map_test; -DROP TABLE test.bf_ngram_map_test; \ No newline at end of file +SELECT * FROM bf_tokenbf_map_test WHERE map_fixed['K1']='V1'; +SELECT * FROM bf_ngram_map_test WHERE map_fixed['K2']='V2'; + +DROP TABLE bf_tokenbf_map_test; +DROP TABLE bf_ngram_map_test; From 43f3b0898c65b9e5e8d0647ca6eadfd459c5eb82 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 21 Sep 2021 16:57:05 +0300 Subject: [PATCH 113/317] Fixed build --- CMakeLists.txt | 63 +++++++++++++++++++++++++++++++++----------------- 1 file changed, 42 insertions(+), 21 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 875a6d1ab61..9c8903d853c 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -45,6 +45,7 @@ include (cmake/arch.cmake) include (cmake/target.cmake) include (cmake/tools.cmake) include (cmake/analysis.cmake) +include (cmake/git_status.cmake) # Ignore export() since we don't use it, # but it gets broken with a global targets via link_libraries() @@ -79,16 +80,16 @@ include (cmake/find/ccache.cmake) # ccache ignore it. option(ENABLE_CHECK_HEAVY_BUILDS "Don't allow C++ translation units to compile too long or to take too much memory while compiling." OFF) if (ENABLE_CHECK_HEAVY_BUILDS) - # set DATA (since RSS does not work since 2.6.x+) to 2G + # set DATA (since RSS does not work since 2.6.x+) to 5G set (RLIMIT_DATA 5000000000) # set VIRT (RLIMIT_AS) to 10G (DATA*10) set (RLIMIT_AS 10000000000) - # set CPU time limit to 600 seconds - set (RLIMIT_CPU 600) + # set CPU time limit to 1000 seconds + set (RLIMIT_CPU 1000) # gcc10/gcc10/clang -fsanitize=memory is too heavy if (SANITIZE STREQUAL "memory" OR COMPILER_GCC) - set (RLIMIT_DATA 10000000000) + set (RLIMIT_DATA 10000000000) # 10G endif() set (CMAKE_CXX_COMPILER_LAUNCHER prlimit --as=${RLIMIT_AS} --data=${RLIMIT_DATA} --cpu=${RLIMIT_CPU} ${CMAKE_CXX_COMPILER_LAUNCHER}) @@ -126,12 +127,13 @@ if (USE_STATIC_LIBRARIES) list(REVERSE CMAKE_FIND_LIBRARY_SUFFIXES) endif () -# Implies ${WITH_COVERAGE} option (ENABLE_FUZZING "Fuzzy testing using libfuzzer" OFF) if (ENABLE_FUZZING) + # Also set WITH_COVERAGE=1 for better fuzzing process + # By default this is disabled, because fuzzers are built in CI with the clickhouse itself. + # And we don't want to enable coverage for it. message (STATUS "Fuzzing instrumentation enabled") - set (WITH_COVERAGE ON) set (FUZZER "libfuzzer") endif() @@ -150,6 +152,7 @@ if (CMAKE_GENERATOR STREQUAL "Ninja" AND NOT DISABLE_COLORED_BUILD) set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fdiagnostics-color=always") endif () +include (cmake/check_flags.cmake) include (cmake/add_warning.cmake) if (NOT MSVC) @@ -163,6 +166,14 @@ if (COMPILER_CLANG) if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") set(COMPILER_FLAGS "${COMPILER_FLAGS} -gdwarf-aranges") endif () + + if (HAS_USE_CTOR_HOMING) + # For more info see https://blog.llvm.org/posts/2021-04-05-constructor-homing-for-debug-info/ + if (CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO") + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Xclang -fuse-ctor-homing") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -Xclang -fuse-ctor-homing") + endif() + endif() endif () # If turned `ON`, assumes the user has either the system GTest library or the bundled one. @@ -183,7 +194,7 @@ endif () # Make sure the final executable has symbols exported set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic") -find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-12" "llvm-objcopy-11" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy") +find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-13" "llvm-objcopy-12" "llvm-objcopy-11" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy") if (NOT OBJCOPY_PATH AND OS_DARWIN) find_program (BREW_PATH NAMES "brew") @@ -271,12 +282,6 @@ endif() include(cmake/cpu_features.cmake) -option(ARCH_NATIVE "Add -march=native compiler flag. This makes your binaries non-portable but more performant code may be generated.") - -if (ARCH_NATIVE) - set (COMPILER_FLAGS "${COMPILER_FLAGS} -march=native") -endif () - # Asynchronous unwind tables are needed for Query Profiler. # They are already by default on some platforms but possibly not on all platforms. # Enable it explicitly. @@ -376,7 +381,7 @@ if (COMPILER_CLANG) endif () # Always prefer llvm tools when using clang. For instance, we cannot use GNU ar when llvm LTO is enabled - find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") + find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-13" "llvm-ar-12" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") if (LLVM_AR_PATH) message(STATUS "Using llvm-ar: ${LLVM_AR_PATH}.") @@ -385,7 +390,7 @@ if (COMPILER_CLANG) message(WARNING "Cannot find llvm-ar. System ar will be used instead. It does not work with ThinLTO.") endif () - find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8") + find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-13" "llvm-ranlib-12" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8") if (LLVM_RANLIB_PATH) message(STATUS "Using llvm-ranlib: ${LLVM_RANLIB_PATH}.") @@ -401,9 +406,10 @@ endif () # Turns on all external libs like s3, kafka, ODBC, ... option(ENABLE_LIBRARIES "Enable all external libraries by default" ON) -# We recommend avoiding this mode for production builds because we can't guarantee all needed libraries exist in your -# system. +# We recommend avoiding this mode for production builds because we can't guarantee +# all needed libraries exist in your system. # This mode exists for enthusiastic developers who are searching for trouble. +# The whole idea of using unknown version of libraries from the OS distribution is deeply flawed. # Useful for maintainers of OS packages. option (UNBUNDLED "Use system libraries instead of ones in contrib/" OFF) @@ -542,6 +548,8 @@ include (cmake/find/libpqxx.cmake) include (cmake/find/nuraft.cmake) include (cmake/find/yaml-cpp.cmake) include (cmake/find/s2geometry.cmake) +include (cmake/find/nlp.cmake) +include (cmake/find/bzip2.cmake) if(NOT USE_INTERNAL_PARQUET_LIBRARY) set (ENABLE_ORC OFF CACHE INTERNAL "") @@ -596,7 +604,23 @@ macro (add_executable target) # disabled for TSAN and gcc since libtsan.a provides overrides too if (TARGET clickhouse_new_delete) # operator::new/delete for executables (MemoryTracker stuff) - target_link_libraries (${target} PRIVATE clickhouse_new_delete ${MALLOC_LIBRARIES}) + target_link_libraries (${target} PRIVATE clickhouse_new_delete) + endif() + + # In case of static jemalloc, because zone_register() is located in zone.c and + # is never used outside (it is declared as constructor) it is omitted + # by the linker, and so jemalloc will not be registered as system + # allocator under osx [1], and clickhouse will SIGSEGV. + # + # [1]: https://github.com/jemalloc/jemalloc/issues/708 + # + # About symbol name: + # - _zone_register not zone_register due to Mach-O binary format, + # - _je_zone_register due to JEMALLOC_PRIVATE_NAMESPACE=je_ under OS X. + # - but jemalloc-cmake does not run private_namespace.sh + # so symbol name should be _zone_register + if (ENABLE_JEMALLOC AND MAKE_STATIC_LIBRARIES AND OS_DARWIN) + set_property(TARGET ${target} APPEND PROPERTY LINK_OPTIONS -u_zone_register) endif() endif() endmacro() @@ -607,9 +631,6 @@ include_directories(${ConfigIncludePath}) # Add as many warnings as possible for our own code. include (cmake/warnings.cmake) -# Check if needed compiler flags are supported -include (cmake/check_flags.cmake) - add_subdirectory (base) add_subdirectory (src) add_subdirectory (programs) From 014876ffcc02670e62c72bed4673afe692a1f31e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 21 Sep 2021 18:23:00 +0300 Subject: [PATCH 114/317] Revert "wip" This reverts commit 33b95bfd7e81b95091a6f33d89eb27f2c38d2b02. --- docs/en/development/contrib.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/development/contrib.md b/docs/en/development/contrib.md index 7518dcc0098..803212330dd 100644 --- a/docs/en/development/contrib.md +++ b/docs/en/development/contrib.md @@ -100,9 +100,7 @@ SELECT library_name, license_type, license_path FROM system.licenses ORDER BY li 3. Branch from the branch you want to integrate, e.g., `master` -> `clickhouse/master`, or `release/vX.Y.Z` -> `clickhouse/release/vX.Y.Z`. 4. All forks in [Clickhouse-extras](https://github.com/ClickHouse-Extras) can be automatically synchronized with upstreams. `clickhouse/...` branches will remain unaffected, since virtually nobody is going to use that naming pattern in their upstream repos. 5. Add submodules under `contrib` of ClickHouse repo that refer the above forks/mirrors. Set the submodules to track the corresponding `clickhouse/...` branches. -6. Main branch(develop/master) is meant to be synced with upstream. If a patch has to be commited to a fork it should remain on top of main branch. Main branch update process is done via git rebase and allow a) to remove patches that were pushed to upstream and b) to maintain simple list of changes over upstream version. -7. Every time the custom changes have to be made in the library code, a dedicated branch should be created, like `clickhouse/my-fix`. Then this branch should be merged into the main branch in a fork when it is ready. -8. ClickHouse main repository shouls reference only commit in `clickhouse/`branches +6. Every time the custom changes have to be made in the library code, a dedicated branch should be created, like `clickhouse/my-fix`. Then this branch should be merged into the branch, that is tracked by the submodule, e.g., `clickhouse/master` or `clickhouse/release/vX.Y.Z`. 7. No code should be pushed in any branch of the forks in [Clickhouse-extras](https://github.com/ClickHouse-Extras), whose names do not follow `clickhouse/...` pattern. 8. Always write the custom changes with the official repo in mind. Once the PR is merged from (a feature/fix branch in) your personal fork into the fork in [Clickhouse-extras](https://github.com/ClickHouse-Extras), and the submodule is bumped in ClickHouse repo, consider opening another PR from (a feature/fix branch in) the fork in [Clickhouse-extras](https://github.com/ClickHouse-Extras) to the official repo of the library. This will make sure, that 1) the contribution has more than a single use case and importance, 2) others will also benefit from it, 3) the change will not remain a maintenance burden solely on ClickHouse developers. 9. When a submodule needs to start using a newer code from the original branch (e.g., `master`), and since the custom changes might be merged in the branch it is tracking (e.g., `clickhouse/master`) and so it may diverge from its original counterpart (i.e., `master`), a careful merge should be carried out first, i.e., `master` -> `clickhouse/master`, and only then the submodule can be bumped in ClickHouse. From c1f24cdd5e13eaf163345a5dccf2fc0da6d51556 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 21 Sep 2021 18:31:32 +0300 Subject: [PATCH 115/317] ro --- tests/integration/helpers/network.py | 3 ++- tests/integration/runner | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 4a745e97e22..decb406879e 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -212,7 +212,8 @@ class _NetworkManager: self._container = self._docker_client.containers.run('clickhouse/integration-helper', auto_remove=True, command=('sleep %s' % self.container_exit_timeout), - volumes={'/run/xtables.lock': {'bind': '/run/xtables.lock', 'mode': 'rw' }}, + # /run/xtables.lock passed inside for correct iptables --wait + volumes={'/run/xtables.lock': {'bind': '/run/xtables.lock', 'mode': 'ro' }}, detach=True, network_mode='host') container_id = self._container.id self._container_expire_time = time.time() + self.container_expire_timeout diff --git a/tests/integration/runner b/tests/integration/runner index ea3c5b5481a..10c940afb46 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -276,7 +276,7 @@ if __name__ == "__main__": --volume={library_bridge_bin}:/clickhouse-library-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ - --volume=/run/xtables.lock:/run/xtables.lock \ + --volume=/run/xtables.lock:/run/xtables.lock:ro \ {dockerd_internal_volume} -e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 \ {env_tags} {env_cleanup} -e PYTEST_OPTS='{parallel} {opts} {tests_list} -vvv' {img} {command}".format( net=net, From db6f1e198cc04a280a86a74a18cdd38e7bc57589 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 21 Sep 2021 19:37:32 +0300 Subject: [PATCH 116/317] 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 117/317] 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 84e37c2d87c9b7ddb3f578c866917ea28ab4b838 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 21 Sep 2021 21:28:52 +0300 Subject: [PATCH 118/317] Fixed tests --- src/Storages/MergeTree/MergeTreeIndexFullText.cpp | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 0b073ba43b8..d841356aef6 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -159,6 +159,8 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, { const auto & column = block.getByName(index_columns[col]).column; + size_t current_position = *pos; + if (column->getDataType() == TypeIndex::Map) { //update for the key of Map type @@ -169,8 +171,8 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, for (size_t i = 0; i < rows_read; ++i) { - size_t element_start_row = column_array.getOffsets()[*pos - 1]; - size_t elements_size = column_array.getOffsets()[*pos] - element_start_row; + size_t element_start_row = column_array.getOffsets()[current_position - 1]; + size_t elements_size = column_array.getOffsets()[current_position] - element_start_row; for (size_t row_num = 0; row_num < elements_size; row_num++) { @@ -178,22 +180,21 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, columnToBloomFilter(ref.data, ref.size, token_extractor, granule->bloom_filters[col]); } - *pos += 1; + current_position += 1; } } else { for (size_t i = 0; i < rows_read; ++i) { - auto ref = column->getDataAt(*pos + i); + auto ref = column->getDataAt(current_position + i); columnToBloomFilter(ref.data, ref.size, token_extractor, granule->bloom_filters[col]); } - - *pos += rows_read; } - } + granule->has_elems = true; + *pos += rows_read; } From 2c7239da66c7d807a2571d0c8219d5b2d0fdb26c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 22 Sep 2021 09:49:59 +0300 Subject: [PATCH 119/317] 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 120/317] 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 121/317] 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 122/317] 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 123/317] 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 124/317] 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 125/317] 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 126/317] 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 127/317] 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 128/317] 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 129/317] 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 155c9773a60e3428ae038b1b3781c776660379ca Mon Sep 17 00:00:00 2001 From: Mark Andreev Date: Thu, 23 Sep 2021 14:55:53 +0300 Subject: [PATCH 130/317] Fix table schema for hits_100m_obfuscated Add schema for hits_100m_obfuscated (copy from play.clickhouse.com instance) --- docs/en/getting-started/example-datasets/metrica.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/getting-started/example-datasets/metrica.md b/docs/en/getting-started/example-datasets/metrica.md index 159c99b15a0..296f18a8f83 100644 --- a/docs/en/getting-started/example-datasets/metrica.md +++ b/docs/en/getting-started/example-datasets/metrica.md @@ -42,7 +42,11 @@ md5sum hits_v1.tsv # Checksum should be equal to: f3631b6295bf06989c1437491f7592cb # now create table clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" +# for hits_v1 clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" +# for hits_100m_obfuscated +clickhouse-client --query="CREATE TABLE hits_100m_obfuscated (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, Refresh UInt8, RefererCategoryID UInt16, RefererRegionID UInt32, URLCategoryID UInt16, URLRegionID UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, OriginalURL String, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), LocalEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, RemoteIP UInt32, WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming UInt32, DNSTiming UInt32, ConnectTiming UInt32, ResponseStartTiming UInt32, ResponseEndTiming UInt32, FetchTiming UInt32, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" + # import data cat hits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.hits_v1 FORMAT TSV" --max_insert_block_size=100000 # optionally you can optimize table From 9af2e4891d030a9eeb561357e7ddb643f466c53a Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 23 Sep 2021 14:30:39 +0300 Subject: [PATCH 131/317] 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 132/317] 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 133/317] 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 134/317] 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 ad4f7498ad24468ee62e876e61581f81935e9276 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 23 Sep 2021 17:35:04 +0300 Subject: [PATCH 135/317] add comment --- tests/integration/helpers/cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f59bafa6109..2330f73eae5 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1746,6 +1746,7 @@ CLICKHOUSE_START_COMMAND = "clickhouse server --config-file=/etc/clickhouse-serv CLICKHOUSE_STAY_ALIVE_COMMAND = 'bash -c "{} --daemon; tail -f /dev/null"'.format(CLICKHOUSE_START_COMMAND) +# /run/xtables.lock passed inside for correct iptables --wait DOCKER_COMPOSE_TEMPLATE = ''' version: '2.3' services: From 604dc8861de14fb7f133904edfcae776a3dadd5c Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Fri, 24 Sep 2021 13:07:24 +0300 Subject: [PATCH 136/317] 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 137/317] 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 138/317] 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 139/317] 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 140/317] 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 141/317] 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 e3610dec922f4af44ca26b027adde38c6a2414c0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 24 Sep 2021 13:46:55 +0300 Subject: [PATCH 142/317] do not lose docker.log if dependencies failed to start before clickhouse up called --- tests/integration/helpers/cluster.py | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index af8e0ff0961..645e1b16172 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1434,6 +1434,7 @@ class ClickHouseCluster: for dir in self.zookeeper_dirs_to_create: os.makedirs(dir) run_and_check(self.base_zookeeper_cmd + common_opts, env=self.env) + self.up_called = True self.wait_zookeeper_secure_to_start() for command in self.pre_zookeeper_commands: @@ -1459,6 +1460,7 @@ class ClickHouseCluster: shutil.copy(os.path.join(HELPERS_DIR, f'keeper_config{i}.xml'), os.path.join(self.keeper_instance_dir_prefix + f"{i}", "config" )) run_and_check(self.base_zookeeper_cmd + common_opts, env=self.env) + self.up_called = True self.wait_zookeeper_to_start() for command in self.pre_zookeeper_commands: @@ -1476,6 +1478,7 @@ class ClickHouseCluster: os.makedirs(self.mysql_logs_dir) os.chmod(self.mysql_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql_cmd + common_opts) + self.up_called = True self.wait_mysql_to_start() if self.with_mysql8 and self.base_mysql8_cmd: @@ -1495,6 +1498,7 @@ class ClickHouseCluster: os.chmod(self.mysql_cluster_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql_cluster_cmd + common_opts) + self.up_called = True self.wait_mysql_cluster_to_start() if self.with_postgres and self.base_postgres_cmd: @@ -1505,6 +1509,7 @@ class ClickHouseCluster: os.chmod(self.postgres_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_postgres_cmd + common_opts) + self.up_called = True self.wait_postgres_to_start() if self.with_postgres_cluster and self.base_postgres_cluster_cmd: @@ -1516,17 +1521,20 @@ class ClickHouseCluster: os.makedirs(self.postgres4_logs_dir) os.chmod(self.postgres4_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_postgres_cluster_cmd + common_opts) + self.up_called = True self.wait_postgres_cluster_to_start() if self.with_kafka and self.base_kafka_cmd: logging.debug('Setup Kafka') subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) + self.up_called = True self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) self.wait_schema_registry_to_start() if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: logging.debug('Setup kerberized kafka') run_and_check(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes']) + self.up_called = True self.wait_kafka_is_available(self.kerberized_kafka_docker_id, self.kerberized_kafka_port, 100) if self.with_rabbitmq and self.base_rabbitmq_cmd: @@ -1536,6 +1544,7 @@ class ClickHouseCluster: for i in range(5): subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) + self.up_called = True self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') logging.debug(f"RabbitMQ checking container try: {i}") if self.wait_rabbitmq_to_start(throw=(i==4)): @@ -1546,6 +1555,7 @@ class ClickHouseCluster: os.makedirs(self.hdfs_logs_dir) os.chmod(self.hdfs_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_hdfs_cmd + common_opts) + self.up_called = True self.make_hdfs_api() self.wait_hdfs_to_start() @@ -1554,23 +1564,27 @@ class ClickHouseCluster: os.makedirs(self.hdfs_kerberized_logs_dir) os.chmod(self.hdfs_kerberized_logs_dir, stat.S_IRWXO) run_and_check(self.base_kerberized_hdfs_cmd + common_opts) + self.up_called = True self.make_hdfs_api(kerberized=True) self.wait_hdfs_to_start(check_marker=True) if self.with_nginx and self.base_nginx_cmd: logging.debug('Setup nginx') subprocess_check_call(self.base_nginx_cmd + common_opts + ['--renew-anon-volumes']) + self.up_called = True self.nginx_docker_id = self.get_instance_docker_id('nginx') self.wait_nginx_to_start() if self.with_mongo and self.base_mongo_cmd: logging.debug('Setup Mongo') run_and_check(self.base_mongo_cmd + common_opts) + self.up_called = True self.wait_mongo_to_start(30, secure=self.with_mongo_secure) if self.with_redis and self.base_redis_cmd: logging.debug('Setup Redis') subprocess_check_call(self.base_redis_cmd + common_opts) + self.up_called = True time.sleep(10) if self.with_minio and self.base_minio_cmd: @@ -1585,11 +1599,13 @@ class ClickHouseCluster: logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd))) run_and_check(minio_start_cmd) + self.up_called = True logging.info("Trying to connect to Minio...") self.wait_minio_to_start(secure=self.minio_certs_dir is not None) if self.with_cassandra and self.base_cassandra_cmd: subprocess_check_call(self.base_cassandra_cmd + ['up', '-d']) + self.up_called = True self.wait_cassandra_to_start() if self.with_jdbc_bridge and self.base_jdbc_bridge_cmd: @@ -1597,6 +1613,7 @@ class ClickHouseCluster: os.chmod(self.jdbc_driver_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_jdbc_bridge_cmd + ['up', '-d']) + self.up_called = True self.jdbc_bridge_ip = self.get_instance_ip(self.jdbc_bridge_host) self.wait_for_url(f"http://{self.jdbc_bridge_ip}:{self.jdbc_bridge_port}/ping") @@ -1665,6 +1682,8 @@ class ClickHouseCluster: subprocess_check_call(self.base_cmd + ['down', '--volumes']) except Exception as e: logging.debug("Down + remove orphans failed durung shutdown. {}".format(repr(e))) + else: + logging.warning("docker-compose up was not called. Trying to export docker.log for running containers") self.cleanup() From 455550ab031c342f10d37d6966a54919dd5db066 Mon Sep 17 00:00:00 2001 From: Alexey Date: Fri, 24 Sep 2021 11:44:29 +0000 Subject: [PATCH 143/317] en draft --- docs/en/sql-reference/functions/index.md | 55 +++++++++++++++++++++++- 1 file changed, 54 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/index.md b/docs/en/sql-reference/functions/index.md index 47da4e6f3cc..643ba660253 100644 --- a/docs/en/sql-reference/functions/index.md +++ b/docs/en/sql-reference/functions/index.md @@ -61,7 +61,60 @@ For some functions the first argument (the lambda function) can be omitted. In t ## User Defined Functions {#user-defined-functions} -Custom functions can be created using the [CREATE FUNCTION](../statements/create/function.md) statement. To delete these functions use the [DROP FUNCTION](../statements/drop.md#drop-function) statement. +Custom functions from lambda expressions can be created using the [CREATE FUNCTION](../statements/create/function.md) statement. To delete these functions use the [DROP FUNCTION](../statements/drop.md#drop-function) statement. + +The other option is to create functions using XML configuration. Add the path to a function configuration file inside `user_defined_executable_functions_config` tag. Wildcard symbol `*` may be used inside the path. +``` xml +*_function.xml +``` +Function configuration files are searched inside path specified by `user_files_path` setting. + +Function configuration contains: + +- `name` - a function name. +- `command` - a command or a script to execute. +- `argument` - argument description with the `type` of an argument. Each argument is described in a separate tag. +- `format` - The format in which arguments are passed to a command. +- `return_type` - the type of a value returned by the function. +- `type` - a function type. If it is set to `executable` then single command is started. If it is set to `executable_pool` then several commands are started. +- `lifetime` - reload interval in seconds. + +A function command must read arguments from STDIN and must output result to STDOUT. It must process arguments in a loop. + +**Example** +The following example creates `my_function`. It gets single argument of type String. `xargs` command listens to STDIN and calls `echo` for every argument. +``` + + + executable + my_function + + String + + String + TabSeparated + xargs -I arg echo Processing arg + 0 + + +``` + +Query: +`my_function` is available in queries. + +``` sql +SELECT number, my_function(toString(number)) FROM numbers(2); +``` + +Result: + +``` text +┌─number─┬─my_function(toString(number))─┐ +│ 0 │ Processing 0 │ +│ 1 │ Processing 1 │ +└────────┴───────────────────────────────┘ +``` + ## Error Handling {#error-handling} From 193f56656892f76e955e4f351e3af6c03ff5b7fb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 Sep 2021 14:54:39 +0300 Subject: [PATCH 144/317] 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 145/317] 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 146/317] 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 2b2bec3679df7965af908ce3f1e8e17e39bd12fe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Sep 2021 02:04:53 +0300 Subject: [PATCH 147/317] Make some changes to Python code that I don't understand --- tests/integration/test_rocksdb_options/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_rocksdb_options/test.py b/tests/integration/test_rocksdb_options/test.py index f7ed8071ca9..e8542749d8d 100644 --- a/tests/integration/test_rocksdb_options/test.py +++ b/tests/integration/test_rocksdb_options/test.py @@ -58,7 +58,7 @@ def test_valid_column_family_options(start_cluster): DROP TABLE test; """) -def test_invalid_column_family_options(): +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): @@ -68,13 +68,13 @@ def test_invalid_column_family_options(): 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(): +def test_table_valid_column_family_options(start_cluster): node.query(""" CREATE TABLE test (key UInt64, value String) Engine=EmbeddedRocksDB PRIMARY KEY(key); DROP TABLE test; """) -def test_table_invalid_column_family_options(): +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): From 3284b2c8de6182468abdb0a98321356b69d7bfb9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Sep 2021 02:10:10 +0300 Subject: [PATCH 148/317] Fix typo in comment --- src/Interpreters/Context.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f8718967aa3..118ab32f772 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -283,7 +283,7 @@ private: /// XXX: move this stuff to shared part instead. ContextMutablePtr buffer_context; /// Buffer context. Could be equal to this. - /// A flag, used to distinguish between user query and internal query to a database engine (MaterializePostgreSQL). + /// A flag, used to distinguish between user query and internal query to a database engine (MaterializedPostgreSQL). bool is_internal_query = false; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index db4b8a72a7d..53b99bc62ec 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -154,7 +154,9 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", serializeAST(*create.storage)); } - if (create.storage->engine->name == "Atomic" || create.storage->engine->name == "Replicated" || create.storage->engine->name == "MaterializedPostgreSQL") + if (create.storage->engine->name == "Atomic" + || create.storage->engine->name == "Replicated" + || create.storage->engine->name == "MaterializedPostgreSQL") { if (create.attach && create.uuid == UUIDHelpers::Nil) throw Exception(ErrorCodes::INCORRECT_QUERY, "UUID must be specified for ATTACH. " @@ -167,7 +169,8 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) if (!create.attach && fs::exists(metadata_path)) throw Exception(ErrorCodes::DATABASE_ALREADY_EXISTS, "Metadata directory {} already exists", metadata_path.string()); } - else if (create.storage->engine->name == "MaterializeMySQL" || create.storage->engine->name == "MaterializedMySQL") + else if (create.storage->engine->name == "MaterializeMySQL" + || create.storage->engine->name == "MaterializedMySQL") { /// It creates nested database with Ordinary or Atomic engine depending on UUID in query and default engine setting. /// Do nothing if it's an internal ATTACH on server startup or short-syntax ATTACH query from user, From a2c2b3a866438a8d60760ed505105b993c4fd11e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Sep 2021 02:12:26 +0300 Subject: [PATCH 149/317] Whitespaces --- src/Interpreters/InterpreterCreateQuery.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 53b99bc62ec..adac31e08e2 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -218,13 +218,17 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) "Enable allow_experimental_database_materialized_mysql to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); } - if (create.storage->engine->name == "Replicated" && !getContext()->getSettingsRef().allow_experimental_database_replicated && !internal) + if (create.storage->engine->name == "Replicated" + && !getContext()->getSettingsRef().allow_experimental_database_replicated + && !internal) { throw Exception("Replicated is an experimental database engine. " "Enable allow_experimental_database_replicated to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); } - if (create.storage->engine->name == "MaterializedPostgreSQL" && !getContext()->getSettingsRef().allow_experimental_database_materialized_postgresql && !internal) + if (create.storage->engine->name == "MaterializedPostgreSQL" + && !getContext()->getSettingsRef().allow_experimental_database_materialized_postgresql + && !internal) { throw Exception("MaterializedPostgreSQL is an experimental database engine. " "Enable allow_experimental_database_materialized_postgresql to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE); @@ -627,7 +631,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti else throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY); - /// Even if query has list of columns, canonicalize it (unfold Nested columns). if (!create.columns_list) create.set(create.columns_list, std::make_shared()); From af4e7cd642c0ef7fd1ff93003b223ba49b94a59b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 25 Sep 2021 02:17:39 +0300 Subject: [PATCH 150/317] Add lld into Build-Depends for debian package --- debian/control | 1 + 1 file changed, 1 insertion(+) diff --git a/debian/control b/debian/control index 633b7e7c8a3..ac75b00df22 100644 --- a/debian/control +++ b/debian/control @@ -7,6 +7,7 @@ Build-Depends: debhelper (>= 9), ninja-build, clang-13, llvm-13, + lld-13, libc6-dev, tzdata Standards-Version: 3.9.8 From 566797807f18e9965c1636221033cbd7221145a2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Sep 2021 02:32:03 +0300 Subject: [PATCH 151/317] Remove assorted trash --- src/Dictionaries/IDictionary.h | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index d7778cc0022..a62fb4a788e 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -1,22 +1,17 @@ #pragma once - #include -#include #include #include -#include -#include -#include -#include "IDictionarySource.h" +#include +#include #include #include -#include -#include #include #include + namespace DB { namespace ErrorCodes From 686989f4f13e7d4dfb815b82d5c08276d2833553 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Sep 2021 02:34:50 +0300 Subject: [PATCH 152/317] Remove some uglyness --- .../getDictionaryConfigurationFromAST.cpp | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 0ed5b3af83d..341a89bfc06 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -28,12 +28,14 @@ namespace ErrorCodes extern const int INCORRECT_DICTIONARY_DEFINITION; } + /// There are a lot of code, but it's very simple and straightforward -/// We just convert +/// We just perform conversion. namespace { using NamesToTypeNames = std::unordered_map; + /// Get value from field and convert it to string. /// Also remove quotes from strings. String getFieldAsString(const Field & field) @@ -46,8 +48,8 @@ String getFieldAsString(const Field & field) using namespace Poco; using namespace Poco::XML; -/* - * Transforms next definition + +/* Transforms next definition * LIFETIME(MIN 10, MAX 100) * to the next configuration * @@ -76,8 +78,7 @@ void buildLifetimeConfiguration( } } -/* - * Transforms next definition +/* Transforms next definition * LAYOUT(FLAT()) * to the next configuration * @@ -104,6 +105,7 @@ void buildLayoutConfiguration( layout_element->appendChild(layout_type_element); if (layout->parameters) + { for (const auto & param : layout->parameters->children) { const ASTPair * pair = param->as(); @@ -136,10 +138,11 @@ void buildLayoutConfiguration( layout_type_parameter_element->appendChild(value_to_append); layout_type_element->appendChild(layout_type_parameter_element); } + } } -/* - * Transforms next definition + +/* Transforms next definition * RANGE(MIN StartDate, MAX EndDate) * to the next configuration * StartDate @@ -216,8 +219,7 @@ void buildAttributeExpressionIfNeeded( } } -/** - * Transofrms single dictionary attribute to configuration +/** Transofrms single dictionary attribute to configuration * third_column UInt8 DEFAULT 2 EXPRESSION rand() % 100 * 77 * to * @@ -281,8 +283,7 @@ void buildSingleAttribute( } -/** - * Transforms +/** Transforms * PRIMARY KEY Attr1 ,..., AttrN * to the next configuration * Attr1 @@ -369,8 +370,7 @@ void buildPrimaryKeyConfiguration( } -/** - * Transforms list of ASTDictionaryAttributeDeclarations to list of dictionary attributes +/** Transforms list of ASTDictionaryAttributeDeclarations to list of dictionary attributes */ NamesToTypeNames buildDictionaryAttributesConfiguration( AutoPtr doc, From 2b195102880d49e6e8f080bd38ea45c69ee12ec6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Sep 2021 02:37:30 +0300 Subject: [PATCH 153/317] Apply a patch from Azat --- programs/install/Install.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 8013b6cf991..3a40ba077ed 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -962,7 +962,7 @@ namespace if (isRunning(pid_file)) { throw Exception(ErrorCodes::CANNOT_KILL, - "The server process still exists after %zu ms", + "The server process still exists after {} tries (delay: {} ms)", num_kill_check_tries, kill_check_delay_ms); } } From cd7f9d981c6aef41cf5d08b016d79d1634f98bd1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Sep 2021 04:22:54 +0300 Subject: [PATCH 154/317] Remove ya.make --- .arcignore | 12 - base/common/ya.make | 63 --- base/common/ya.make.in | 41 -- base/daemon/ya.make | 19 - base/loggers/ya.make | 19 - base/mysqlxx/ya.make | 39 -- base/mysqlxx/ya.make.in | 28 - base/pcg-random/ya.make | 7 - base/readpassphrase/ya.make | 11 - base/widechar_width/ya.make | 13 - base/ya.make | 11 - cmake/yandex/ya.make.versions.inc | 25 - programs/server/ya.make | 22 - programs/ya.make | 32 -- src/Access/ya.make | 54 -- src/Access/ya.make.in | 14 - src/AggregateFunctions/ya.make | 74 --- src/AggregateFunctions/ya.make.in | 14 - src/Backups/ya.make | 27 - src/Backups/ya.make.in | 14 - src/Bridge/ya.make | 17 - src/Bridge/ya.make.in | 14 - src/Client/ya.make | 24 - src/Client/ya.make.in | 15 - src/Columns/ya.make | 43 -- src/Columns/ya.make.in | 23 - src/Common/ya.make | 133 ----- src/Common/ya.make.in | 30 -- src/Compression/ya.make | 42 -- src/Compression/ya.make.in | 21 - src/Coordination/ya.make | 13 - src/Coordination/ya.make.in | 12 - src/Core/ya.make | 51 -- src/Core/ya.make.in | 16 - src/DataStreams/ya.make | 59 -- src/DataStreams/ya.make.in | 18 - src/DataTypes/ya.make | 74 --- src/DataTypes/ya.make.in | 15 - src/Databases/ya.make | 38 -- src/Databases/ya.make.in | 14 - src/Dictionaries/ya.make | 72 --- src/Dictionaries/ya.make.in | 25 - src/Disks/S3/ya.make | 17 - src/Disks/ya.make | 37 -- src/Disks/ya.make.in | 13 - src/Formats/ya.make | 30 -- src/Formats/ya.make.in | 16 - src/Functions/ya.make | 591 --------------------- src/Functions/ya.make.in | 43 -- src/IO/ya.make | 92 ---- src/IO/ya.make.in | 25 - src/Interpreters/ya.make | 199 ------- src/Interpreters/ya.make.in | 23 - src/Parsers/ya.make | 161 ------ src/Parsers/ya.make.in | 14 - src/Processors/ya.make | 192 ------- src/Processors/ya.make.in | 24 - src/Server/ya.make | 41 -- src/Server/ya.make.in | 15 - src/Storages/ya.make | 241 --------- src/Storages/ya.make.in | 16 - src/TableFunctions/ya.make | 36 -- src/TableFunctions/ya.make.in | 14 - src/ya.make | 32 -- utils/check-style/check-style | 20 - utils/generate-ya-make/generate-ya-make.sh | 16 - utils/github/ya.make | 13 - ya.make | 12 - 68 files changed, 3241 deletions(-) delete mode 100644 .arcignore delete mode 100644 base/common/ya.make delete mode 100644 base/common/ya.make.in delete mode 100644 base/daemon/ya.make delete mode 100644 base/loggers/ya.make delete mode 100644 base/mysqlxx/ya.make delete mode 100644 base/mysqlxx/ya.make.in delete mode 100644 base/pcg-random/ya.make delete mode 100644 base/readpassphrase/ya.make delete mode 100644 base/widechar_width/ya.make delete mode 100644 base/ya.make delete mode 100644 cmake/yandex/ya.make.versions.inc delete mode 100644 programs/server/ya.make delete mode 100644 programs/ya.make delete mode 100644 src/Access/ya.make delete mode 100644 src/Access/ya.make.in delete mode 100644 src/AggregateFunctions/ya.make delete mode 100644 src/AggregateFunctions/ya.make.in delete mode 100644 src/Backups/ya.make delete mode 100644 src/Backups/ya.make.in delete mode 100644 src/Bridge/ya.make delete mode 100644 src/Bridge/ya.make.in delete mode 100644 src/Client/ya.make delete mode 100644 src/Client/ya.make.in delete mode 100644 src/Columns/ya.make delete mode 100644 src/Columns/ya.make.in delete mode 100644 src/Common/ya.make delete mode 100644 src/Common/ya.make.in delete mode 100644 src/Compression/ya.make delete mode 100644 src/Compression/ya.make.in delete mode 100644 src/Coordination/ya.make delete mode 100644 src/Coordination/ya.make.in delete mode 100644 src/Core/ya.make delete mode 100644 src/Core/ya.make.in delete mode 100644 src/DataStreams/ya.make delete mode 100644 src/DataStreams/ya.make.in delete mode 100644 src/DataTypes/ya.make delete mode 100644 src/DataTypes/ya.make.in delete mode 100644 src/Databases/ya.make delete mode 100644 src/Databases/ya.make.in delete mode 100644 src/Dictionaries/ya.make delete mode 100644 src/Dictionaries/ya.make.in delete mode 100644 src/Disks/S3/ya.make delete mode 100644 src/Disks/ya.make delete mode 100644 src/Disks/ya.make.in delete mode 100644 src/Formats/ya.make delete mode 100644 src/Formats/ya.make.in delete mode 100644 src/Functions/ya.make delete mode 100644 src/Functions/ya.make.in delete mode 100644 src/IO/ya.make delete mode 100644 src/IO/ya.make.in delete mode 100644 src/Interpreters/ya.make delete mode 100644 src/Interpreters/ya.make.in delete mode 100644 src/Parsers/ya.make delete mode 100644 src/Parsers/ya.make.in delete mode 100644 src/Processors/ya.make delete mode 100644 src/Processors/ya.make.in delete mode 100644 src/Server/ya.make delete mode 100644 src/Server/ya.make.in delete mode 100644 src/Storages/ya.make delete mode 100644 src/Storages/ya.make.in delete mode 100644 src/TableFunctions/ya.make delete mode 100644 src/TableFunctions/ya.make.in delete mode 100644 src/ya.make delete mode 100755 utils/generate-ya-make/generate-ya-make.sh delete mode 100644 utils/github/ya.make delete mode 100644 ya.make diff --git a/.arcignore b/.arcignore deleted file mode 100644 index cd917b06fb7..00000000000 --- a/.arcignore +++ /dev/null @@ -1,12 +0,0 @@ -# .arcignore is the same as .gitignore but for Arc VCS. -# Arc VCS is a proprietary VCS in Yandex that is very similar to Git -# from the user perspective but with the following differences: -# 1. Data is stored in distributed object storage. -# 2. Local copy works via FUSE without downloading all the objects. -# For this reason, it is better suited for huge monorepositories that can be found in large companies (e.g. Yandex, Google). -# As ClickHouse developers, we don't use Arc as a VCS (we use Git). -# But the ClickHouse source code is also mirrored into internal monorepository and our collegues are using Arc. -# You can read more about Arc here: https://habr.com/en/company/yandex/blog/482926/ - -# Repository is synchronized without 3rd-party submodules. -contrib diff --git a/base/common/ya.make b/base/common/ya.make deleted file mode 100644 index fd88760c933..00000000000 --- a/base/common/ya.make +++ /dev/null @@ -1,63 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL( - GLOBAL clickhouse/base -) - -CFLAGS (GLOBAL -DARCADIA_BUILD) - -CFLAGS (GLOBAL -DUSE_CPUID=1) -CFLAGS (GLOBAL -DUSE_JEMALLOC=0) -CFLAGS (GLOBAL -DUSE_RAPIDJSON=1) -CFLAGS (GLOBAL -DUSE_SSL=1) - -IF (OS_DARWIN) - CFLAGS (GLOBAL -DOS_DARWIN) -ELSEIF (OS_FREEBSD) - CFLAGS (GLOBAL -DOS_FREEBSD) -ELSEIF (OS_LINUX) - CFLAGS (GLOBAL -DOS_LINUX) -ENDIF () - -PEERDIR( - contrib/libs/cctz - contrib/libs/cxxsupp/libcxx-filesystem - contrib/libs/poco/Net - contrib/libs/poco/Util - contrib/libs/poco/NetSSL_OpenSSL - contrib/libs/fmt - contrib/restricted/boost - contrib/restricted/cityhash-1.0.2 -) - -CFLAGS(-g0) - -SRCS( - DateLUT.cpp - DateLUTImpl.cpp - JSON.cpp - LineReader.cpp - StringRef.cpp - argsToConfig.cpp - coverage.cpp - demangle.cpp - errnoToString.cpp - getFQDNOrHostName.cpp - getMemoryAmount.cpp - getPageSize.cpp - getResource.cpp - getThreadId.cpp - mremap.cpp - phdr_cache.cpp - preciseExp10.cpp - setTerminalEcho.cpp - shift10.cpp - sleep.cpp - terminalColors.cpp - -) - -END() diff --git a/base/common/ya.make.in b/base/common/ya.make.in deleted file mode 100644 index 3deb36a2c71..00000000000 --- a/base/common/ya.make.in +++ /dev/null @@ -1,41 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL( - GLOBAL clickhouse/base -) - -CFLAGS (GLOBAL -DARCADIA_BUILD) - -CFLAGS (GLOBAL -DUSE_CPUID=1) -CFLAGS (GLOBAL -DUSE_JEMALLOC=0) -CFLAGS (GLOBAL -DUSE_RAPIDJSON=1) -CFLAGS (GLOBAL -DUSE_SSL=1) - -IF (OS_DARWIN) - CFLAGS (GLOBAL -DOS_DARWIN) -ELSEIF (OS_FREEBSD) - CFLAGS (GLOBAL -DOS_FREEBSD) -ELSEIF (OS_LINUX) - CFLAGS (GLOBAL -DOS_LINUX) -ENDIF () - -PEERDIR( - contrib/libs/cctz - contrib/libs/cxxsupp/libcxx-filesystem - contrib/libs/poco/Net - contrib/libs/poco/Util - contrib/libs/poco/NetSSL_OpenSSL - contrib/libs/fmt - contrib/restricted/boost - contrib/restricted/cityhash-1.0.2 -) - -CFLAGS(-g0) - -SRCS( - -) - -END() diff --git a/base/daemon/ya.make b/base/daemon/ya.make deleted file mode 100644 index f3b4059f002..00000000000 --- a/base/daemon/ya.make +++ /dev/null @@ -1,19 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -NO_COMPILER_WARNINGS() - -PEERDIR( - clickhouse/src/Common -) - -CFLAGS(-g0) - -SRCS( - BaseDaemon.cpp - GraphiteWriter.cpp - SentryWriter.cpp -) - -END() diff --git a/base/loggers/ya.make b/base/loggers/ya.make deleted file mode 100644 index 943b6f12b73..00000000000 --- a/base/loggers/ya.make +++ /dev/null @@ -1,19 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - -CFLAGS(-g0) - -SRCS( - ExtendedLogChannel.cpp - Loggers.cpp - OwnFormattingChannel.cpp - OwnPatternFormatter.cpp - OwnSplitChannel.cpp -) - -END() diff --git a/base/mysqlxx/ya.make b/base/mysqlxx/ya.make deleted file mode 100644 index aabc9922e72..00000000000 --- a/base/mysqlxx/ya.make +++ /dev/null @@ -1,39 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -LIBRARY() - -OWNER(g:clickhouse) - -CFLAGS(-g0) - -PEERDIR( - contrib/restricted/boost/libs - contrib/libs/libmysql_r - contrib/libs/poco/Foundation - contrib/libs/poco/Util -) - -ADDINCL( - GLOBAL clickhouse/base - clickhouse/base - contrib/libs/libmysql_r -) - -NO_COMPILER_WARNINGS() - -NO_UTIL() - -SRCS( - Connection.cpp - Exception.cpp - Pool.cpp - PoolFactory.cpp - PoolWithFailover.cpp - Query.cpp - ResultBase.cpp - Row.cpp - UseQueryResult.cpp - Value.cpp - -) - -END() diff --git a/base/mysqlxx/ya.make.in b/base/mysqlxx/ya.make.in deleted file mode 100644 index 10755078e20..00000000000 --- a/base/mysqlxx/ya.make.in +++ /dev/null @@ -1,28 +0,0 @@ -LIBRARY() - -OWNER(g:clickhouse) - -CFLAGS(-g0) - -PEERDIR( - contrib/restricted/boost/libs - contrib/libs/libmysql_r - contrib/libs/poco/Foundation - contrib/libs/poco/Util -) - -ADDINCL( - GLOBAL clickhouse/base - clickhouse/base - contrib/libs/libmysql_r -) - -NO_COMPILER_WARNINGS() - -NO_UTIL() - -SRCS( - -) - -END() diff --git a/base/pcg-random/ya.make b/base/pcg-random/ya.make deleted file mode 100644 index 705cdc05341..00000000000 --- a/base/pcg-random/ya.make +++ /dev/null @@ -1,7 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL (GLOBAL clickhouse/base/pcg-random) - -END() diff --git a/base/readpassphrase/ya.make b/base/readpassphrase/ya.make deleted file mode 100644 index d1ace8925ae..00000000000 --- a/base/readpassphrase/ya.make +++ /dev/null @@ -1,11 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -CFLAGS(-g0) - -SRCS( - readpassphrase.c -) - -END() diff --git a/base/widechar_width/ya.make b/base/widechar_width/ya.make deleted file mode 100644 index 0d61e0dbf70..00000000000 --- a/base/widechar_width/ya.make +++ /dev/null @@ -1,13 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL(GLOBAL clickhouse/base/widechar_width) - -CFLAGS(-g0) - -SRCS( - widechar_width.cpp -) - -END() diff --git a/base/ya.make b/base/ya.make deleted file mode 100644 index 19a16044280..00000000000 --- a/base/ya.make +++ /dev/null @@ -1,11 +0,0 @@ -OWNER(g:clickhouse) - -RECURSE( - common - daemon - loggers - mysqlxx - pcg-random - widechar_width - readpassphrase -) diff --git a/cmake/yandex/ya.make.versions.inc b/cmake/yandex/ya.make.versions.inc deleted file mode 100644 index 6910164d0aa..00000000000 --- a/cmake/yandex/ya.make.versions.inc +++ /dev/null @@ -1,25 +0,0 @@ -INCLUDE(${ARCADIA_ROOT}/clickhouse/cmake/autogenerated_versions.txt) - -# TODO: not sure if this is customizable per-binary -SET(VERSION_NAME "ClickHouse") - -# TODO: not quite sure how to replace dash with space in ya.make -SET(VERSION_FULL "${VERSION_NAME}-${VERSION_STRING}") - -CFLAGS (GLOBAL -DDBMS_NAME=\"ClickHouse\") -CFLAGS (GLOBAL -DDBMS_VERSION_MAJOR=${VERSION_MAJOR}) -CFLAGS (GLOBAL -DDBMS_VERSION_MINOR=${VERSION_MINOR}) -CFLAGS (GLOBAL -DDBMS_VERSION_PATCH=${VERSION_PATCH}) -CFLAGS (GLOBAL -DVERSION_FULL=\"\\\"${VERSION_FULL}\\\"\") -CFLAGS (GLOBAL -DVERSION_MAJOR=${VERSION_MAJOR}) -CFLAGS (GLOBAL -DVERSION_MINOR=${VERSION_MINOR}) -CFLAGS (GLOBAL -DVERSION_PATCH=${VERSION_PATCH}) - -# TODO: not supported yet, not sure if ya.make supports arithmetic. -CFLAGS (GLOBAL -DVERSION_INTEGER=0) - -CFLAGS (GLOBAL -DVERSION_NAME=\"\\\"${VERSION_NAME}\\\"\") -CFLAGS (GLOBAL -DVERSION_OFFICIAL=\"-arcadia\") -CFLAGS (GLOBAL -DVERSION_REVISION=${VERSION_REVISION}) -CFLAGS (GLOBAL -DVERSION_STRING=\"\\\"${VERSION_STRING}\\\"\") - diff --git a/programs/server/ya.make b/programs/server/ya.make deleted file mode 100644 index 1b99fb31a3f..00000000000 --- a/programs/server/ya.make +++ /dev/null @@ -1,22 +0,0 @@ -OWNER(g:clickhouse) - -PROGRAM(clickhouse-server) - -PEERDIR( - clickhouse/base/common - clickhouse/base/daemon - clickhouse/base/loggers - clickhouse/src - contrib/libs/poco/NetSSL_OpenSSL -) - -CFLAGS(-g0) - -SRCS( - clickhouse-server.cpp - - MetricsTransmitter.cpp - Server.cpp -) - -END() diff --git a/programs/ya.make b/programs/ya.make deleted file mode 100644 index 66a0b076442..00000000000 --- a/programs/ya.make +++ /dev/null @@ -1,32 +0,0 @@ -OWNER(g:clickhouse) - -PROGRAM(clickhouse) - -CFLAGS( - -DENABLE_CLICKHOUSE_CLIENT - -DENABLE_CLICKHOUSE_EXTRACT_FROM_CONFIG - -DENABLE_CLICKHOUSE_SERVER -) - -PEERDIR( - clickhouse/base/daemon - clickhouse/base/loggers - clickhouse/src -) - -CFLAGS(-g0) - -SRCS( - main.cpp - - client/Client.cpp - client/QueryFuzzer.cpp - client/ConnectionParameters.cpp - client/Suggest.cpp - client/TestHint.cpp - extract-from-config/ExtractFromConfig.cpp - server/Server.cpp - server/MetricsTransmitter.cpp -) - -END() diff --git a/src/Access/ya.make b/src/Access/ya.make deleted file mode 100644 index 8a1f96bb870..00000000000 --- a/src/Access/ya.make +++ /dev/null @@ -1,54 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - AccessControlManager.cpp - AccessEntityIO.cpp - AccessRights.cpp - AccessRightsElement.cpp - AllowedClientHosts.cpp - Authentication.cpp - ContextAccess.cpp - Credentials.cpp - DiskAccessStorage.cpp - EnabledQuota.cpp - EnabledRoles.cpp - EnabledRolesInfo.cpp - EnabledRowPolicies.cpp - EnabledSettings.cpp - ExternalAuthenticators.cpp - GSSAcceptor.cpp - GrantedRoles.cpp - IAccessEntity.cpp - IAccessStorage.cpp - LDAPAccessStorage.cpp - LDAPClient.cpp - MemoryAccessStorage.cpp - MultipleAccessStorage.cpp - Quota.cpp - QuotaCache.cpp - QuotaUsage.cpp - ReplicatedAccessStorage.cpp - Role.cpp - RoleCache.cpp - RolesOrUsersSet.cpp - RowPolicy.cpp - RowPolicyCache.cpp - SettingsConstraints.cpp - SettingsProfile.cpp - SettingsProfileElement.cpp - SettingsProfilesCache.cpp - SettingsProfilesInfo.cpp - User.cpp - UsersConfigAccessStorage.cpp - -) - -END() diff --git a/src/Access/ya.make.in b/src/Access/ya.make.in deleted file mode 100644 index 7d91972194d..00000000000 --- a/src/Access/ya.make.in +++ /dev/null @@ -1,14 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - -) - -END() diff --git a/src/AggregateFunctions/ya.make b/src/AggregateFunctions/ya.make deleted file mode 100644 index e18ec35c540..00000000000 --- a/src/AggregateFunctions/ya.make +++ /dev/null @@ -1,74 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - AggregateFunctionAggThrow.cpp - AggregateFunctionAny.cpp - AggregateFunctionArray.cpp - AggregateFunctionAvg.cpp - AggregateFunctionAvgWeighted.cpp - AggregateFunctionBitwise.cpp - AggregateFunctionBoundingRatio.cpp - AggregateFunctionCategoricalInformationValue.cpp - AggregateFunctionCombinatorFactory.cpp - AggregateFunctionCount.cpp - AggregateFunctionDeltaSum.cpp - AggregateFunctionDeltaSumTimestamp.cpp - AggregateFunctionDistinct.cpp - AggregateFunctionEntropy.cpp - AggregateFunctionFactory.cpp - AggregateFunctionForEach.cpp - AggregateFunctionGroupArray.cpp - AggregateFunctionGroupArrayInsertAt.cpp - AggregateFunctionGroupArrayMoving.cpp - AggregateFunctionGroupUniqArray.cpp - AggregateFunctionHistogram.cpp - AggregateFunctionIf.cpp - AggregateFunctionIntervalLengthSum.cpp - AggregateFunctionMLMethod.cpp - AggregateFunctionMannWhitney.cpp - AggregateFunctionMax.cpp - AggregateFunctionMaxIntersections.cpp - AggregateFunctionMerge.cpp - AggregateFunctionMin.cpp - AggregateFunctionNull.cpp - AggregateFunctionOrFill.cpp - AggregateFunctionQuantile.cpp - AggregateFunctionRankCorrelation.cpp - AggregateFunctionResample.cpp - AggregateFunctionRetention.cpp - AggregateFunctionSequenceMatch.cpp - AggregateFunctionSequenceNextNode.cpp - AggregateFunctionSimpleLinearRegression.cpp - AggregateFunctionSimpleState.cpp - AggregateFunctionSingleValueOrNull.cpp - AggregateFunctionSparkbar.cpp - AggregateFunctionState.cpp - AggregateFunctionStatistics.cpp - AggregateFunctionStatisticsSimple.cpp - AggregateFunctionStudentTTest.cpp - AggregateFunctionSum.cpp - AggregateFunctionSumCount.cpp - AggregateFunctionSumMap.cpp - AggregateFunctionTopK.cpp - AggregateFunctionUniq.cpp - AggregateFunctionUniqCombined.cpp - AggregateFunctionUniqUpTo.cpp - AggregateFunctionWelchTTest.cpp - AggregateFunctionWindowFunnel.cpp - IAggregateFunction.cpp - UniqCombinedBiasData.cpp - UniqVariadicHash.cpp - parseAggregateFunctionParameters.cpp - registerAggregateFunctions.cpp - -) - -END() diff --git a/src/AggregateFunctions/ya.make.in b/src/AggregateFunctions/ya.make.in deleted file mode 100644 index 78d82238168..00000000000 --- a/src/AggregateFunctions/ya.make.in +++ /dev/null @@ -1,14 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - -) - -END() diff --git a/src/Backups/ya.make b/src/Backups/ya.make deleted file mode 100644 index 96217a56ef0..00000000000 --- a/src/Backups/ya.make +++ /dev/null @@ -1,27 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - BackupEntryConcat.cpp - BackupEntryFromAppendOnlyFile.cpp - BackupEntryFromImmutableFile.cpp - BackupEntryFromMemory.cpp - BackupEntryFromSmallFile.cpp - BackupFactory.cpp - BackupInDirectory.cpp - BackupRenamingConfig.cpp - BackupSettings.cpp - BackupUtils.cpp - hasCompatibleDataToRestoreTable.cpp - renameInCreateQuery.cpp - -) - -END() diff --git a/src/Backups/ya.make.in b/src/Backups/ya.make.in deleted file mode 100644 index 7d91972194d..00000000000 --- a/src/Backups/ya.make.in +++ /dev/null @@ -1,14 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - -) - -END() diff --git a/src/Bridge/ya.make b/src/Bridge/ya.make deleted file mode 100644 index e900bab036e..00000000000 --- a/src/Bridge/ya.make +++ /dev/null @@ -1,17 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - IBridgeHelper.cpp - LibraryBridgeHelper.cpp - -) - -END() diff --git a/src/Bridge/ya.make.in b/src/Bridge/ya.make.in deleted file mode 100644 index 7d91972194d..00000000000 --- a/src/Bridge/ya.make.in +++ /dev/null @@ -1,14 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - -) - -END() diff --git a/src/Client/ya.make b/src/Client/ya.make deleted file mode 100644 index 88fa14ad377..00000000000 --- a/src/Client/ya.make +++ /dev/null @@ -1,24 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/poco/NetSSL_OpenSSL -) - - -SRCS( - Connection.cpp - ConnectionEstablisher.cpp - ConnectionPool.cpp - ConnectionPoolWithFailover.cpp - HedgedConnections.cpp - HedgedConnectionsFactory.cpp - IConnections.cpp - MultiplexedConnections.cpp - -) - -END() diff --git a/src/Client/ya.make.in b/src/Client/ya.make.in deleted file mode 100644 index ccf46673c66..00000000000 --- a/src/Client/ya.make.in +++ /dev/null @@ -1,15 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/poco/NetSSL_OpenSSL -) - - -SRCS( - -) - -END() diff --git a/src/Columns/ya.make b/src/Columns/ya.make deleted file mode 100644 index d5b5047dc25..00000000000 --- a/src/Columns/ya.make +++ /dev/null @@ -1,43 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL( - contrib/libs/icu/common - contrib/libs/icu/i18n - contrib/libs/pdqsort - contrib/libs/lz4 -) - -PEERDIR( - clickhouse/src/Common - contrib/libs/icu - contrib/libs/pdqsort - contrib/libs/lz4 -) - -SRCS( - Collator.cpp - ColumnAggregateFunction.cpp - ColumnArray.cpp - ColumnCompressed.cpp - ColumnConst.cpp - ColumnDecimal.cpp - ColumnFixedString.cpp - ColumnFunction.cpp - ColumnLowCardinality.cpp - ColumnMap.cpp - ColumnNullable.cpp - ColumnString.cpp - ColumnTuple.cpp - ColumnVector.cpp - ColumnsCommon.cpp - FilterDescription.cpp - IColumn.cpp - MaskOperations.cpp - getLeastSuperColumn.cpp - -) - -END() diff --git a/src/Columns/ya.make.in b/src/Columns/ya.make.in deleted file mode 100644 index 5d2b0f3d06a..00000000000 --- a/src/Columns/ya.make.in +++ /dev/null @@ -1,23 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL( - contrib/libs/icu/common - contrib/libs/icu/i18n - contrib/libs/pdqsort - contrib/libs/lz4 -) - -PEERDIR( - clickhouse/src/Common - contrib/libs/icu - contrib/libs/pdqsort - contrib/libs/lz4 -) - -SRCS( - -) - -END() diff --git a/src/Common/ya.make b/src/Common/ya.make deleted file mode 100644 index e130fc7ebea..00000000000 --- a/src/Common/ya.make +++ /dev/null @@ -1,133 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL ( - GLOBAL clickhouse/src - contrib/libs/libcpuid - contrib/libs/libunwind/include - GLOBAL contrib/restricted/dragonbox -) - -PEERDIR( - clickhouse/base/common - clickhouse/base/pcg-random - clickhouse/base/widechar_width - contrib/libs/libcpuid - contrib/libs/openssl - contrib/libs/poco/NetSSL_OpenSSL - contrib/libs/re2 - contrib/restricted/dragonbox -) - -INCLUDE(${ARCADIA_ROOT}/clickhouse/cmake/yandex/ya.make.versions.inc) - - -SRCS( - ActionLock.cpp - AlignedBuffer.cpp - Allocator.cpp - ClickHouseRevision.cpp - Config/AbstractConfigurationComparison.cpp - Config/ConfigProcessor.cpp - Config/ConfigReloader.cpp - Config/YAMLParser.cpp - Config/configReadClient.cpp - CurrentMemoryTracker.cpp - CurrentMetrics.cpp - CurrentThread.cpp - DNSResolver.cpp - Dwarf.cpp - Elf.cpp - Epoll.cpp - ErrorCodes.cpp - Exception.cpp - FieldVisitorDump.cpp - FieldVisitorHash.cpp - FieldVisitorSum.cpp - FieldVisitorToString.cpp - FieldVisitorWriteBinary.cpp - FileChecker.cpp - IO.cpp - IPv6ToBinary.cpp - IntervalKind.cpp - JSONBuilder.cpp - Macros.cpp - MemoryStatisticsOS.cpp - MemoryTracker.cpp - OpenSSLHelpers.cpp - OptimizedRegularExpression.cpp - PODArray.cpp - PipeFDs.cpp - ProcfsMetricsProvider.cpp - ProfileEvents.cpp - ProgressIndication.cpp - QueryProfiler.cpp - RWLock.cpp - RemoteHostFilter.cpp - SensitiveDataMasker.cpp - SettingsChanges.cpp - SharedLibrary.cpp - ShellCommand.cpp - StackTrace.cpp - StatusFile.cpp - StatusInfo.cpp - Stopwatch.cpp - StringUtils/StringUtils.cpp - StudentTTest.cpp - SymbolIndex.cpp - TLDListsHolder.cpp - TaskStatsInfoGetter.cpp - TerminalSize.cpp - ThreadFuzzer.cpp - ThreadPool.cpp - ThreadProfileEvents.cpp - ThreadStatus.cpp - Throttler.cpp - TimerDescriptor.cpp - TraceCollector.cpp - UTF8Helpers.cpp - UnicodeBar.cpp - VersionNumber.cpp - WeakHash.cpp - ZooKeeper/IKeeper.cpp - ZooKeeper/TestKeeper.cpp - ZooKeeper/ZooKeeper.cpp - ZooKeeper/ZooKeeperCommon.cpp - ZooKeeper/ZooKeeperConstants.cpp - ZooKeeper/ZooKeeperIO.cpp - ZooKeeper/ZooKeeperImpl.cpp - ZooKeeper/ZooKeeperNodeCache.cpp - checkStackSize.cpp - clearPasswordFromCommandLine.cpp - clickhouse_malloc.cpp - createHardLink.cpp - escapeForFileName.cpp - filesystemHelpers.cpp - formatIPv6.cpp - formatReadable.cpp - getExecutablePath.cpp - getHashOfLoadedBinary.cpp - getMappedArea.cpp - getMultipleKeysFromConfig.cpp - getNumberOfPhysicalCPUCores.cpp - hasLinuxCapability.cpp - hex.cpp - isLocalAddress.cpp - isValidUTF8.cpp - malloc.cpp - new_delete.cpp - parseAddress.cpp - parseGlobs.cpp - parseRemoteDescription.cpp - quoteString.cpp - randomSeed.cpp - remapExecutable.cpp - renameat2.cpp - setThreadName.cpp - thread_local_rng.cpp - -) - -END() diff --git a/src/Common/ya.make.in b/src/Common/ya.make.in deleted file mode 100644 index 9fd6c4b4708..00000000000 --- a/src/Common/ya.make.in +++ /dev/null @@ -1,30 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL ( - GLOBAL clickhouse/src - contrib/libs/libcpuid - contrib/libs/libunwind/include - GLOBAL contrib/restricted/dragonbox -) - -PEERDIR( - clickhouse/base/common - clickhouse/base/pcg-random - clickhouse/base/widechar_width - contrib/libs/libcpuid - contrib/libs/openssl - contrib/libs/poco/NetSSL_OpenSSL - contrib/libs/re2 - contrib/restricted/dragonbox -) - -INCLUDE(${ARCADIA_ROOT}/clickhouse/cmake/yandex/ya.make.versions.inc) - - -SRCS( - -) - -END() diff --git a/src/Compression/ya.make b/src/Compression/ya.make deleted file mode 100644 index 8ba48696225..00000000000 --- a/src/Compression/ya.make +++ /dev/null @@ -1,42 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL( - contrib/libs/lz4 - contrib/libs/zstd/include -) - -PEERDIR( - clickhouse/src/Common - contrib/libs/lz4 - contrib/libs/zstd -) - - -SRCS( - CachedCompressedReadBuffer.cpp - CheckingCompressedReadBuffer.cpp - CompressedReadBuffer.cpp - CompressedReadBufferBase.cpp - CompressedReadBufferFromFile.cpp - CompressedWriteBuffer.cpp - CompressionCodecDelta.cpp - CompressionCodecDoubleDelta.cpp - CompressionCodecEncrypted.cpp - CompressionCodecGorilla.cpp - CompressionCodecLZ4.cpp - CompressionCodecMultiple.cpp - CompressionCodecNone.cpp - CompressionCodecT64.cpp - CompressionCodecZSTD.cpp - CompressionFactory.cpp - CompressionFactoryAdditions.cpp - ICompressionCodec.cpp - LZ4_decompress_faster.cpp - getCompressionCodecForFile.cpp - -) - -END() diff --git a/src/Compression/ya.make.in b/src/Compression/ya.make.in deleted file mode 100644 index ec4bf556901..00000000000 --- a/src/Compression/ya.make.in +++ /dev/null @@ -1,21 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL( - contrib/libs/lz4 - contrib/libs/zstd/include -) - -PEERDIR( - clickhouse/src/Common - contrib/libs/lz4 - contrib/libs/zstd -) - - -SRCS( - -) - -END() diff --git a/src/Coordination/ya.make b/src/Coordination/ya.make deleted file mode 100644 index f3eae68806c..00000000000 --- a/src/Coordination/ya.make +++ /dev/null @@ -1,13 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - -SRCS( -) - -END() diff --git a/src/Coordination/ya.make.in b/src/Coordination/ya.make.in deleted file mode 100644 index ba5f8bcbea4..00000000000 --- a/src/Coordination/ya.make.in +++ /dev/null @@ -1,12 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - -SRCS( -) - -END() diff --git a/src/Core/ya.make b/src/Core/ya.make deleted file mode 100644 index 2362f88e5f6..00000000000 --- a/src/Core/ya.make +++ /dev/null @@ -1,51 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/sparsehash - contrib/restricted/boost/libs -) - - -SRCS( - BackgroundSchedulePool.cpp - BaseSettings.cpp - Block.cpp - BlockInfo.cpp - ColumnWithTypeAndName.cpp - ExternalResultDescription.cpp - ExternalTable.cpp - Field.cpp - MySQL/Authentication.cpp - MySQL/IMySQLReadPacket.cpp - MySQL/IMySQLWritePacket.cpp - MySQL/MySQLClient.cpp - MySQL/MySQLGtid.cpp - MySQL/MySQLReplication.cpp - MySQL/PacketEndpoint.cpp - MySQL/PacketsConnection.cpp - MySQL/PacketsGeneric.cpp - MySQL/PacketsProtocolText.cpp - MySQL/PacketsReplication.cpp - NamesAndTypes.cpp - PostgreSQL/Connection.cpp - PostgreSQL/PoolWithFailover.cpp - PostgreSQL/Utils.cpp - PostgreSQL/insertPostgreSQLValue.cpp - PostgreSQLProtocol.cpp - QueryProcessingStage.cpp - ServerUUID.cpp - Settings.cpp - SettingsEnums.cpp - SettingsFields.cpp - SettingsQuirks.cpp - SortDescription.cpp - UUID.cpp - iostream_debug_helpers.cpp - -) - -END() diff --git a/src/Core/ya.make.in b/src/Core/ya.make.in deleted file mode 100644 index 2938fb13e9a..00000000000 --- a/src/Core/ya.make.in +++ /dev/null @@ -1,16 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/sparsehash - contrib/restricted/boost/libs -) - - -SRCS( - -) - -END() diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make deleted file mode 100644 index 7f16e2360c9..00000000000 --- a/src/DataStreams/ya.make +++ /dev/null @@ -1,59 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/poco/MongoDB - contrib/restricted/boost/libs -) - -NO_COMPILER_WARNINGS() - - -SRCS( - AddingDefaultBlockOutputStream.cpp - AsynchronousBlockInputStream.cpp - BlockIO.cpp - BlockStreamProfileInfo.cpp - CheckConstraintsBlockOutputStream.cpp - ColumnGathererStream.cpp - ConnectionCollector.cpp - ConvertingBlockInputStream.cpp - CountingBlockOutputStream.cpp - DistinctSortedBlockInputStream.cpp - ExecutionSpeedLimits.cpp - ExpressionBlockInputStream.cpp - IBlockInputStream.cpp - ITTLAlgorithm.cpp - InternalTextLogsRowOutputStream.cpp - MaterializingBlockInputStream.cpp - MongoDBSource.cpp - NativeBlockInputStream.cpp - NativeBlockOutputStream.cpp - PushingToViewsBlockOutputStream.cpp - RemoteBlockInputStream.cpp - RemoteBlockOutputStream.cpp - RemoteQueryExecutor.cpp - RemoteQueryExecutorReadContext.cpp - SQLiteSource.cpp - SizeLimits.cpp - SquashingBlockInputStream.cpp - SquashingBlockOutputStream.cpp - SquashingTransform.cpp - TTLAggregationAlgorithm.cpp - TTLBlockInputStream.cpp - TTLCalcInputStream.cpp - TTLColumnAlgorithm.cpp - TTLDeleteAlgorithm.cpp - TTLUpdateInfoAlgorithm.cpp - copyData.cpp - finalizeBlock.cpp - formatBlock.cpp - materializeBlock.cpp - narrowBlockInputStreams.cpp - -) - -END() diff --git a/src/DataStreams/ya.make.in b/src/DataStreams/ya.make.in deleted file mode 100644 index 6969f644307..00000000000 --- a/src/DataStreams/ya.make.in +++ /dev/null @@ -1,18 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/poco/MongoDB - contrib/restricted/boost/libs -) - -NO_COMPILER_WARNINGS() - - -SRCS( - -) - -END() diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make deleted file mode 100644 index ab34e467795..00000000000 --- a/src/DataTypes/ya.make +++ /dev/null @@ -1,74 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - clickhouse/src/Formats -) - - -SRCS( - DataTypeAggregateFunction.cpp - DataTypeArray.cpp - DataTypeCustomGeo.cpp - DataTypeCustomIPv4AndIPv6.cpp - DataTypeCustomSimpleAggregateFunction.cpp - DataTypeDate.cpp - DataTypeDate32.cpp - DataTypeDateTime.cpp - DataTypeDateTime64.cpp - DataTypeDecimalBase.cpp - DataTypeEnum.cpp - DataTypeFactory.cpp - DataTypeFixedString.cpp - DataTypeFunction.cpp - DataTypeInterval.cpp - DataTypeLowCardinality.cpp - DataTypeLowCardinalityHelpers.cpp - DataTypeMap.cpp - DataTypeNested.cpp - DataTypeNothing.cpp - DataTypeNullable.cpp - DataTypeNumberBase.cpp - DataTypeString.cpp - DataTypeTuple.cpp - DataTypeUUID.cpp - DataTypesDecimal.cpp - DataTypesNumber.cpp - EnumValues.cpp - FieldToDataType.cpp - IDataType.cpp - NestedUtils.cpp - Serializations/ISerialization.cpp - Serializations/SerializationAggregateFunction.cpp - Serializations/SerializationArray.cpp - Serializations/SerializationCustomSimpleText.cpp - Serializations/SerializationDate.cpp - Serializations/SerializationDate32.cpp - Serializations/SerializationDateTime.cpp - Serializations/SerializationDateTime64.cpp - Serializations/SerializationDecimal.cpp - Serializations/SerializationDecimalBase.cpp - Serializations/SerializationEnum.cpp - Serializations/SerializationFixedString.cpp - Serializations/SerializationIP.cpp - Serializations/SerializationLowCardinality.cpp - Serializations/SerializationMap.cpp - Serializations/SerializationNothing.cpp - Serializations/SerializationNullable.cpp - Serializations/SerializationNumber.cpp - Serializations/SerializationString.cpp - Serializations/SerializationTuple.cpp - Serializations/SerializationTupleElement.cpp - Serializations/SerializationUUID.cpp - Serializations/SerializationWrapper.cpp - convertMySQLDataType.cpp - getLeastSupertype.cpp - getMostSubtype.cpp - registerDataTypeDateTime.cpp - -) - -END() diff --git a/src/DataTypes/ya.make.in b/src/DataTypes/ya.make.in deleted file mode 100644 index 91a24812170..00000000000 --- a/src/DataTypes/ya.make.in +++ /dev/null @@ -1,15 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - clickhouse/src/Formats -) - - -SRCS( - -) - -END() diff --git a/src/Databases/ya.make b/src/Databases/ya.make deleted file mode 100644 index d088ba16fe2..00000000000 --- a/src/Databases/ya.make +++ /dev/null @@ -1,38 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - DDLDependencyVisitor.cpp - DatabaseAtomic.cpp - DatabaseDictionary.cpp - DatabaseFactory.cpp - DatabaseLazy.cpp - DatabaseMemory.cpp - DatabaseOnDisk.cpp - DatabaseOrdinary.cpp - DatabaseReplicated.cpp - DatabaseReplicatedSettings.cpp - DatabaseReplicatedWorker.cpp - DatabasesCommon.cpp - MySQL/ConnectionMySQLSettings.cpp - MySQL/DatabaseMaterializedMySQL.cpp - MySQL/DatabaseMySQL.cpp - MySQL/FetchTablesColumnsList.cpp - MySQL/MaterializeMetadata.cpp - MySQL/MaterializedMySQLSettings.cpp - MySQL/MaterializedMySQLSyncThread.cpp - SQLite/DatabaseSQLite.cpp - SQLite/SQLiteUtils.cpp - SQLite/fetchSQLiteTableStructure.cpp - TablesLoader.cpp - -) - -END() diff --git a/src/Databases/ya.make.in b/src/Databases/ya.make.in deleted file mode 100644 index cb8b23b43af..00000000000 --- a/src/Databases/ya.make.in +++ /dev/null @@ -1,14 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - -) - -END() diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make deleted file mode 100644 index 6498367bb6a..00000000000 --- a/src/Dictionaries/ya.make +++ /dev/null @@ -1,72 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/poco/Data - contrib/libs/poco/MongoDB - contrib/libs/poco/Redis - contrib/libs/sparsehash - contrib/restricted/abseil-cpp -) - -IF (USE_ODBC) - PEERDIR(contrib/libs/poco/Data/ODBC) -ENDIF () - -NO_COMPILER_WARNINGS() - - -SRCS( - CacheDictionary.cpp - CacheDictionaryUpdateQueue.cpp - CassandraDictionarySource.cpp - CassandraHelpers.cpp - CassandraSource.cpp - ClickHouseDictionarySource.cpp - DictionaryFactory.cpp - DictionaryHelpers.cpp - DictionarySource.cpp - DictionarySourceBase.cpp - DictionarySourceFactory.cpp - DictionarySourceHelpers.cpp - DictionaryStructure.cpp - DirectDictionary.cpp - Embedded/GeoDictionariesLoader.cpp - Embedded/GeodataProviders/HierarchiesProvider.cpp - Embedded/GeodataProviders/HierarchyFormatReader.cpp - Embedded/GeodataProviders/NamesFormatReader.cpp - Embedded/GeodataProviders/NamesProvider.cpp - Embedded/RegionsHierarchies.cpp - Embedded/RegionsHierarchy.cpp - Embedded/RegionsNames.cpp - ExecutableDictionarySource.cpp - ExecutablePoolDictionarySource.cpp - ExternalQueryBuilder.cpp - FileDictionarySource.cpp - FlatDictionary.cpp - HTTPDictionarySource.cpp - HashedDictionary.cpp - HierarchyDictionariesUtils.cpp - IPAddressDictionary.cpp - LibraryDictionarySource.cpp - MongoDBDictionarySource.cpp - MySQLDictionarySource.cpp - PolygonDictionary.cpp - PolygonDictionaryImplementations.cpp - PolygonDictionaryUtils.cpp - RangeHashedDictionary.cpp - RedisDictionarySource.cpp - RedisSource.cpp - XDBCDictionarySource.cpp - getDictionaryConfigurationFromAST.cpp - readInvalidateQuery.cpp - registerCacheDictionaries.cpp - registerDictionaries.cpp - writeParenthesisedString.cpp - -) - -END() diff --git a/src/Dictionaries/ya.make.in b/src/Dictionaries/ya.make.in deleted file mode 100644 index 051696a18ed..00000000000 --- a/src/Dictionaries/ya.make.in +++ /dev/null @@ -1,25 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/poco/Data - contrib/libs/poco/MongoDB - contrib/libs/poco/Redis - contrib/libs/sparsehash - contrib/restricted/abseil-cpp -) - -IF (USE_ODBC) - PEERDIR(contrib/libs/poco/Data/ODBC) -ENDIF () - -NO_COMPILER_WARNINGS() - - -SRCS( - -) - -END() diff --git a/src/Disks/S3/ya.make b/src/Disks/S3/ya.make deleted file mode 100644 index f4df540168d..00000000000 --- a/src/Disks/S3/ya.make +++ /dev/null @@ -1,17 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - DiskS3.cpp - registerDiskS3.cpp - ProxyListConfiguration.cpp - ProxyResolverConfiguration.cpp -) - -END() diff --git a/src/Disks/ya.make b/src/Disks/ya.make deleted file mode 100644 index 3f57930f717..00000000000 --- a/src/Disks/ya.make +++ /dev/null @@ -1,37 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - -SRCS( - DiskCacheWrapper.cpp - DiskDecorator.cpp - DiskEncrypted.cpp - DiskFactory.cpp - DiskLocal.cpp - DiskMemory.cpp - DiskRestartProxy.cpp - DiskSelector.cpp - DiskWebServer.cpp - IDisk.cpp - IDiskRemote.cpp - IVolume.cpp - LocalDirectorySyncGuard.cpp - ReadIndirectBufferFromRemoteFS.cpp - ReadIndirectBufferFromWebServer.cpp - SingleDiskVolume.cpp - StoragePolicy.cpp - TemporaryFileOnDisk.cpp - VolumeJBOD.cpp - VolumeRAID1.cpp - WriteIndirectBufferFromRemoteFS.cpp - createVolume.cpp - registerDisks.cpp - -) - -END() diff --git a/src/Disks/ya.make.in b/src/Disks/ya.make.in deleted file mode 100644 index 303a2118784..00000000000 --- a/src/Disks/ya.make.in +++ /dev/null @@ -1,13 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - -SRCS( - -) - -END() diff --git a/src/Formats/ya.make b/src/Formats/ya.make deleted file mode 100644 index 90184350359..00000000000 --- a/src/Formats/ya.make +++ /dev/null @@ -1,30 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/protobuf - contrib/libs/protoc -) - - -SRCS( - FormatFactory.cpp - FormatSchemaInfo.cpp - JSONEachRowUtils.cpp - MySQLSource.cpp - NativeFormat.cpp - NullFormat.cpp - ParsedTemplateFormatString.cpp - ProtobufReader.cpp - ProtobufSchemas.cpp - ProtobufSerializer.cpp - ProtobufWriter.cpp - registerFormats.cpp - verbosePrintString.cpp - -) - -END() diff --git a/src/Formats/ya.make.in b/src/Formats/ya.make.in deleted file mode 100644 index 9a257d7846e..00000000000 --- a/src/Formats/ya.make.in +++ /dev/null @@ -1,16 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/protobuf - contrib/libs/protoc -) - - -SRCS( - -) - -END() diff --git a/src/Functions/ya.make b/src/Functions/ya.make deleted file mode 100644 index 33f2d3f7767..00000000000 --- a/src/Functions/ya.make +++ /dev/null @@ -1,591 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -CFLAGS( - -DUSE_SSL -DUSE_XXHASH -) - -ADDINCL( - library/cpp/consistent_hashing - contrib/libs/farmhash - contrib/libs/h3/h3lib/include - contrib/libs/hyperscan/src - contrib/libs/libdivide - contrib/libs/rapidjson/include - contrib/libs/xxhash - GLOBAL contrib/restricted/murmurhash -) - -PEERDIR( - clickhouse/src/Common - clickhouse/src/Parsers - clickhouse/src/Dictionaries - contrib/libs/farmhash - contrib/libs/fastops/fastops - contrib/libs/h3 - contrib/libs/hyperscan - contrib/libs/icu - contrib/libs/libdivide - contrib/libs/metrohash - contrib/libs/rapidjson - contrib/libs/xxhash - contrib/restricted/murmurhash - library/cpp/consistent_hashing -) - -# "Arcadia" build is slightly deficient. It lacks many libraries that we need. - -SRCS( - CRC.cpp - CastOverloadResolver.cpp - DateOrDateTimeFunctionsConvertion.cpp - FunctionChar.cpp - FunctionFQDN.cpp - FunctionFactory.cpp - FunctionFile.cpp - FunctionHelpers.cpp - FunctionJoinGet.cpp - FunctionSQLJSON.cpp - FunctionsAES.cpp - FunctionsBinaryRepr.cpp - FunctionsBitToArray.cpp - FunctionsCodingIP.cpp - FunctionsCodingUUID.cpp - FunctionsConversion.cpp - FunctionsEmbeddedDictionaries.cpp - FunctionsExternalDictionaries.cpp - FunctionsExternalModels.cpp - FunctionsHashing.cpp - FunctionsJSON.cpp - FunctionsLogical.cpp - FunctionsRandom.cpp - FunctionsRound.cpp - FunctionsStringArray.cpp - FunctionsStringHash.cpp - FunctionsStringSimilarity.cpp - GatherUtils/concat.cpp - GatherUtils/createArraySink.cpp - GatherUtils/createArraySource.cpp - GatherUtils/createValueSource.cpp - GatherUtils/has_all.cpp - GatherUtils/has_any.cpp - GatherUtils/has_substr.cpp - GatherUtils/push.cpp - GatherUtils/resizeConstantSize.cpp - GatherUtils/resizeDynamicSize.cpp - GatherUtils/sliceDynamicOffsetBounded.cpp - GatherUtils/sliceDynamicOffsetUnbounded.cpp - GatherUtils/sliceFromLeftConstantOffsetBounded.cpp - GatherUtils/sliceFromLeftConstantOffsetUnbounded.cpp - GatherUtils/sliceFromRightConstantOffsetBounded.cpp - GatherUtils/sliceFromRightConstantOffsetUnbounded.cpp - GeoHash.cpp - IFunction.cpp - JSONPath/Parsers/ParserJSONPath.cpp - JSONPath/Parsers/ParserJSONPathMemberAccess.cpp - JSONPath/Parsers/ParserJSONPathQuery.cpp - JSONPath/Parsers/ParserJSONPathRange.cpp - JSONPath/Parsers/ParserJSONPathRoot.cpp - JSONPath/Parsers/ParserJSONPathStar.cpp - TargetSpecific.cpp - URL/URLHierarchy.cpp - URL/URLPathHierarchy.cpp - URL/basename.cpp - URL/cutFragment.cpp - URL/cutQueryString.cpp - URL/cutQueryStringAndFragment.cpp - URL/cutToFirstSignificantSubdomain.cpp - URL/cutToFirstSignificantSubdomainCustom.cpp - URL/cutURLParameter.cpp - URL/cutWWW.cpp - URL/decodeURLComponent.cpp - URL/domain.cpp - URL/domainWithoutWWW.cpp - URL/extractURLParameter.cpp - URL/extractURLParameterNames.cpp - URL/extractURLParameters.cpp - URL/firstSignificantSubdomain.cpp - URL/firstSignificantSubdomainCustom.cpp - URL/fragment.cpp - URL/netloc.cpp - URL/path.cpp - URL/pathFull.cpp - URL/port.cpp - URL/protocol.cpp - URL/queryString.cpp - URL/queryStringAndFragment.cpp - URL/registerFunctionsURL.cpp - URL/tldLookup.generated.cpp - URL/topLevelDomain.cpp - abs.cpp - acos.cpp - acosh.cpp - addDays.cpp - addHours.cpp - addMinutes.cpp - addMonths.cpp - addQuarters.cpp - addSeconds.cpp - addWeeks.cpp - addYears.cpp - addressToLine.cpp - addressToSymbol.cpp - aes_decrypt_mysql.cpp - aes_encrypt_mysql.cpp - appendTrailingCharIfAbsent.cpp - array/array.cpp - array/arrayAUC.cpp - array/arrayAggregation.cpp - array/arrayAll.cpp - array/arrayCompact.cpp - array/arrayConcat.cpp - array/arrayCount.cpp - array/arrayCumSum.cpp - array/arrayCumSumNonNegative.cpp - array/arrayDifference.cpp - array/arrayDistinct.cpp - array/arrayElement.cpp - array/arrayEnumerate.cpp - array/arrayEnumerateDense.cpp - array/arrayEnumerateDenseRanked.cpp - array/arrayEnumerateRanked.cpp - array/arrayEnumerateUniq.cpp - array/arrayEnumerateUniqRanked.cpp - array/arrayExists.cpp - array/arrayFill.cpp - array/arrayFilter.cpp - array/arrayFirst.cpp - array/arrayFirstIndex.cpp - array/arrayFlatten.cpp - array/arrayIntersect.cpp - array/arrayJoin.cpp - array/arrayMap.cpp - array/arrayPopBack.cpp - array/arrayPopFront.cpp - array/arrayPushBack.cpp - array/arrayPushFront.cpp - array/arrayReduce.cpp - array/arrayReduceInRanges.cpp - array/arrayResize.cpp - array/arrayReverse.cpp - array/arraySlice.cpp - array/arraySort.cpp - array/arraySplit.cpp - array/arrayUniq.cpp - array/arrayWithConstant.cpp - array/arrayZip.cpp - array/countEqual.cpp - array/emptyArray.cpp - array/emptyArrayToSingle.cpp - array/has.cpp - array/hasAll.cpp - array/hasAny.cpp - array/hasSubstr.cpp - array/indexOf.cpp - array/length.cpp - array/mapOp.cpp - array/mapPopulateSeries.cpp - array/range.cpp - array/registerFunctionsArray.cpp - asin.cpp - asinh.cpp - assumeNotNull.cpp - atan.cpp - atan2.cpp - atanh.cpp - bar.cpp - base64Decode.cpp - base64Encode.cpp - bitAnd.cpp - bitBoolMaskAnd.cpp - bitBoolMaskOr.cpp - bitCount.cpp - bitHammingDistance.cpp - bitNot.cpp - bitOr.cpp - bitRotateLeft.cpp - bitRotateRight.cpp - bitShiftLeft.cpp - bitShiftRight.cpp - bitSwapLastTwo.cpp - bitTest.cpp - bitTestAll.cpp - bitTestAny.cpp - bitWrapperFunc.cpp - bitXor.cpp - blockNumber.cpp - blockSerializedSize.cpp - blockSize.cpp - byteSize.cpp - caseWithExpression.cpp - cbrt.cpp - coalesce.cpp - concat.cpp - connectionId.cpp - convertCharset.cpp - cos.cpp - cosh.cpp - countDigits.cpp - countMatches.cpp - countSubstrings.cpp - countSubstringsCaseInsensitive.cpp - countSubstringsCaseInsensitiveUTF8.cpp - currentDatabase.cpp - currentProfiles.cpp - currentRoles.cpp - currentUser.cpp - dateDiff.cpp - dateName.cpp - date_trunc.cpp - decodeXMLComponent.cpp - decrypt.cpp - defaultValueOfArgumentType.cpp - defaultValueOfTypeName.cpp - demange.cpp - divide.cpp - divide/divide.cpp - divide/divideImpl.cpp - dumpColumnStructure.cpp - empty.cpp - encodeXMLComponent.cpp - encrypt.cpp - endsWith.cpp - equals.cpp - erf.cpp - erfc.cpp - errorCodeToName.cpp - evalMLMethod.cpp - exp.cpp - exp10.cpp - exp2.cpp - extract.cpp - extractAllGroupsHorizontal.cpp - extractAllGroupsVertical.cpp - extractGroups.cpp - extractTextFromHTML.cpp - extractTimeZoneFromFunctionArguments.cpp - filesystem.cpp - finalizeAggregation.cpp - formatDateTime.cpp - formatReadableQuantity.cpp - formatReadableSize.cpp - formatReadableTimeDelta.cpp - formatRow.cpp - formatString.cpp - fromModifiedJulianDay.cpp - fromUnixTimestamp64Micro.cpp - fromUnixTimestamp64Milli.cpp - fromUnixTimestamp64Nano.cpp - fuzzBits.cpp - gcd.cpp - generateUUIDv4.cpp - geoToH3.cpp - geoToS2.cpp - geohashDecode.cpp - geohashEncode.cpp - geohashesInBox.cpp - getMacro.cpp - getScalar.cpp - getServerPort.cpp - getSetting.cpp - getSizeOfEnumType.cpp - globalVariable.cpp - greatCircleDistance.cpp - greater.cpp - greaterOrEquals.cpp - greatest.cpp - h3EdgeAngle.cpp - h3EdgeLengthM.cpp - h3GetBaseCell.cpp - h3GetFaces.cpp - h3GetResolution.cpp - h3HexAreaM2.cpp - h3IndexesAreNeighbors.cpp - h3IsPentagon.cpp - h3IsValid.cpp - h3ResIsClassIII.cpp - h3ToChildren.cpp - h3ToGeoBoundary.cpp - h3ToParent.cpp - h3ToString.cpp - h3kRing.cpp - h3toGeo.cpp - hasColumnInTable.cpp - hasThreadFuzzer.cpp - hasToken.cpp - hasTokenCaseInsensitive.cpp - hyperscanRegexpChecker.cpp - hypot.cpp - identity.cpp - if.cpp - ifNotFinite.cpp - ifNull.cpp - ignore.cpp - ilike.cpp - in.cpp - indexHint.cpp - initialQueryID.cpp - initializeAggregation.cpp - intDiv.cpp - intDivOrZero.cpp - intExp10.cpp - intExp2.cpp - isConstant.cpp - isDecimalOverflow.cpp - isFinite.cpp - isIPAddressContainedIn.cpp - isInfinite.cpp - isNaN.cpp - isNotNull.cpp - isNull.cpp - isValidUTF8.cpp - isZeroOrNull.cpp - jumpConsistentHash.cpp - lcm.cpp - least.cpp - lemmatize.cpp - lengthUTF8.cpp - less.cpp - lessOrEquals.cpp - lgamma.cpp - like.cpp - log.cpp - log10.cpp - log1p.cpp - log2.cpp - logTrace.cpp - lowCardinalityIndices.cpp - lowCardinalityKeys.cpp - lower.cpp - lowerUTF8.cpp - map.cpp - match.cpp - materialize.cpp - mathConstants.cpp - minus.cpp - modulo.cpp - moduloOrZero.cpp - multiFuzzyMatchAllIndices.cpp - multiFuzzyMatchAny.cpp - multiFuzzyMatchAnyIndex.cpp - multiIf.cpp - multiMatchAllIndices.cpp - multiMatchAny.cpp - multiMatchAnyIndex.cpp - multiSearchAllPositions.cpp - multiSearchAllPositionsCaseInsensitive.cpp - multiSearchAllPositionsCaseInsensitiveUTF8.cpp - multiSearchAllPositionsUTF8.cpp - multiSearchAny.cpp - multiSearchAnyCaseInsensitive.cpp - multiSearchAnyCaseInsensitiveUTF8.cpp - multiSearchAnyUTF8.cpp - multiSearchFirstIndex.cpp - multiSearchFirstIndexCaseInsensitive.cpp - multiSearchFirstIndexCaseInsensitiveUTF8.cpp - multiSearchFirstIndexUTF8.cpp - multiSearchFirstPosition.cpp - multiSearchFirstPositionCaseInsensitive.cpp - multiSearchFirstPositionCaseInsensitiveUTF8.cpp - multiSearchFirstPositionUTF8.cpp - multiply.cpp - negate.cpp - neighbor.cpp - normalizeQuery.cpp - normalizedQueryHash.cpp - notEmpty.cpp - notEquals.cpp - notILike.cpp - notLike.cpp - now.cpp - now64.cpp - nullIf.cpp - padString.cpp - partitionId.cpp - plus.cpp - pointInEllipses.cpp - pointInPolygon.cpp - polygonArea.cpp - polygonConvexHull.cpp - polygonPerimeter.cpp - polygonsDistance.cpp - polygonsEquals.cpp - polygonsIntersection.cpp - polygonsSymDifference.cpp - polygonsUnion.cpp - polygonsWithin.cpp - position.cpp - positionCaseInsensitive.cpp - positionCaseInsensitiveUTF8.cpp - positionUTF8.cpp - pow.cpp - queryID.cpp - rand.cpp - rand64.cpp - randConstant.cpp - randomFixedString.cpp - randomPrintableASCII.cpp - randomString.cpp - randomStringUTF8.cpp - readWkt.cpp - regexpQuoteMeta.cpp - registerFunctions.cpp - registerFunctionsArithmetic.cpp - registerFunctionsComparison.cpp - registerFunctionsConditional.cpp - registerFunctionsConsistentHashing.cpp - registerFunctionsDateTime.cpp - registerFunctionsFormatting.cpp - registerFunctionsGeo.cpp - registerFunctionsHigherOrder.cpp - registerFunctionsIntrospection.cpp - registerFunctionsMath.cpp - registerFunctionsMiscellaneous.cpp - registerFunctionsNull.cpp - registerFunctionsRandom.cpp - registerFunctionsReinterpret.cpp - registerFunctionsSnowflake.cpp - registerFunctionsString.cpp - registerFunctionsStringRegexp.cpp - registerFunctionsStringSearch.cpp - registerFunctionsTuple.cpp - registerFunctionsUnixTimestamp64.cpp - registerFunctionsVisitParam.cpp - reinterpretAs.cpp - repeat.cpp - replaceAll.cpp - replaceOne.cpp - replaceRegexpAll.cpp - replaceRegexpOne.cpp - replicate.cpp - reverse.cpp - reverseUTF8.cpp - roundAge.cpp - roundDuration.cpp - roundToExp2.cpp - rowNumberInAllBlocks.cpp - rowNumberInBlock.cpp - runningAccumulate.cpp - runningConcurrency.cpp - runningDifference.cpp - runningDifferenceStartingWithFirstValue.cpp - s2CapContains.cpp - s2CapUnion.cpp - s2CellsIntersect.cpp - s2GetNeighbors.cpp - s2RectAdd.cpp - s2RectContains.cpp - s2RectIntersection.cpp - s2RectUnion.cpp - s2ToGeo.cpp - serverConstants.cpp - sigmoid.cpp - sign.cpp - sin.cpp - sinh.cpp - sleep.cpp - sleepEachRow.cpp - snowflake.cpp - sqrt.cpp - startsWith.cpp - stem.cpp - stringCutToZero.cpp - stringToH3.cpp - substring.cpp - subtractDays.cpp - subtractHours.cpp - subtractMinutes.cpp - subtractMonths.cpp - subtractQuarters.cpp - subtractSeconds.cpp - subtractWeeks.cpp - subtractYears.cpp - svg.cpp - synonyms.cpp - tan.cpp - tanh.cpp - tgamma.cpp - throwIf.cpp - tid.cpp - timeSlot.cpp - timeSlots.cpp - timezoneOf.cpp - timezoneOffset.cpp - toColumnTypeName.cpp - toCustomWeek.cpp - toDayOfMonth.cpp - toDayOfWeek.cpp - toDayOfYear.cpp - toFixedString.cpp - toHour.cpp - toISOWeek.cpp - toISOYear.cpp - toJSONString.cpp - toLowCardinality.cpp - toMinute.cpp - toModifiedJulianDay.cpp - toMonday.cpp - toMonth.cpp - toNullable.cpp - toQuarter.cpp - toRelativeDayNum.cpp - toRelativeHourNum.cpp - toRelativeMinuteNum.cpp - toRelativeMonthNum.cpp - toRelativeQuarterNum.cpp - toRelativeSecondNum.cpp - toRelativeWeekNum.cpp - toRelativeYearNum.cpp - toSecond.cpp - toStartOfDay.cpp - toStartOfFifteenMinutes.cpp - toStartOfFiveMinute.cpp - toStartOfHour.cpp - toStartOfISOYear.cpp - toStartOfInterval.cpp - toStartOfMinute.cpp - toStartOfMonth.cpp - toStartOfQuarter.cpp - toStartOfSecond.cpp - toStartOfTenMinutes.cpp - toStartOfYear.cpp - toTime.cpp - toTimezone.cpp - toTypeName.cpp - toUnixTimestamp64Micro.cpp - toUnixTimestamp64Milli.cpp - toUnixTimestamp64Nano.cpp - toValidUTF8.cpp - toYYYYMM.cpp - toYYYYMMDD.cpp - toYYYYMMDDhhmmss.cpp - toYear.cpp - today.cpp - transform.cpp - trap.cpp - trim.cpp - tryBase64Decode.cpp - tuple.cpp - tupleElement.cpp - tupleHammingDistance.cpp - tupleToNameValuePairs.cpp - upper.cpp - upperUTF8.cpp - validateNestedArraySizes.cpp - visibleWidth.cpp - visitParamExtractBool.cpp - visitParamExtractFloat.cpp - visitParamExtractInt.cpp - visitParamExtractRaw.cpp - visitParamExtractString.cpp - visitParamExtractUInt.cpp - visitParamHas.cpp - wkt.cpp - yandexConsistentHash.cpp - yesterday.cpp - -) - -END() diff --git a/src/Functions/ya.make.in b/src/Functions/ya.make.in deleted file mode 100644 index b21bf64304a..00000000000 --- a/src/Functions/ya.make.in +++ /dev/null @@ -1,43 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -CFLAGS( - -DUSE_SSL -DUSE_XXHASH -) - -ADDINCL( - library/cpp/consistent_hashing - contrib/libs/farmhash - contrib/libs/h3/h3lib/include - contrib/libs/hyperscan/src - contrib/libs/libdivide - contrib/libs/rapidjson/include - contrib/libs/xxhash - GLOBAL contrib/restricted/murmurhash -) - -PEERDIR( - clickhouse/src/Common - clickhouse/src/Parsers - clickhouse/src/Dictionaries - contrib/libs/farmhash - contrib/libs/fastops/fastops - contrib/libs/h3 - contrib/libs/hyperscan - contrib/libs/icu - contrib/libs/libdivide - contrib/libs/metrohash - contrib/libs/rapidjson - contrib/libs/xxhash - contrib/restricted/murmurhash - library/cpp/consistent_hashing -) - -# "Arcadia" build is slightly deficient. It lacks many libraries that we need. - -SRCS( - -) - -END() diff --git a/src/IO/ya.make b/src/IO/ya.make deleted file mode 100644 index 6857a0ccdf4..00000000000 --- a/src/IO/ya.make +++ /dev/null @@ -1,92 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL( - contrib/libs/zstd/include - contrib/libs/lz4 - contrib/restricted/fast_float/include -) - -PEERDIR( - clickhouse/src/Common - contrib/libs/brotli/dec - contrib/libs/brotli/enc - contrib/libs/poco/NetSSL_OpenSSL - contrib/libs/zstd - contrib/libs/lz4 - contrib/restricted/fast_float -) - -SRCS( - AIO.cpp - AsynchronousReadBufferFromFile.cpp - AsynchronousReadBufferFromFileDescriptor.cpp - BrotliReadBuffer.cpp - BrotliWriteBuffer.cpp - Bzip2ReadBuffer.cpp - Bzip2WriteBuffer.cpp - CascadeWriteBuffer.cpp - CompressionMethod.cpp - DoubleConverter.cpp - FileEncryptionCommon.cpp - HTTPChunkedReadBuffer.cpp - HTTPCommon.cpp - HashingWriteBuffer.cpp - LZMADeflatingWriteBuffer.cpp - LZMAInflatingReadBuffer.cpp - LimitReadBuffer.cpp - Lz4DeflatingWriteBuffer.cpp - Lz4InflatingReadBuffer.cpp - MMapReadBufferFromFile.cpp - MMapReadBufferFromFileDescriptor.cpp - MMapReadBufferFromFileWithCache.cpp - MMappedFile.cpp - MMappedFileDescriptor.cpp - MemoryReadWriteBuffer.cpp - MySQLBinlogEventReadBuffer.cpp - MySQLPacketPayloadReadBuffer.cpp - MySQLPacketPayloadWriteBuffer.cpp - NullWriteBuffer.cpp - OpenedFile.cpp - PeekableReadBuffer.cpp - Progress.cpp - ReadBufferFromEncryptedFile.cpp - ReadBufferFromFile.cpp - ReadBufferFromFileBase.cpp - ReadBufferFromFileDecorator.cpp - ReadBufferFromFileDescriptor.cpp - ReadBufferFromIStream.cpp - ReadBufferFromMemory.cpp - ReadBufferFromPocoSocket.cpp - ReadHelpers.cpp - SeekAvoidingReadBuffer.cpp - SynchronousReader.cpp - ThreadPoolReader.cpp - TimeoutSetter.cpp - UseSSL.cpp - WriteBufferFromEncryptedFile.cpp - WriteBufferFromFile.cpp - WriteBufferFromFileBase.cpp - WriteBufferFromFileDecorator.cpp - WriteBufferFromFileDescriptor.cpp - WriteBufferFromFileDescriptorDiscardOnFailure.cpp - WriteBufferFromHTTP.cpp - WriteBufferFromOStream.cpp - WriteBufferFromPocoSocket.cpp - WriteBufferFromTemporaryFile.cpp - WriteBufferValidUTF8.cpp - WriteHelpers.cpp - ZlibDeflatingWriteBuffer.cpp - ZlibInflatingReadBuffer.cpp - ZstdDeflatingWriteBuffer.cpp - ZstdInflatingReadBuffer.cpp - copyData.cpp - createReadBufferFromFileBase.cpp - parseDateTimeBestEffort.cpp - readFloatText.cpp - -) - -END() diff --git a/src/IO/ya.make.in b/src/IO/ya.make.in deleted file mode 100644 index 21e64baa10a..00000000000 --- a/src/IO/ya.make.in +++ /dev/null @@ -1,25 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL( - contrib/libs/zstd/include - contrib/libs/lz4 - contrib/restricted/fast_float/include -) - -PEERDIR( - clickhouse/src/Common - contrib/libs/brotli/dec - contrib/libs/brotli/enc - contrib/libs/poco/NetSSL_OpenSSL - contrib/libs/zstd - contrib/libs/lz4 - contrib/restricted/fast_float -) - -SRCS( - -) - -END() diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make deleted file mode 100644 index f24a10e55df..00000000000 --- a/src/Interpreters/ya.make +++ /dev/null @@ -1,199 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL( - contrib/libs/libdivide - contrib/libs/pdqsort -) - -PEERDIR( - clickhouse/src/Core - contrib/libs/libdivide - contrib/libs/pdqsort -) - -NO_COMPILER_WARNINGS() - - -SRCS( - ActionLocksManager.cpp - ActionsDAG.cpp - ActionsVisitor.cpp - AggregateDescription.cpp - Aggregator.cpp - ApplyWithAliasVisitor.cpp - ApplyWithGlobalVisitor.cpp - ApplyWithSubqueryVisitor.cpp - ArithmeticOperationsInAgrFuncOptimize.cpp - ArrayJoinAction.cpp - AsynchronousInsertQueue.cpp - AsynchronousMetricLog.cpp - AsynchronousMetrics.cpp - BloomFilter.cpp - CatBoostModel.cpp - ClientInfo.cpp - Cluster.cpp - ClusterProxy/SelectStreamFactory.cpp - ClusterProxy/executeQuery.cpp - CollectJoinOnKeysVisitor.cpp - ColumnAliasesVisitor.cpp - Context.cpp - ConvertStringsToEnumVisitor.cpp - CrashLog.cpp - CrossToInnerJoinVisitor.cpp - DDLTask.cpp - DDLWorker.cpp - DNSCacheUpdater.cpp - DatabaseAndTableWithAlias.cpp - DatabaseCatalog.cpp - DictionaryReader.cpp - DuplicateOrderByVisitor.cpp - EmbeddedDictionaries.cpp - ExecuteScalarSubqueriesVisitor.cpp - ExpressionActions.cpp - ExpressionActionsSettings.cpp - ExpressionAnalyzer.cpp - ExternalDictionariesLoader.cpp - ExternalLoader.cpp - ExternalLoaderDictionaryStorageConfigRepository.cpp - ExternalLoaderTempConfigRepository.cpp - ExternalLoaderXMLConfigRepository.cpp - ExternalModelsLoader.cpp - ExternalUserDefinedExecutableFunctionsLoader.cpp - ExtractExpressionInfoVisitor.cpp - FillingRow.cpp - FunctionNameNormalizer.cpp - GatherFunctionQuantileVisitor.cpp - HashJoin.cpp - IExternalLoadable.cpp - IInterpreter.cpp - IInterpreterUnionOrSelectQuery.cpp - IdentifierSemantic.cpp - InJoinSubqueriesPreprocessor.cpp - InternalTextLogsQueue.cpp - InterpreterAlterQuery.cpp - InterpreterBackupQuery.cpp - InterpreterCheckQuery.cpp - InterpreterCreateFunctionQuery.cpp - InterpreterCreateQuery.cpp - InterpreterCreateQuotaQuery.cpp - InterpreterCreateRoleQuery.cpp - InterpreterCreateRowPolicyQuery.cpp - InterpreterCreateSettingsProfileQuery.cpp - InterpreterCreateUserQuery.cpp - InterpreterDescribeQuery.cpp - InterpreterDropAccessEntityQuery.cpp - InterpreterDropFunctionQuery.cpp - InterpreterDropQuery.cpp - InterpreterExistsQuery.cpp - InterpreterExplainQuery.cpp - InterpreterExternalDDLQuery.cpp - InterpreterFactory.cpp - InterpreterGrantQuery.cpp - InterpreterInsertQuery.cpp - InterpreterKillQueryQuery.cpp - InterpreterOptimizeQuery.cpp - InterpreterRenameQuery.cpp - InterpreterSelectIntersectExceptQuery.cpp - InterpreterSelectQuery.cpp - InterpreterSelectWithUnionQuery.cpp - InterpreterSetQuery.cpp - InterpreterSetRoleQuery.cpp - InterpreterShowAccessEntitiesQuery.cpp - InterpreterShowAccessQuery.cpp - InterpreterShowCreateAccessEntityQuery.cpp - InterpreterShowCreateQuery.cpp - InterpreterShowGrantsQuery.cpp - InterpreterShowPrivilegesQuery.cpp - InterpreterShowProcesslistQuery.cpp - InterpreterShowTablesQuery.cpp - InterpreterSystemQuery.cpp - InterpreterUseQuery.cpp - InterpreterWatchQuery.cpp - InterserverCredentials.cpp - JoinSwitcher.cpp - JoinToSubqueryTransformVisitor.cpp - JoinedTables.cpp - Lemmatizers.cpp - LogicalExpressionsOptimizer.cpp - MarkTableIdentifiersVisitor.cpp - MergeJoin.cpp - MetricLog.cpp - MutationsInterpreter.cpp - MySQL/InterpretersMySQLDDLQuery.cpp - NormalizeSelectWithUnionQueryVisitor.cpp - NullableUtils.cpp - OpenTelemetrySpanLog.cpp - OptimizeIfChains.cpp - OptimizeIfWithConstantConditionVisitor.cpp - OptimizeShardingKeyRewriteInVisitor.cpp - PartLog.cpp - PredicateExpressionsOptimizer.cpp - PredicateRewriteVisitor.cpp - ProcessList.cpp - ProfileEventsExt.cpp - QueryAliasesVisitor.cpp - QueryLog.cpp - QueryNormalizer.cpp - QueryParameterVisitor.cpp - QueryThreadLog.cpp - QueryViewsLog.cpp - RemoveInjectiveFunctionsVisitor.cpp - RenameColumnVisitor.cpp - ReplaceQueryParameterVisitor.cpp - RequiredSourceColumnsData.cpp - RequiredSourceColumnsVisitor.cpp - RewriteAnyFunctionVisitor.cpp - RewriteCountVariantsVisitor.cpp - RewriteFunctionToSubcolumnVisitor.cpp - RewriteSumIfFunctionVisitor.cpp - RowRefs.cpp - SelectIntersectExceptQueryVisitor.cpp - Session.cpp - SessionLog.cpp - Set.cpp - SetVariants.cpp - SortedBlocksWriter.cpp - StorageID.cpp - SubqueryForSet.cpp - SynonymsExtensions.cpp - SystemLog.cpp - TableJoin.cpp - TablesStatus.cpp - TextLog.cpp - ThreadStatusExt.cpp - TraceLog.cpp - TranslateQualifiedNamesVisitor.cpp - TreeOptimizer.cpp - TreeRewriter.cpp - UserDefinedExecutableFunction.cpp - UserDefinedExecutableFunctionFactory.cpp - UserDefinedSQLFunctionFactory.cpp - UserDefinedSQLFunctionVisitor.cpp - UserDefinedSQLObjectsLoader.cpp - WindowDescription.cpp - ZooKeeperLog.cpp - addMissingDefaults.cpp - addTypeConversionToAST.cpp - castColumn.cpp - convertFieldToType.cpp - createBlockSelector.cpp - evaluateConstantExpression.cpp - executeDDLQueryOnCluster.cpp - executeQuery.cpp - getClusterName.cpp - getHeaderForProcessingStage.cpp - getTableExpressions.cpp - inplaceBlockConversions.cpp - interpretSubquery.cpp - join_common.cpp - loadMetadata.cpp - processColumnTransformers.cpp - replaceAliasColumnsInQuery.cpp - sortBlock.cpp - -) - -END() diff --git a/src/Interpreters/ya.make.in b/src/Interpreters/ya.make.in deleted file mode 100644 index 4481f8c6136..00000000000 --- a/src/Interpreters/ya.make.in +++ /dev/null @@ -1,23 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -ADDINCL( - contrib/libs/libdivide - contrib/libs/pdqsort -) - -PEERDIR( - clickhouse/src/Core - contrib/libs/libdivide - contrib/libs/pdqsort -) - -NO_COMPILER_WARNINGS() - - -SRCS( - -) - -END() diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make deleted file mode 100644 index d17b73104d6..00000000000 --- a/src/Parsers/ya.make +++ /dev/null @@ -1,161 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - ASTAlterQuery.cpp - ASTAsterisk.cpp - ASTBackupQuery.cpp - ASTColumnDeclaration.cpp - ASTColumnsMatcher.cpp - ASTColumnsTransformers.cpp - ASTConstraintDeclaration.cpp - ASTCreateFunctionQuery.cpp - ASTCreateQuery.cpp - ASTCreateQuotaQuery.cpp - ASTCreateRoleQuery.cpp - ASTCreateRowPolicyQuery.cpp - ASTCreateSettingsProfileQuery.cpp - ASTCreateUserQuery.cpp - ASTDatabaseOrNone.cpp - ASTDictionary.cpp - ASTDictionaryAttributeDeclaration.cpp - ASTDropAccessEntityQuery.cpp - ASTDropFunctionQuery.cpp - ASTDropQuery.cpp - ASTExpressionList.cpp - ASTFunction.cpp - ASTFunctionWithKeyValueArguments.cpp - ASTGrantQuery.cpp - ASTIdentifier.cpp - ASTIndexDeclaration.cpp - ASTInsertQuery.cpp - ASTKillQueryQuery.cpp - ASTLiteral.cpp - ASTNameTypePair.cpp - ASTOptimizeQuery.cpp - ASTOrderByElement.cpp - ASTPartition.cpp - ASTProjectionDeclaration.cpp - ASTProjectionSelectQuery.cpp - ASTQualifiedAsterisk.cpp - ASTQueryParameter.cpp - ASTQueryWithOnCluster.cpp - ASTQueryWithOutput.cpp - ASTQueryWithTableAndOutput.cpp - ASTRolesOrUsersSet.cpp - ASTRowPolicyName.cpp - ASTSampleRatio.cpp - ASTSelectIntersectExceptQuery.cpp - ASTSelectQuery.cpp - ASTSelectWithUnionQuery.cpp - ASTSetQuery.cpp - ASTSetRoleQuery.cpp - ASTSettingsProfileElement.cpp - ASTShowAccessEntitiesQuery.cpp - ASTShowCreateAccessEntityQuery.cpp - ASTShowGrantsQuery.cpp - ASTShowTablesQuery.cpp - ASTSubquery.cpp - ASTSystemQuery.cpp - ASTTTLElement.cpp - ASTTablesInSelectQuery.cpp - ASTUserNameWithHost.cpp - ASTWindowDefinition.cpp - ASTWithAlias.cpp - ASTWithElement.cpp - CommonParsers.cpp - ExpressionElementParsers.cpp - ExpressionListParsers.cpp - IAST.cpp - IParserBase.cpp - InsertQuerySettingsPushDownVisitor.cpp - Lexer.cpp - MySQL/ASTAlterCommand.cpp - MySQL/ASTAlterQuery.cpp - MySQL/ASTCreateDefines.cpp - MySQL/ASTCreateQuery.cpp - MySQL/ASTDeclareColumn.cpp - MySQL/ASTDeclareConstraint.cpp - MySQL/ASTDeclareIndex.cpp - MySQL/ASTDeclareOption.cpp - MySQL/ASTDeclarePartition.cpp - MySQL/ASTDeclarePartitionOptions.cpp - MySQL/ASTDeclareReference.cpp - MySQL/ASTDeclareSubPartition.cpp - MySQL/ASTDeclareTableOptions.cpp - ParserAlterQuery.cpp - ParserBackupQuery.cpp - ParserCase.cpp - ParserCheckQuery.cpp - ParserCreateFunctionQuery.cpp - ParserCreateQuery.cpp - ParserCreateQuotaQuery.cpp - ParserCreateRoleQuery.cpp - ParserCreateRowPolicyQuery.cpp - ParserCreateSettingsProfileQuery.cpp - ParserCreateUserQuery.cpp - ParserDataType.cpp - ParserDatabaseOrNone.cpp - ParserDescribeTableQuery.cpp - ParserDictionary.cpp - ParserDictionaryAttributeDeclaration.cpp - ParserDropAccessEntityQuery.cpp - ParserDropFunctionQuery.cpp - ParserDropQuery.cpp - ParserExplainQuery.cpp - ParserExternalDDLQuery.cpp - ParserGrantQuery.cpp - ParserInsertQuery.cpp - ParserKillQueryQuery.cpp - ParserOptimizeQuery.cpp - ParserPartition.cpp - ParserProjectionSelectQuery.cpp - ParserQuery.cpp - ParserQueryWithOutput.cpp - ParserRenameQuery.cpp - ParserRolesOrUsersSet.cpp - ParserRowPolicyName.cpp - ParserSampleRatio.cpp - ParserSelectQuery.cpp - ParserSelectWithUnionQuery.cpp - ParserSetQuery.cpp - ParserSetRoleQuery.cpp - ParserSettingsProfileElement.cpp - ParserShowAccessEntitiesQuery.cpp - ParserShowCreateAccessEntityQuery.cpp - ParserShowGrantsQuery.cpp - ParserShowPrivilegesQuery.cpp - ParserShowTablesQuery.cpp - ParserSystemQuery.cpp - ParserTablePropertiesQuery.cpp - ParserTablesInSelectQuery.cpp - ParserUnionQueryElement.cpp - ParserUseQuery.cpp - ParserUserNameWithHost.cpp - ParserWatchQuery.cpp - ParserWithElement.cpp - QueryWithOutputSettingsPushDownVisitor.cpp - TokenIterator.cpp - formatAST.cpp - formatSettingName.cpp - getInsertQuery.cpp - iostream_debug_helpers.cpp - makeASTForLogicalFunction.cpp - obfuscateQueries.cpp - parseDatabaseAndTableName.cpp - parseIdentifierOrStringLiteral.cpp - parseIntervalKind.cpp - parseQuery.cpp - parseUserName.cpp - queryToString.cpp - -) - -END() diff --git a/src/Parsers/ya.make.in b/src/Parsers/ya.make.in deleted file mode 100644 index 600c2bd9715..00000000000 --- a/src/Parsers/ya.make.in +++ /dev/null @@ -1,14 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - -) - -END() diff --git a/src/Processors/ya.make b/src/Processors/ya.make deleted file mode 100644 index f2063609440..00000000000 --- a/src/Processors/ya.make +++ /dev/null @@ -1,192 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/msgpack - contrib/libs/protobuf - contrib/libs/apache/arrow - contrib/libs/apache/orc -) - -ADDINCL( - contrib/libs/apache/arrow/src - contrib/libs/apache/orc/c++/include -) - -CFLAGS(-DUSE_ARROW=1 -DUSE_PARQUET=1 -DUSE_ORC=1) - -SRCS( - Chunk.cpp - ConcatProcessor.cpp - DelayedPortsProcessor.cpp - Executors/ExecutingGraph.cpp - Executors/PipelineExecutingBlockInputStream.cpp - Executors/PipelineExecutor.cpp - Executors/PollingQueue.cpp - Executors/PullingAsyncPipelineExecutor.cpp - Executors/PullingPipelineExecutor.cpp - Executors/StreamingFormatExecutor.cpp - ForkProcessor.cpp - Formats/IInputFormat.cpp - Formats/IOutputFormat.cpp - Formats/IRowInputFormat.cpp - Formats/IRowOutputFormat.cpp - Formats/Impl/ArrowBlockInputFormat.cpp - Formats/Impl/ArrowBlockOutputFormat.cpp - Formats/Impl/ArrowBufferedStreams.cpp - Formats/Impl/ArrowColumnToCHColumn.cpp - Formats/Impl/BinaryRowInputFormat.cpp - Formats/Impl/BinaryRowOutputFormat.cpp - Formats/Impl/CHColumnToArrowColumn.cpp - Formats/Impl/CSVRowInputFormat.cpp - Formats/Impl/CSVRowOutputFormat.cpp - Formats/Impl/ConstantExpressionTemplate.cpp - Formats/Impl/JSONAsStringRowInputFormat.cpp - Formats/Impl/JSONCompactEachRowRowInputFormat.cpp - Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp - Formats/Impl/JSONCompactRowOutputFormat.cpp - Formats/Impl/JSONEachRowRowInputFormat.cpp - Formats/Impl/JSONEachRowRowOutputFormat.cpp - Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp - Formats/Impl/JSONRowOutputFormat.cpp - Formats/Impl/LineAsStringRowInputFormat.cpp - Formats/Impl/MarkdownRowOutputFormat.cpp - Formats/Impl/MsgPackRowInputFormat.cpp - Formats/Impl/MsgPackRowOutputFormat.cpp - Formats/Impl/MySQLOutputFormat.cpp - Formats/Impl/NullFormat.cpp - Formats/Impl/ODBCDriver2BlockOutputFormat.cpp - Formats/Impl/ParallelFormattingOutputFormat.cpp - Formats/Impl/ParallelParsingInputFormat.cpp - Formats/Impl/PostgreSQLOutputFormat.cpp - Formats/Impl/PrettyBlockOutputFormat.cpp - Formats/Impl/PrettyCompactBlockOutputFormat.cpp - Formats/Impl/PrettySpaceBlockOutputFormat.cpp - Formats/Impl/ProtobufRowInputFormat.cpp - Formats/Impl/ProtobufRowOutputFormat.cpp - Formats/Impl/RawBLOBRowInputFormat.cpp - Formats/Impl/RawBLOBRowOutputFormat.cpp - Formats/Impl/RegexpRowInputFormat.cpp - Formats/Impl/TSKVRowInputFormat.cpp - Formats/Impl/TSKVRowOutputFormat.cpp - Formats/Impl/TabSeparatedRowInputFormat.cpp - Formats/Impl/TabSeparatedRowOutputFormat.cpp - Formats/Impl/TemplateBlockOutputFormat.cpp - Formats/Impl/TemplateRowInputFormat.cpp - Formats/Impl/ValuesBlockInputFormat.cpp - Formats/Impl/ValuesRowOutputFormat.cpp - Formats/Impl/VerticalRowOutputFormat.cpp - Formats/Impl/XMLRowOutputFormat.cpp - Formats/LazyOutputFormat.cpp - Formats/OutputStreamToOutputFormat.cpp - Formats/PullingOutputFormat.cpp - Formats/RowInputFormatWithDiagnosticInfo.cpp - IAccumulatingTransform.cpp - IInflatingTransform.cpp - IProcessor.cpp - ISimpleTransform.cpp - ISink.cpp - ISource.cpp - LimitTransform.cpp - Merges/Algorithms/AggregatingSortedAlgorithm.cpp - Merges/Algorithms/CollapsingSortedAlgorithm.cpp - Merges/Algorithms/FinishAggregatingInOrderAlgorithm.cpp - Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp - Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp - Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp - Merges/Algorithms/MergingSortedAlgorithm.cpp - Merges/Algorithms/ReplacingSortedAlgorithm.cpp - Merges/Algorithms/SummingSortedAlgorithm.cpp - Merges/Algorithms/VersionedCollapsingAlgorithm.cpp - Merges/IMergingTransform.cpp - Merges/MergingSortedTransform.cpp - OffsetTransform.cpp - Pipe.cpp - Port.cpp - QueryPipeline.cpp - QueryPlan/AggregatingStep.cpp - QueryPlan/ArrayJoinStep.cpp - QueryPlan/BuildQueryPipelineSettings.cpp - QueryPlan/CreatingSetsStep.cpp - QueryPlan/CubeStep.cpp - QueryPlan/DistinctStep.cpp - QueryPlan/ExpressionStep.cpp - QueryPlan/ExtremesStep.cpp - QueryPlan/FillingStep.cpp - QueryPlan/FilterStep.cpp - QueryPlan/FinishSortingStep.cpp - QueryPlan/IQueryPlanStep.cpp - QueryPlan/ISourceStep.cpp - QueryPlan/ITransformingStep.cpp - QueryPlan/IntersectOrExceptStep.cpp - QueryPlan/JoinStep.cpp - QueryPlan/LimitByStep.cpp - QueryPlan/LimitStep.cpp - QueryPlan/MergeSortingStep.cpp - QueryPlan/MergingAggregatedStep.cpp - QueryPlan/MergingSortedStep.cpp - QueryPlan/OffsetStep.cpp - QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp - QueryPlan/Optimizations/filterPushDown.cpp - QueryPlan/Optimizations/liftUpArrayJoin.cpp - QueryPlan/Optimizations/limitPushDown.cpp - QueryPlan/Optimizations/mergeExpressions.cpp - QueryPlan/Optimizations/optimizeTree.cpp - QueryPlan/Optimizations/splitFilter.cpp - QueryPlan/PartialSortingStep.cpp - QueryPlan/QueryIdHolder.cpp - QueryPlan/QueryPlan.cpp - QueryPlan/ReadFromMergeTree.cpp - QueryPlan/ReadFromPreparedSource.cpp - QueryPlan/ReadFromRemote.cpp - QueryPlan/ReadNothingStep.cpp - QueryPlan/RollupStep.cpp - QueryPlan/SettingQuotaAndLimitsStep.cpp - QueryPlan/TotalsHavingStep.cpp - QueryPlan/UnionStep.cpp - QueryPlan/WindowStep.cpp - ResizeProcessor.cpp - Sources/DelayedSource.cpp - Sources/RemoteSource.cpp - Sources/SinkToOutputStream.cpp - Sources/SourceFromInputStream.cpp - Sources/SourceWithProgress.cpp - Transforms/AddingDefaultsTransform.cpp - Transforms/AddingSelectorTransform.cpp - Transforms/AggregatingInOrderTransform.cpp - Transforms/AggregatingTransform.cpp - Transforms/ArrayJoinTransform.cpp - Transforms/CheckSortedTransform.cpp - Transforms/CopyTransform.cpp - Transforms/CreatingSetsTransform.cpp - Transforms/CubeTransform.cpp - Transforms/DistinctTransform.cpp - Transforms/ExpressionTransform.cpp - Transforms/ExtremesTransform.cpp - Transforms/FillingTransform.cpp - Transforms/FilterTransform.cpp - Transforms/FinishSortingTransform.cpp - Transforms/IntersectOrExceptTransform.cpp - Transforms/JoiningTransform.cpp - Transforms/LimitByTransform.cpp - Transforms/LimitsCheckingTransform.cpp - Transforms/MaterializingTransform.cpp - Transforms/MergeSortingTransform.cpp - Transforms/MergingAggregatedMemoryEfficientTransform.cpp - Transforms/MergingAggregatedTransform.cpp - Transforms/PartialSortingTransform.cpp - Transforms/ReverseTransform.cpp - Transforms/RollupTransform.cpp - Transforms/SortingTransform.cpp - Transforms/SquashingChunksTransform.cpp - Transforms/TotalsHavingTransform.cpp - Transforms/WindowTransform.cpp - Transforms/getSourceFromASTInsertQuery.cpp - printPipeline.cpp - -) - -END() diff --git a/src/Processors/ya.make.in b/src/Processors/ya.make.in deleted file mode 100644 index 8105d8b1940..00000000000 --- a/src/Processors/ya.make.in +++ /dev/null @@ -1,24 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/msgpack - contrib/libs/protobuf - contrib/libs/apache/arrow - contrib/libs/apache/orc -) - -ADDINCL( - contrib/libs/apache/arrow/src - contrib/libs/apache/orc/c++/include -) - -CFLAGS(-DUSE_ARROW=1 -DUSE_PARQUET=1 -DUSE_ORC=1) - -SRCS( - -) - -END() diff --git a/src/Server/ya.make b/src/Server/ya.make deleted file mode 100644 index 6a6a442fce8..00000000000 --- a/src/Server/ya.make +++ /dev/null @@ -1,41 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/poco/Util -) - - -SRCS( - GRPCServer.cpp - HTTP/HTMLForm.cpp - HTTP/HTTPServer.cpp - HTTP/HTTPServerConnection.cpp - HTTP/HTTPServerConnectionFactory.cpp - HTTP/HTTPServerRequest.cpp - HTTP/HTTPServerResponse.cpp - HTTP/ReadHeaders.cpp - HTTP/WriteBufferFromHTTPServerResponse.cpp - HTTPHandler.cpp - HTTPHandlerFactory.cpp - InterserverIOHTTPHandler.cpp - KeeperTCPHandler.cpp - MySQLHandler.cpp - MySQLHandlerFactory.cpp - NotFoundHandler.cpp - PostgreSQLHandler.cpp - PostgreSQLHandlerFactory.cpp - PrometheusMetricsWriter.cpp - PrometheusRequestHandler.cpp - ProtocolServerAdapter.cpp - ReplicasStatusHandler.cpp - StaticRequestHandler.cpp - TCPHandler.cpp - WebUIRequestHandler.cpp - -) - -END() diff --git a/src/Server/ya.make.in b/src/Server/ya.make.in deleted file mode 100644 index fd1b414edf5..00000000000 --- a/src/Server/ya.make.in +++ /dev/null @@ -1,15 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/poco/Util -) - - -SRCS( - -) - -END() diff --git a/src/Storages/ya.make b/src/Storages/ya.make deleted file mode 100644 index c0da9b29382..00000000000 --- a/src/Storages/ya.make +++ /dev/null @@ -1,241 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/sparsehash - contrib/libs/poco/MongoDB -) - - -SRCS( - AlterCommands.cpp - ColumnDefault.cpp - ColumnsDescription.cpp - ConstraintsDescription.cpp - Distributed/DirectoryMonitor.cpp - Distributed/DistributedSettings.cpp - Distributed/DistributedSink.cpp - ExecutableSettings.cpp - IStorage.cpp - IndicesDescription.cpp - JoinSettings.cpp - KeyDescription.cpp - LiveView/StorageLiveView.cpp - LiveView/TemporaryLiveViewCleaner.cpp - MemorySettings.cpp - MergeTree/ActiveDataPartSet.cpp - MergeTree/AllMergeSelector.cpp - MergeTree/BackgroundJobsAssignee.cpp - MergeTree/BoolMask.cpp - MergeTree/DataPartsExchange.cpp - MergeTree/DropPartsRanges.cpp - MergeTree/EphemeralLockInZooKeeper.cpp - MergeTree/FutureMergedMutatedPart.cpp - MergeTree/IMergeTreeDataPart.cpp - MergeTree/IMergeTreeDataPartWriter.cpp - MergeTree/IMergeTreeReader.cpp - MergeTree/IMergedBlockOutputStream.cpp - MergeTree/KeyCondition.cpp - MergeTree/LevelMergeSelector.cpp - MergeTree/MergeAlgorithm.cpp - MergeTree/MergeFromLogEntryTask.cpp - MergeTree/MergeList.cpp - MergeTree/MergePlainMergeTreeTask.cpp - MergeTree/MergeTask.cpp - MergeTree/MergeTreeBackgroundExecutor.cpp - MergeTree/MergeTreeBaseSelectProcessor.cpp - MergeTree/MergeTreeBlockReadUtils.cpp - MergeTree/MergeTreeData.cpp - MergeTree/MergeTreeDataMergerMutator.cpp - MergeTree/MergeTreeDataPartChecksum.cpp - MergeTree/MergeTreeDataPartCompact.cpp - MergeTree/MergeTreeDataPartInMemory.cpp - MergeTree/MergeTreeDataPartTTLInfo.cpp - MergeTree/MergeTreeDataPartType.cpp - MergeTree/MergeTreeDataPartUUID.cpp - MergeTree/MergeTreeDataPartWide.cpp - MergeTree/MergeTreeDataPartWriterCompact.cpp - MergeTree/MergeTreeDataPartWriterInMemory.cpp - MergeTree/MergeTreeDataPartWriterOnDisk.cpp - MergeTree/MergeTreeDataPartWriterWide.cpp - MergeTree/MergeTreeDataSelectExecutor.cpp - MergeTree/MergeTreeDataWriter.cpp - MergeTree/MergeTreeDeduplicationLog.cpp - MergeTree/MergeTreeInOrderSelectProcessor.cpp - MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp - MergeTree/MergeTreeIndexBloomFilter.cpp - MergeTree/MergeTreeIndexConditionBloomFilter.cpp - MergeTree/MergeTreeIndexFullText.cpp - MergeTree/MergeTreeIndexGranularity.cpp - MergeTree/MergeTreeIndexGranularityInfo.cpp - MergeTree/MergeTreeIndexGranuleBloomFilter.cpp - MergeTree/MergeTreeIndexMinMax.cpp - MergeTree/MergeTreeIndexReader.cpp - MergeTree/MergeTreeIndexSet.cpp - MergeTree/MergeTreeIndices.cpp - MergeTree/MergeTreeMarksLoader.cpp - MergeTree/MergeTreeMutationEntry.cpp - MergeTree/MergeTreeMutationStatus.cpp - MergeTree/MergeTreePartInfo.cpp - MergeTree/MergeTreePartition.cpp - MergeTree/MergeTreePartsMover.cpp - MergeTree/MergeTreeRangeReader.cpp - MergeTree/MergeTreeReadPool.cpp - MergeTree/MergeTreeReaderCompact.cpp - MergeTree/MergeTreeReaderInMemory.cpp - MergeTree/MergeTreeReaderStream.cpp - MergeTree/MergeTreeReaderWide.cpp - MergeTree/MergeTreeReverseSelectProcessor.cpp - MergeTree/MergeTreeSelectProcessor.cpp - MergeTree/MergeTreeSequentialSource.cpp - MergeTree/MergeTreeSettings.cpp - MergeTree/MergeTreeSink.cpp - MergeTree/MergeTreeThreadSelectProcessor.cpp - MergeTree/MergeTreeWhereOptimizer.cpp - MergeTree/MergeTreeWriteAheadLog.cpp - MergeTree/MergeType.cpp - MergeTree/MergedBlockOutputStream.cpp - MergeTree/MergedColumnOnlyOutputStream.cpp - MergeTree/MutateFromLogEntryTask.cpp - MergeTree/MutatePlainMergeTreeTask.cpp - MergeTree/MutateTask.cpp - MergeTree/PartMovesBetweenShardsOrchestrator.cpp - MergeTree/PartitionPruner.cpp - MergeTree/PinnedPartUUIDs.cpp - MergeTree/ReplicatedFetchList.cpp - MergeTree/ReplicatedMergeMutateTaskBase.cpp - MergeTree/ReplicatedMergeTreeAddress.cpp - MergeTree/ReplicatedMergeTreeAltersSequence.cpp - MergeTree/ReplicatedMergeTreeCleanupThread.cpp - MergeTree/ReplicatedMergeTreeLogEntry.cpp - MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp - MergeTree/ReplicatedMergeTreeMutationEntry.cpp - MergeTree/ReplicatedMergeTreePartCheckThread.cpp - MergeTree/ReplicatedMergeTreePartHeader.cpp - MergeTree/ReplicatedMergeTreeQueue.cpp - MergeTree/ReplicatedMergeTreeRestartingThread.cpp - MergeTree/ReplicatedMergeTreeSink.cpp - MergeTree/ReplicatedMergeTreeTableMetadata.cpp - MergeTree/SimpleMergeSelector.cpp - MergeTree/TTLMergeSelector.cpp - MergeTree/checkDataPart.cpp - MergeTree/localBackup.cpp - MergeTree/registerStorageMergeTree.cpp - MutationCommands.cpp - MySQL/MySQLSettings.cpp - PartitionCommands.cpp - ProjectionsDescription.cpp - ReadFinalForExternalReplicaStorage.cpp - ReadInOrderOptimizer.cpp - SelectQueryDescription.cpp - SetSettings.cpp - StorageBuffer.cpp - StorageDictionary.cpp - StorageDistributed.cpp - StorageExecutable.cpp - StorageExternalDistributed.cpp - StorageFactory.cpp - StorageFile.cpp - StorageGenerateRandom.cpp - StorageInMemoryMetadata.cpp - StorageInput.cpp - StorageJoin.cpp - StorageLog.cpp - StorageLogSettings.cpp - StorageMaterializedMySQL.cpp - StorageMaterializedView.cpp - StorageMemory.cpp - StorageMerge.cpp - StorageMergeTree.cpp - StorageMongoDB.cpp - StorageMongoDBSocketFactory.cpp - StorageMySQL.cpp - StorageNull.cpp - StorageReplicatedMergeTree.cpp - StorageSQLite.cpp - StorageSet.cpp - StorageStripeLog.cpp - StorageTinyLog.cpp - StorageURL.cpp - StorageValues.cpp - StorageView.cpp - StorageXDBC.cpp - System/StorageSystemAggregateFunctionCombinators.cpp - System/StorageSystemAsynchronousInserts.cpp - System/StorageSystemAsynchronousMetrics.cpp - System/StorageSystemBuildOptions.cpp - System/StorageSystemClusters.cpp - System/StorageSystemCollations.cpp - System/StorageSystemColumns.cpp - System/StorageSystemContributors.cpp - System/StorageSystemContributors.generated.cpp - System/StorageSystemCurrentRoles.cpp - System/StorageSystemDDLWorkerQueue.cpp - System/StorageSystemDataSkippingIndices.cpp - System/StorageSystemDataTypeFamilies.cpp - System/StorageSystemDatabases.cpp - System/StorageSystemDetachedParts.cpp - System/StorageSystemDictionaries.cpp - System/StorageSystemDisks.cpp - System/StorageSystemDistributionQueue.cpp - System/StorageSystemEnabledRoles.cpp - System/StorageSystemErrors.cpp - System/StorageSystemEvents.cpp - System/StorageSystemFormats.cpp - System/StorageSystemFunctions.cpp - System/StorageSystemGrants.cpp - System/StorageSystemGraphite.cpp - System/StorageSystemMacros.cpp - System/StorageSystemMergeTreeSettings.cpp - System/StorageSystemMerges.cpp - System/StorageSystemMetrics.cpp - System/StorageSystemModels.cpp - System/StorageSystemMutations.cpp - System/StorageSystemNumbers.cpp - System/StorageSystemOne.cpp - System/StorageSystemPartMovesBetweenShards.cpp - System/StorageSystemParts.cpp - System/StorageSystemPartsBase.cpp - System/StorageSystemPartsColumns.cpp - System/StorageSystemPrivileges.cpp - System/StorageSystemProcesses.cpp - System/StorageSystemProjectionParts.cpp - System/StorageSystemProjectionPartsColumns.cpp - System/StorageSystemQuotaLimits.cpp - System/StorageSystemQuotaUsage.cpp - System/StorageSystemQuotas.cpp - System/StorageSystemQuotasUsage.cpp - System/StorageSystemReplicas.cpp - System/StorageSystemReplicatedFetches.cpp - System/StorageSystemReplicationQueue.cpp - System/StorageSystemRoleGrants.cpp - System/StorageSystemRoles.cpp - System/StorageSystemRowPolicies.cpp - System/StorageSystemSettings.cpp - System/StorageSystemSettingsProfileElements.cpp - System/StorageSystemSettingsProfiles.cpp - System/StorageSystemStackTrace.cpp - System/StorageSystemStoragePolicies.cpp - System/StorageSystemTableEngines.cpp - System/StorageSystemTableFunctions.cpp - System/StorageSystemTables.cpp - System/StorageSystemUserDirectories.cpp - System/StorageSystemUsers.cpp - System/StorageSystemWarnings.cpp - System/StorageSystemZeros.cpp - System/StorageSystemZooKeeper.cpp - System/attachInformationSchemaTables.cpp - System/attachSystemTables.cpp - TTLDescription.cpp - VirtualColumnUtils.cpp - extractKeyExpressionList.cpp - getStructureOfRemoteTable.cpp - registerStorages.cpp - transformQueryForExternalDatabase.cpp - -) - -END() diff --git a/src/Storages/ya.make.in b/src/Storages/ya.make.in deleted file mode 100644 index 9a31eef3836..00000000000 --- a/src/Storages/ya.make.in +++ /dev/null @@ -1,16 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/sparsehash - contrib/libs/poco/MongoDB -) - - -SRCS( - -) - -END() diff --git a/src/TableFunctions/ya.make b/src/TableFunctions/ya.make deleted file mode 100644 index 3f616cc3750..00000000000 --- a/src/TableFunctions/ya.make +++ /dev/null @@ -1,36 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - ITableFunction.cpp - ITableFunctionFileLike.cpp - ITableFunctionXDBC.cpp - TableFunctionDictionary.cpp - TableFunctionExecutable.cpp - TableFunctionFactory.cpp - TableFunctionFile.cpp - TableFunctionGenerateRandom.cpp - TableFunctionInput.cpp - TableFunctionMerge.cpp - TableFunctionMySQL.cpp - TableFunctionNull.cpp - TableFunctionNumbers.cpp - TableFunctionRemote.cpp - TableFunctionSQLite.cpp - TableFunctionURL.cpp - TableFunctionValues.cpp - TableFunctionView.cpp - TableFunctionZeros.cpp - parseColumnsListForTableFunction.cpp - registerTableFunctions.cpp - -) - -END() diff --git a/src/TableFunctions/ya.make.in b/src/TableFunctions/ya.make.in deleted file mode 100644 index 6f351b3f646..00000000000 --- a/src/TableFunctions/ya.make.in +++ /dev/null @@ -1,14 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Common -) - - -SRCS( - -) - -END() diff --git a/src/ya.make b/src/ya.make deleted file mode 100644 index fb6f077502b..00000000000 --- a/src/ya.make +++ /dev/null @@ -1,32 +0,0 @@ -OWNER(g:clickhouse) - -LIBRARY() - -PEERDIR( - clickhouse/src/Access - clickhouse/src/AggregateFunctions - clickhouse/src/Backups - clickhouse/src/Bridge - clickhouse/src/Client - clickhouse/src/Columns - clickhouse/src/Common - clickhouse/src/Compression - clickhouse/src/Coordination - clickhouse/src/Core - clickhouse/src/Databases - clickhouse/src/DataStreams - clickhouse/src/DataTypes - clickhouse/src/Formats - clickhouse/src/Dictionaries - clickhouse/src/Disks - clickhouse/src/Functions - clickhouse/src/Interpreters - clickhouse/src/IO - clickhouse/src/Parsers - clickhouse/src/Processors - clickhouse/src/Server - clickhouse/src/Storages - clickhouse/src/TableFunctions -) - -END() diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 9b9daa9470a..8c1d56c90d8 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -167,26 +167,6 @@ find $ROOT_PATH -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' # There shouldn't be any docker compose files outside docker directory #find $ROOT_PATH -not -path $ROOT_PATH'/tests/testflows*' -not -path $ROOT_PATH'/docker*' -not -path $ROOT_PATH'/contrib*' -name '*compose*.yml' -type f 2>/dev/null | xargs --no-run-if-empty grep -l "version:" | xargs --no-run-if-empty -n1 echo "Please move docker compose to docker directory:" -# Check that ya.make files are auto-generated -"$ROOT_PATH"/utils/generate-ya-make/generate-ya-make.sh -# FIXME: apparently sandbox (don't confuse it with docker) cloning sources -# using some ancient git version, <2.8, that contains one bug for submodules -# initialization [1]: -# -# " * A partial rewrite of "git submodule" in the 2.7 timeframe changed -# the way the gitdir: pointer in the submodules point at the real -# repository location to use absolute paths by accident. This has -# been corrected." -# -# [1]: https://github.com/git/git/blob/cf11a67975b057a144618badf16dc4e3d25b9407/Documentation/RelNotes/2.8.3.txt#L33-L36 -# -# Due to which "git status" will report the following error: -# -# fatal: not a git repository: /place/sandbox-data/tasks/0/2/882869720/ClickHouse/.git/modules/contrib/AMQP-CPP -# -# Anyway this check does not requires any submodule traverse, so it is fine to ignore those errors. -git status -uno 2> >(grep "fatal: not a git repository: /place/sandbox-data/tasks/.*/ClickHouse/\\.git/modules/contrib") | grep ya.make && echo "ya.make files should be generated with utils/generate-ya-make/generate-ya-make.sh" - # Check that every header file has #pragma once in first line find $ROOT_PATH/{src,programs,utils} -name '*.h' | grep -vP $EXCLUDE_DIRS | diff --git a/utils/generate-ya-make/generate-ya-make.sh b/utils/generate-ya-make/generate-ya-make.sh deleted file mode 100755 index d20f8d4fb8c..00000000000 --- a/utils/generate-ya-make/generate-ya-make.sh +++ /dev/null @@ -1,16 +0,0 @@ -#!/bin/bash - -# This script searches for ya.make.in files in repository and generates ya.make files from them. -# ya.make.in is a template with substitutions in form of -# command is interpreted by bash and output is put in place of substitution - -ROOT_PATH=$(git rev-parse --show-toplevel) -EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing' - -# Otherwise 'sort' behaves differently on Mac OS and also depends on locale. -export LC_ALL=C - -find "${ROOT_PATH}" -name 'ya.make.in' | while read path; do - echo "# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it." > "${path/.in/}" - (cd $(dirname "${path}") && perl -pne 's/<\?(.+?)\?>/`$1`/e' < "${path}" >> "${path/.in/}") -done diff --git a/utils/github/ya.make b/utils/github/ya.make deleted file mode 100644 index 3b19a5b69d1..00000000000 --- a/utils/github/ya.make +++ /dev/null @@ -1,13 +0,0 @@ -OWNER(g:clickhouse) - -PY23_LIBRARY() - -PY_SRCS( - __init__.py - backport.py - cherrypick.py - local.py - query.py -) - -END() diff --git a/ya.make b/ya.make deleted file mode 100644 index 7f7c1bbce2f..00000000000 --- a/ya.make +++ /dev/null @@ -1,12 +0,0 @@ -# What are "ya.make" files? -# "ya.make" file is a configuration file for "ya" - an internal Yandex build system that is used to build internal Yandex monorepository. -# We don't use nor "ya" build system neither the Yandex monorepository for development of ClickHouse and you should not pay attention to these files. -# But ClickHouse source code is synchronized with internal Yandex monorepository, that's why "ya.make" files exist. - -OWNER(g:clickhouse) - -RECURSE( - base - programs - src -) From 06fc4664386bef80fe17ef84e2f17517bbc82ec9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Sep 2021 04:39:38 +0300 Subject: [PATCH 155/317] Remove window function 'nth_value' --- src/Processors/Transforms/WindowTransform.cpp | 69 ------------------- .../01591_window_functions.reference | 64 ----------------- .../0_stateless/01591_window_functions.sql | 41 ----------- 3 files changed, 174 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 2f8231d8cfb..ff62df8d939 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1658,68 +1658,6 @@ struct WindowFunctionLagLeadInFrame final : public WindowFunction } }; -struct WindowFunctionNthValue final : public WindowFunction -{ - WindowFunctionNthValue(const std::string & name_, - const DataTypes & argument_types_, const Array & parameters_) - : WindowFunction(name_, argument_types_, parameters_) - { - if (!parameters.empty()) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Function {} cannot be parameterized", name_); - } - - if (argument_types.size() != 2) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Function '{}' accepts 2 arguments, {} given", - name_, argument_types.size()); - } - } - - DataTypePtr getReturnType() const override - { return argument_types[0]; } - - bool allocatesMemoryInArena() const override { return false; } - - void windowInsertResultInto(const WindowTransform * transform, - size_t function_index) override - { - const auto & current_block = transform->blockAt(transform->current_row); - IColumn & to = *(current_block.output_columns[function_index]); - const auto & workspace = transform->workspaces[function_index]; - - int64_t offset = (*current_block.input_columns[ - workspace.argument_column_indices[1]])[ - transform->current_row.row].get(); - - /// Either overflow or really negative value, both is not acceptable. - if (offset <= 0) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "The offset for function {} must be in (0, {}], {} given", - getName(), INT64_MAX, offset); - } - - --offset; - const auto [target_row, offset_left] = transform->moveRowNumber(transform->frame_start, offset); - if (offset_left != 0 - || target_row < transform->frame_start - || transform->frame_end <= target_row) - { - // Offset is outside the frame. - to.insertDefault(); - } - else - { - // Offset is inside the frame. - to.insertFrom(*transform->blockAt(target_row).input_columns[ - workspace.argument_column_indices[0]], - target_row.row); - } - } -}; void registerWindowFunctions(AggregateFunctionFactory & factory) { @@ -1784,13 +1722,6 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) return std::make_shared>( name, argument_types, parameters); }, properties}); - - factory.registerFunction("nth_value", {[](const std::string & name, - const DataTypes & argument_types, const Array & parameters, const Settings *) - { - return std::make_shared( - name, argument_types, parameters); - }, properties}, AggregateFunctionFactory::CaseInsensitive); } } diff --git a/tests/queries/0_stateless/01591_window_functions.reference b/tests/queries/0_stateless/01591_window_functions.reference index 92671805436..f54c10ee8b9 100644 --- a/tests/queries/0_stateless/01591_window_functions.reference +++ b/tests/queries/0_stateless/01591_window_functions.reference @@ -1094,70 +1094,6 @@ order by number 7 6 8 8 7 9 9 8 9 --- nth_value without specific frame range given -select - number, - nth_value(number, 1) over w as firstValue, - nth_value(number, 2) over w as secondValue, - nth_value(number, 3) over w as thirdValue, - nth_value(number, 4) over w as fourthValue -from numbers(10) -window w as (order by number) -order by number -; -0 0 0 0 0 -1 0 1 0 0 -2 0 1 2 0 -3 0 1 2 3 -4 0 1 2 3 -5 0 1 2 3 -6 0 1 2 3 -7 0 1 2 3 -8 0 1 2 3 -9 0 1 2 3 --- nth_value with frame range specified -select - number, - nth_value(number, 1) over w as firstValue, - nth_value(number, 2) over w as secondValue, - nth_value(number, 3) over w as thirdValue, - nth_value(number, 4) over w as fourthValue -from numbers(10) -window w as (order by number range between 1 preceding and 1 following) -order by number -; -0 0 1 0 0 -1 0 1 2 0 -2 1 2 3 0 -3 2 3 4 0 -4 3 4 5 0 -5 4 5 6 0 -6 5 6 7 0 -7 6 7 8 0 -8 7 8 9 0 -9 8 9 0 0 --- to make nth_value return null for out-of-frame rows, cast the argument to --- Nullable; otherwise, it returns default values. -SELECT - number, - nth_value(toNullable(number), 1) OVER w as firstValue, - nth_value(toNullable(number), 3) OVER w as thridValue -FROM numbers(5) -WINDOW w AS (ORDER BY number ASC) -; -0 0 \N -1 0 \N -2 0 2 -3 0 2 -4 0 2 --- nth_value UBsan -SELECT nth_value(1, -1) OVER (); -- { serverError BAD_ARGUMENTS } -SELECT nth_value(1, 0) OVER (); -- { serverError BAD_ARGUMENTS } -SELECT nth_value(1, /* INT64_MAX+1 */ 0x7fffffffffffffff+1) OVER (); -- { serverError BAD_ARGUMENTS } -SELECT nth_value(1, /* INT64_MAX */ 0x7fffffffffffffff) OVER (); -0 -SELECT nth_value(1, 1) OVER (); -1 -- lagInFrame UBsan SELECT lagInFrame(1, -1) OVER (); -- { serverError BAD_ARGUMENTS } SELECT lagInFrame(1, 0) OVER (); diff --git a/tests/queries/0_stateless/01591_window_functions.sql b/tests/queries/0_stateless/01591_window_functions.sql index 43946ddb3fe..aa9bd9795e7 100644 --- a/tests/queries/0_stateless/01591_window_functions.sql +++ b/tests/queries/0_stateless/01591_window_functions.sql @@ -401,47 +401,6 @@ window w as (order by number range between 1 preceding and 1 following) order by number ; --- nth_value without specific frame range given -select - number, - nth_value(number, 1) over w as firstValue, - nth_value(number, 2) over w as secondValue, - nth_value(number, 3) over w as thirdValue, - nth_value(number, 4) over w as fourthValue -from numbers(10) -window w as (order by number) -order by number -; - --- nth_value with frame range specified -select - number, - nth_value(number, 1) over w as firstValue, - nth_value(number, 2) over w as secondValue, - nth_value(number, 3) over w as thirdValue, - nth_value(number, 4) over w as fourthValue -from numbers(10) -window w as (order by number range between 1 preceding and 1 following) -order by number -; - --- to make nth_value return null for out-of-frame rows, cast the argument to --- Nullable; otherwise, it returns default values. -SELECT - number, - nth_value(toNullable(number), 1) OVER w as firstValue, - nth_value(toNullable(number), 3) OVER w as thridValue -FROM numbers(5) -WINDOW w AS (ORDER BY number ASC) -; - --- nth_value UBsan -SELECT nth_value(1, -1) OVER (); -- { serverError BAD_ARGUMENTS } -SELECT nth_value(1, 0) OVER (); -- { serverError BAD_ARGUMENTS } -SELECT nth_value(1, /* INT64_MAX+1 */ 0x7fffffffffffffff+1) OVER (); -- { serverError BAD_ARGUMENTS } -SELECT nth_value(1, /* INT64_MAX */ 0x7fffffffffffffff) OVER (); -SELECT nth_value(1, 1) OVER (); - -- lagInFrame UBsan SELECT lagInFrame(1, -1) OVER (); -- { serverError BAD_ARGUMENTS } SELECT lagInFrame(1, 0) OVER (); From 1881c67eff6390caf22d8186131626b539a50c61 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 25 Sep 2021 04:44:06 +0300 Subject: [PATCH 156/317] Remove trash --- tests/integration/test_rocksdb_options/test.py | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/tests/integration/test_rocksdb_options/test.py b/tests/integration/test_rocksdb_options/test.py index e8542749d8d..e9e9cfdf20f 100644 --- a/tests/integration/test_rocksdb_options/test.py +++ b/tests/integration/test_rocksdb_options/test.py @@ -58,16 +58,6 @@ 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 fe44be522ee11822ac8b86c1ffab8560b5fb37d9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Sep 2021 05:48:24 +0300 Subject: [PATCH 157/317] 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 eaba6a7f59df475acb0c8cbfbfb8fcd8d71abe3d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Sep 2021 07:02:05 +0300 Subject: [PATCH 158/317] Remove useless files related to pytest --- tests/queries/__init__.py | 0 tests/queries/conftest.py | 77 --- tests/queries/pytest.ini | 2 - tests/queries/query_test.py | 260 ------- tests/queries/server.py | 1303 ----------------------------------- 5 files changed, 1642 deletions(-) delete mode 100644 tests/queries/__init__.py delete mode 100644 tests/queries/conftest.py delete mode 100644 tests/queries/pytest.ini delete mode 100644 tests/queries/query_test.py delete mode 100644 tests/queries/server.py diff --git a/tests/queries/__init__.py b/tests/queries/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/conftest.py b/tests/queries/conftest.py deleted file mode 100644 index 2d13443e981..00000000000 --- a/tests/queries/conftest.py +++ /dev/null @@ -1,77 +0,0 @@ -import os -import re -import sys - -import pytest - -from .server import ServerThread - - -# Command-line arguments - -def pytest_addoption(parser): - parser.addoption("--builddir", action="store", default=None, help="Path to build directory to use binaries from") - - -# HTML report hooks - -def pytest_html_report_title(report): - report.title = "ClickHouse Functional Stateless Tests (PyTest)" - - -RE_TEST_NAME = re.compile(r"\[(.*)\]") -def pytest_itemcollected(item): - match = RE_TEST_NAME.search(item.name) - if match: - item._nodeid = match.group(1) - - -# Fixtures - -@pytest.fixture(scope='module') -def cmdopts(request): - return { - 'builddir': request.config.getoption("--builddir") - } - - -@pytest.fixture(scope='module') -def bin_prefix(cmdopts): - prefix = 'clickhouse' - if cmdopts['builddir'] is not None: - prefix = os.path.join(cmdopts['builddir'], 'programs', prefix) - # FIXME: does this hangs the server start for some reason? - # if not os.path.isabs(prefix): - # prefix = os.path.abspath(prefix) - return prefix - - -# TODO: also support stateful queries. -QUERIES_PATH = os.path.join(os.path.dirname(os.path.abspath(__file__)), '0_stateless') - -@pytest.fixture(scope='module', params=[f for f in os.listdir(QUERIES_PATH) if f.endswith('.sql')]) -def sql_query(request): - return os.path.join(QUERIES_PATH, os.path.splitext(request.param)[0]) - - -@pytest.fixture(scope='module', params=[f for f in os.listdir(QUERIES_PATH) if f.endswith('.sh')]) -def shell_query(request): - return os.path.join(QUERIES_PATH, os.path.splitext(request.param)[0]) - - -@pytest.fixture -def standalone_server(bin_prefix, tmp_path): - server = ServerThread(bin_prefix, str(tmp_path)) - server.start() - wait_result = server.wait() - - if wait_result is not None: - with open(os.path.join(server.log_dir, 'server', 'stdout.txt'), 'r') as f: - print(f.read(), file=sys.stderr) - with open(os.path.join(server.log_dir, 'server', 'stderr.txt'), 'r') as f: - print(f.read(), file=sys.stderr) - pytest.fail('Server died unexpectedly with code {code}'.format(code=server._proc.returncode), pytrace=False) - - yield server - - server.stop() diff --git a/tests/queries/pytest.ini b/tests/queries/pytest.ini deleted file mode 100644 index 13a2ebbaf83..00000000000 --- a/tests/queries/pytest.ini +++ /dev/null @@ -1,2 +0,0 @@ -[pytest] -render_collapsed = True diff --git a/tests/queries/query_test.py b/tests/queries/query_test.py deleted file mode 100644 index 1735febee44..00000000000 --- a/tests/queries/query_test.py +++ /dev/null @@ -1,260 +0,0 @@ -import difflib -import os -import random -import string -import subprocess -import sys - -import pytest - - -SKIP_LIST = [ - # these couple of tests hangs everything - "00600_replace_running_query", - "00987_distributed_stack_overflow", - "01954_clickhouse_benchmark_multiple_long", - - # just fail - "00133_long_shard_memory_tracker_and_exception_safety", - "00463_long_sessions_in_http_interface", - "00505_secure", - "00505_shard_secure", - "00646_url_engine", - "00725_memory_tracking", # BROKEN - "00738_lock_for_inner_table", - "00821_distributed_storage_with_join_on", - "00825_protobuf_format_array_3dim", - "00825_protobuf_format_array_of_arrays", - "00825_protobuf_format_enum_mapping", - "00825_protobuf_format_nested_in_nested", - "00825_protobuf_format_nested_optional", - "00825_protobuf_format_no_length_delimiter", - "00825_protobuf_format_persons", - "00825_protobuf_format_squares", - "00825_protobuf_format_table_default", - "00834_cancel_http_readonly_queries_on_client_close", - "00877_memory_limit_for_new_delete", - "00900_parquet_load", - "00933_test_fix_extra_seek_on_compressed_cache", - "00965_logs_level_bugfix", - "00965_send_logs_level_concurrent_queries", - "00974_query_profiler", - "00990_hasToken", - "00990_metric_log_table_not_empty", - "01014_lazy_database_concurrent_recreate_reattach_and_show_tables", - "01017_uniqCombined_memory_usage", - "01018_Distributed__shard_num", - "01018_ip_dictionary_long", - "01035_lc_empty_part_bug", # FLAKY - "01050_clickhouse_dict_source_with_subquery", - "01053_ssd_dictionary", - "01054_cache_dictionary_overflow_cell", - "01057_http_compression_prefer_brotli", - "01080_check_for_error_incorrect_size_of_nested_column", - "01083_expressions_in_engine_arguments", - "01086_odbc_roundtrip", - "01088_benchmark_query_id", - "01092_memory_profiler", - "01098_temporary_and_external_tables", - "01099_parallel_distributed_insert_select", - "01103_check_cpu_instructions_at_startup", - "01107_atomic_db_detach_attach", - "01114_database_atomic", - "01148_zookeeper_path_macros_unfolding", - "01152_cross_replication", # tcp port in reference - "01175_distributed_ddl_output_mode_long", - "01181_db_atomic_drop_on_cluster", # tcp port in reference - "01280_ssd_complex_key_dictionary", - "01293_client_interactive_vertical_multiline", # expect-test - "01293_client_interactive_vertical_singleline", # expect-test - "01293_show_clusters", - "01293_show_settings", - "01293_system_distribution_queue", # FLAKY - "01294_lazy_database_concurrent_recreate_reattach_and_show_tables_long", - "01294_system_distributed_on_cluster", - "01300_client_save_history_when_terminated", # expect-test - "01304_direct_io", - "01306_benchmark_json", - "01035_lc_empty_part_bug", # FLAKY - "01175_distributed_ddl_output_mode_long", # tcp port in reference - "01320_create_sync_race_condition_zookeeper", - "01355_CSV_input_format_allow_errors", - "01370_client_autocomplete_word_break_characters", # expect-test - "01376_GROUP_BY_injective_elimination_dictGet", - "01393_benchmark_secure_port", - "01418_custom_settings", - "01451_wrong_error_long_query", - "01455_opentelemetry_distributed", - "01473_event_time_microseconds", - "01474_executable_dictionary", - "01507_clickhouse_server_start_with_embedded_config", - "01514_distributed_cancel_query_on_error", - "01520_client_print_query_id", # expect-test - "01526_client_start_and_exit", # expect-test - "01526_max_untracked_memory", - "01527_dist_sharding_key_dictGet_reload", - "01528_play", - "01545_url_file_format_settings", - "01553_datetime64_comparison", - "01555_system_distribution_queue_mask", - "01558_ttest_scipy", - "01561_mann_whitney_scipy", - "01582_distinct_optimization", - "01591_window_functions", - "01594_too_low_memory_limits", - "01599_multiline_input_and_singleline_comments", # expect-test - "01601_custom_tld", - "01606_git_import", - "01610_client_spawn_editor", # expect-test - "01654_test_writer_block_sequence", # No module named 'pandas' - "01658_read_file_to_stringcolumn", - "01666_merge_tree_max_query_limit", - "01674_unicode_asan", - "01676_clickhouse_client_autocomplete", # expect-test (partially) - "01676_long_clickhouse_client_autocomplete", - "01683_text_log_deadlock", # secure tcp - "01684_ssd_cache_dictionary_simple_key", - "01685_ssd_cache_dictionary_complex_key", - "01737_clickhouse_server_wait_server_pool_long", - "01746_executable_pool_dictionary", - "01747_executable_pool_dictionary_implicit_key.sql", - "01747_join_view_filter_dictionary", - "01748_dictionary_table_dot", - "01754_cluster_all_replicas_shard_num", - "01759_optimize_skip_unused_shards_zero_shards", - "01763_max_distributed_depth", # BROKEN - "01780_clickhouse_dictionary_source_loop", - "01801_s3_cluster", - "01802_test_postgresql_protocol_with_row_policy", - "01804_dictionary_decimal256_type", # hardcoded path - "01848_http_insert_segfault", - "01875_ssd_cache_dictionary_decimal256_type", - "01880_remote_ipv6", - "01889_check_row_policy_defined_using_user_function", - "01889_clickhouse_client_config_format", - "01903_ssd_cache_dictionary_array_type", -] - - -def check_result(result, error, return_code, reference, replace_map): - if replace_map: - for old, new in replace_map.items(): - result = result.replace(old.encode('utf-8'), new.encode('utf-8')) - - if return_code != 0: - try: - print(error.decode('utf-8'), file=sys.stderr) - except UnicodeDecodeError: - print(error.decode('latin1'), file=sys.stderr) # encoding with 1 symbol per 1 byte, covering all values - pytest.fail('Client died unexpectedly with code {code}'.format(code=return_code), pytrace=False) - elif result != reference: - pytest.fail("Query output doesn't match reference:{eol}{diff}".format( - eol=os.linesep, - diff=os.linesep.join(l.strip() for l in difflib.unified_diff(reference.decode('utf-8').splitlines(), - result.decode('utf-8').splitlines(), - fromfile='expected', tofile='actual'))), - pytrace=False) - - -def run_client(bin_prefix, port, database, query, reference, replace_map=None): - # We can't use `text=True` since some tests may return binary data - cmd = [bin_prefix + '-client', '--port', str(port), '-d', database, '-m', '-n', '--testmode'] - client = subprocess.Popen(cmd, stdin=subprocess.PIPE, stdout=subprocess.PIPE, stderr=subprocess.PIPE) - result, error = client.communicate(query.encode('utf-8')) - assert client.returncode is not None, "Client should exit after processing all queries" - - check_result(result, error, client.returncode, reference, replace_map) - - -def run_shell(bin_prefix, server, database, path, reference, replace_map=None): - env = { - 'CLICKHOUSE_BINARY': bin_prefix, - 'CLICKHOUSE_DATABASE': database, - 'CLICKHOUSE_PORT_TCP': str(server.tcp_port), - 'CLICKHOUSE_PORT_TCP_SECURE': str(server.tcps_port), - 'CLICKHOUSE_PORT_TCP_WITH_PROXY': str(server.proxy_port), - 'CLICKHOUSE_PORT_HTTP': str(server.http_port), - 'CLICKHOUSE_PORT_INTERSERVER': str(server.inter_port), - 'CLICKHOUSE_PORT_POSTGRESQL': str(server.postgresql_port), - 'CLICKHOUSE_TMP': server.tmp_dir, - 'CLICKHOUSE_CONFIG_CLIENT': server.client_config, - 'PROTOC_BINARY': os.path.abspath(os.path.join(os.path.dirname(bin_prefix), '..', 'contrib', 'protobuf', 'protoc')), # FIXME: adhoc solution - } - shell = subprocess.Popen([path], env=env, shell=True, stdout=subprocess.PIPE, stderr=subprocess.PIPE) - result, error = shell.communicate() - assert shell.returncode is not None, "Script should exit after executing all commands" - - check_result(result, error, shell.returncode, reference, replace_map) - - -def random_str(length=10): - alphabet = string.ascii_lowercase + string.digits - random.seed(os.urandom(8)) - return ''.join(random.choice(alphabet) for _ in range(length)) - - -def test_sql_query(bin_prefix, sql_query, standalone_server): - for test in SKIP_LIST: - if test in sql_query: - pytest.skip("Test matches skip-list: " + test) - return - - tcp_port = standalone_server.tcp_port - - query_path = sql_query + ".sql" - reference_path = sql_query + ".reference" - - if not os.path.exists(reference_path): - pytest.skip('No .reference file found') - - with open(query_path, 'r') as file: - query = file.read() - with open(reference_path, 'rb') as file: - reference = file.read() - - random_name = 'test_{random}'.format(random=random_str()) - run_client(bin_prefix, tcp_port, 'default', 'CREATE DATABASE {random};'.format(random=random_name), b'') - - run_client(bin_prefix, tcp_port, random_name, query, reference, {random_name: 'default'}) - - query = "SELECT 'SHOW ORPHANED TABLES'; SELECT name FROM system.tables WHERE database != 'system' ORDER BY (database, name);" - run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED TABLES\n') - - query = 'DROP DATABASE {random};'.format(random=random_name) - run_client(bin_prefix, tcp_port, 'default', query, b'') - - query = "SELECT 'SHOW ORPHANED DATABASES'; SHOW DATABASES;" - run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED DATABASES\ndefault\nsystem\n') - - -def test_shell_query(bin_prefix, shell_query, standalone_server): - for test in SKIP_LIST: - if test in shell_query: - pytest.skip("Test matches skip-list: " + test) - return - - tcp_port = standalone_server.tcp_port - - shell_path = shell_query + ".sh" - reference_path = shell_query + ".reference" - - if not os.path.exists(reference_path): - pytest.skip('No .reference file found') - - with open(reference_path, 'rb') as file: - reference = file.read() - - random_name = 'test_{random}'.format(random=random_str()) - query = 'CREATE DATABASE {random};'.format(random=random_name) - run_client(bin_prefix, tcp_port, 'default', query, b'') - - run_shell(bin_prefix, standalone_server, random_name, shell_path, reference, {random_name: 'default'}) - - query = "SELECT 'SHOW ORPHANED TABLES'; SELECT name FROM system.tables WHERE database != 'system' ORDER BY (database, name);" - run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED TABLES\n') - - query = 'DROP DATABASE {random};'.format(random=random_name) - run_client(bin_prefix, tcp_port, 'default', query, b'') - - query = "SELECT 'SHOW ORPHANED DATABASES'; SHOW DATABASES;" - run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED DATABASES\ndefault\nsystem\n') diff --git a/tests/queries/server.py b/tests/queries/server.py deleted file mode 100644 index c08004aff61..00000000000 --- a/tests/queries/server.py +++ /dev/null @@ -1,1303 +0,0 @@ -import os -import random -import socket -import subprocess -import sys -import threading -import time - - -class ServerThread(threading.Thread): - DEFAULT_RETRIES = 3 - DEFAULT_SERVER_DELAY = 0.5 # seconds - DEFAULT_CONNECTION_TIMEOUT = 1.0 # seconds - - def __init__(self, bin_prefix, tmp_dir): - self._bin = bin_prefix + '-server' - self._lock = threading.Lock() - threading.Thread.__init__(self) - self._lock.acquire() - - self.tmp_dir = tmp_dir - self.log_dir = os.path.join(tmp_dir, 'log') - self.etc_dir = os.path.join(tmp_dir, 'etc') - self.server_config = os.path.join(self.etc_dir, 'server-config.xml') - self.users_config = os.path.join(self.etc_dir, 'users.xml') - self.dicts_config = os.path.join(self.etc_dir, 'dictionaries.xml') - self.client_config = os.path.join(self.etc_dir, 'client-config.xml') - - os.makedirs(self.log_dir) - os.makedirs(self.etc_dir) - - def _choose_ports_and_args(self): - port_base = random.SystemRandom().randrange(10000, 60000) - self.tcp_port = port_base + 1 - self.http_port = port_base + 2 - self.inter_port = port_base + 3 - self.tcps_port = port_base + 4 - self.https_port = port_base + 5 - self.odbc_port = port_base + 6 - self.proxy_port = port_base + 7 - self.postgresql_port = port_base + 8 - - self._args = [ - '--config-file={config_path}'.format(config_path=self.server_config), - '--', - '--tcp_port={tcp_port}'.format(tcp_port=self.tcp_port), - '--http_port={http_port}'.format(http_port=self.http_port), - '--interserver_http_port={inter_port}'.format(inter_port=self.inter_port), - '--tcp_with_proxy_port={proxy_port}'.format(proxy_port=self.proxy_port), - '--postgresql_port={psql_port}'.format(psql_port=self.postgresql_port), - # TODO: SSL certificate is not specified '--tcp_port_secure={tcps_port}'.format(tcps_port=self.tcps_port), - ] - - with open(self.server_config, 'w') as f: - f.write(ServerThread.DEFAULT_SERVER_CONFIG.format( - tmp_dir=self.tmp_dir, log_dir=self.log_dir, tcp_port=self.tcp_port)) - - with open(self.users_config, 'w') as f: - f.write(ServerThread.DEFAULT_USERS_CONFIG) - - with open(self.dicts_config, 'w') as f: - f.write(ServerThread.DEFAULT_DICTIONARIES_CONFIG.format(tcp_port=self.tcp_port)) - - with open(self.client_config, 'w') as f: - f.write(ServerThread.DEFAULT_CLIENT_CONFIG) - - def run(self): - retries = ServerThread.DEFAULT_RETRIES - - while retries: - self._choose_ports_and_args() - print('Start clickhouse-server with args:', self._args) - self._proc = subprocess.Popen([self._bin] + self._args, shell=False, stdout=subprocess.PIPE, stderr=subprocess.PIPE) - - while self._proc.poll() is None: - try: - time.sleep(ServerThread.DEFAULT_SERVER_DELAY) - s = socket.create_connection(('localhost', self.tcp_port), ServerThread.DEFAULT_CONNECTION_TIMEOUT) - s.sendall(b'G') # trigger expected "bad" HELLO response - s.recv(1024) # FIXME: read whole buffered response - s.shutdown(socket.SHUT_RDWR) - s.close() - except Exception: - # Failed to connect to server - try again - continue - else: - break - - # If process has died then try to fetch output before releasing lock - if self._proc.returncode is not None: - stdout, stderr = self._proc.communicate() - print(stdout.decode('utf-8'), file=sys.stderr) - print(stderr.decode('utf-8'), file=sys.stderr) - - if self._proc.returncode == 70: # Address already in use - retries -= 1 - continue - - break - - self._lock.release() - - if not retries: - print('Failed to start server', file=sys.stderr) - return - - while self._proc.returncode is None: - self._proc.communicate() - - def wait(self): - self._lock.acquire() - if self._proc.returncode is not None: - self.join() - self._lock.release() - return self._proc.returncode - - def stop(self): - if self._proc.returncode is None: - self._proc.terminate() - self.join() - print('Stopped clickhouse-server') - - -ServerThread.DEFAULT_SERVER_CONFIG = \ -"""\ - - - - trace - {log_dir}/server/stdout.txt - {log_dir}/server/stderr.txt - never - 1 - - - :: - - {tmp_dir}/data/ - {tmp_dir}/tmp/ - {tmp_dir}/data/access/ - users.xml - dictionaries.xml - 5368709120 - - 3 - Europe/Moscow - - - Hello, world! - s1 - r1 - - - - Version - - sum - sum - - 0 - 600 - - - 172800 - 6000 - - - - max - - 0 - 600 - - - 172800 - 6000 - - - - - - - TOPSECRET.TOPSECRET - [hidden] - - - - - - - - localhost - {tcp_port} - - - - - - - - localhost - {tcp_port} - - - - - localhost - {tcp_port} - - - - - - - - 127.0.0.1 - {tcp_port} - - - - - 127.0.0.2 - {tcp_port} - - - - - - - - localhost - {tcp_port} - - - - - localhost - 1 - - - - - - - - shard_0 - localhost - {tcp_port} - - - - - shard_1 - localhost - {tcp_port} - - - - - - - true - - 127.0.0.1 - {tcp_port} - - - - true - - 127.0.0.2 - {tcp_port} - - - - - - - true - - 127.0.0.1 - {tcp_port} - - foo - - - 127.0.0.2 - {tcp_port} - - foo - - - - - - - - 127.0.0.1 - {tcp_port} - - - 127.0.0.2 - {tcp_port} - - - - - - - - shard_0 - localhost - {tcp_port} - - - shard_1 - localhost - {tcp_port} - - - - - - - - - memory - - - - - - testkeeper - - - - /clickhouse/task_queue/ddl - - - - system - part_log
-
- - - system - query_log
-
- - - system - query_thread_log
-
- - - system - text_log
-
- - - system - trace_log
-
-
-""" - - -ServerThread.DEFAULT_USERS_CONFIG = \ -"""\ - - - - - 10000000000 - 0 - random - - - - 1 - - - - - - - - ::/0 - - - default - - default - - 1 - - - - - - ::1 - 127.0.0.1 - - - readonly - - default - - - - - - - 3600 - 0 - 0 - 0 - 0 - 0 - - - - -""" - - -ServerThread.DEFAULT_DICTIONARIES_CONFIG = \ -"""\ - - - flat_ints - - - localhost - {tcp_port} - default - - system - ints
-
- - 0 - - - - - - key - - - i8 - Int8 - 0 - - - i16 - Int16 - 0 - - - i32 - Int32 - 0 - - - i64 - Int64 - 0 - - - u8 - UInt8 - 0 - - - u16 - UInt16 - 0 - - - u32 - UInt32 - 0 - - - u64 - UInt64 - 0 - - -
- - - hashed_ints - - - localhost - {tcp_port} - default - - system - ints
-
- - 0 - - - - - - key - - - i8 - Int8 - 0 - - - i16 - Int16 - 0 - - - i32 - Int32 - 0 - - - i64 - Int64 - 0 - - - u8 - UInt8 - 0 - - - u16 - UInt16 - 0 - - - u32 - UInt32 - 0 - - - u64 - UInt64 - 0 - - -
- - - hashed_sparse_ints - - - localhost - {tcp_port} - default - - system - ints
-
- - 0 - - - - - - key - - - i8 - Int8 - 0 - - - i16 - Int16 - 0 - - - i32 - Int32 - 0 - - - i64 - Int64 - 0 - - - u8 - UInt8 - 0 - - - u16 - UInt16 - 0 - - - u32 - UInt32 - 0 - - - u64 - UInt64 - 0 - - -
- - - cache_ints - - - localhost - {tcp_port} - default - - system - ints
-
- - 0 - - 1000 - - - - key - - - i8 - Int8 - 0 - - - i16 - Int16 - 0 - - - i32 - Int32 - 0 - - - i64 - Int64 - 0 - - - u8 - UInt8 - 0 - - - u16 - UInt16 - 0 - - - u32 - UInt32 - 0 - - - u64 - UInt64 - 0 - - -
- - - complex_hashed_ints - - - localhost - {tcp_port} - default - - system - ints
-
- - 0 - - - - - - - key - UInt64 - - - - i8 - Int8 - 0 - - - i16 - Int16 - 0 - - - i32 - Int32 - 0 - - - i64 - Int64 - 0 - - - u8 - UInt8 - 0 - - - u16 - UInt16 - 0 - - - u32 - UInt32 - 0 - - - u64 - UInt64 - 0 - - -
- - - complex_cache_ints - - - localhost - {tcp_port} - default - - system - ints
-
- - 0 - - 1000 - - - - - key - UInt64 - - - - i8 - Int8 - 0 - - - i16 - Int16 - 0 - - - i32 - Int32 - 0 - - - i64 - Int64 - 0 - - - u8 - UInt8 - 0 - - - u16 - UInt16 - 0 - - - u32 - UInt32 - 0 - - - u64 - UInt64 - 0 - - -
- - - flat_strings - - - localhost - {tcp_port} - default - - system - strings
-
- - 0 - - - - - - key - - - str - String - - - -
- - - hashed_strings - - - localhost - {tcp_port} - default - - system - strings
-
- - 0 - - - - - - key - - - str - String - - - -
- - - cache_strings - - - localhost - {tcp_port} - default - - system - strings
-
- - 0 - - 1000 - - - - key - - - str - String - - - -
- - - complex_hashed_strings - - - localhost - {tcp_port} - default - - system - strings
-
- - 0 - - - - - - - key - UInt64 - - - - str - String - - - -
- - - complex_cache_strings - - - localhost - {tcp_port} - default - - system - strings
-
- - 0 - - 1000 - - - - - key - UInt64 - - - - str - String - - - -
- - - flat_decimals - - - localhost - {tcp_port} - default - - system - decimals
-
- - 0 - - - - - - key - - - d32 - Decimal32(4) - 0 - - - d64 - Decimal64(6) - 0 - - - d128 - Decimal128(1) - 0 - - -
- - - hashed_decimals - - - localhost - {tcp_port} - default - - system - decimals
-
- - 0 - - - - - - key - - - d32 - Decimal32(4) - 0 - - - d64 - Decimal64(6) - 0 - - - d128 - Decimal128(1) - 0 - - -
- - - cache_decimals - - - localhost - {tcp_port} - default - - system - decimals
-
- - 0 - - 1000 - - - - key - - - d32 - Decimal32(4) - 0 - - - d64 - Decimal64(6) - 0 - - - d128 - Decimal128(1) - 0 - - -
- - - complex_hashed_decimals - - - localhost - {tcp_port} - default - - system - decimals
-
- - 0 - - - - - - - key - UInt64 - - - - d32 - Decimal32(4) - 0 - - - d64 - Decimal64(6) - 0 - - - d128 - Decimal128(1) - 0 - - -
- - - complex_cache_decimals - - - localhost - {tcp_port} - default - - system - decimals
-
- - 0 - - 1000 - - - - - key - UInt64 - - - - d32 - Decimal32(4) - 0 - - - d64 - Decimal64(6) - 0 - - - d128 - Decimal128(1) - 0 - - -
- - - simple_executable_cache_dictionary_no_implicit_key - - - id - UInt64 - - - - value - String - - - - - - echo "1\tValue" - TabSeparated - false - - - - - 10000 - - - 300 - - - - simple_executable_cache_dictionary_implicit_key - - - id - UInt64 - - - - value - String - - - - - - echo "Value" - TabSeparated - true - - - - - 10000 - - - 300 - - - - complex_executable_cache_dictionary_no_implicit_key - - - - id - UInt64 - - - - id_key - String - - - - - value - String - - - - - - echo "1\tFirstKey\tValue" - TabSeparated - false - - - - - 10000 - - - 300 - - - - complex_executable_cache_dictionary_implicit_key - - - - id - UInt64 - - - - id_key - String - - - - - value - String - - - - - - echo "Value" - TabSeparated - true - - - - - 10000 - - - 300 - -
-""" - -ServerThread.DEFAULT_CLIENT_CONFIG = \ -"""\ - - - - true - true - sslv2,sslv3 - true - - AcceptCertificateHandler - - - - -""" From 29feb3309b8d10b47409c1d39a876eab70f5519a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 25 Sep 2021 07:09:41 +0300 Subject: [PATCH 159/317] Remove obsolete non-automated tests --- .../catboost/data/build_catboost.sh | 17 - .../catboost/helpers/__init__.py | 0 .../catboost/helpers/client.py | 42 --- .../catboost/helpers/generate.py | 15 - .../catboost/helpers/server.py | 67 ---- .../catboost/helpers/server_with_models.py | 168 ---------- .../external_models/catboost/helpers/table.py | 74 ----- .../external_models/catboost/helpers/train.py | 28 -- tests/external_models/catboost/pytest.ini | 3 - .../test_apply_catboost_model/test.py | 294 ------------------ 10 files changed, 708 deletions(-) delete mode 100755 tests/external_models/catboost/data/build_catboost.sh delete mode 100644 tests/external_models/catboost/helpers/__init__.py delete mode 100644 tests/external_models/catboost/helpers/client.py delete mode 100644 tests/external_models/catboost/helpers/generate.py delete mode 100644 tests/external_models/catboost/helpers/server.py delete mode 100644 tests/external_models/catboost/helpers/server_with_models.py delete mode 100644 tests/external_models/catboost/helpers/table.py delete mode 100644 tests/external_models/catboost/helpers/train.py delete mode 100644 tests/external_models/catboost/pytest.ini delete mode 100644 tests/external_models/catboost/test_apply_catboost_model/test.py diff --git a/tests/external_models/catboost/data/build_catboost.sh b/tests/external_models/catboost/data/build_catboost.sh deleted file mode 100755 index 50a3fb43ef8..00000000000 --- a/tests/external_models/catboost/data/build_catboost.sh +++ /dev/null @@ -1,17 +0,0 @@ -#!/usr/bin/env bash - -DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" - -cd $DIR -git clone https://github.com/catboost/catboost.git - - -cd "${DIR}/catboost/catboost/libs/model_interface" -../../../ya make -r -o "${DIR}/build/lib" -j4 -cd $DIR -ln -sf "${DIR}/build/lib/catboost/libs/model_interface/libcatboostmodel.so" libcatboostmodel.so - -cd "${DIR}/catboost/catboost/python-package/catboost" -../../../ya make -r -DUSE_ARCADIA_PYTHON=no -DOS_SDK=local -DPYTHON_CONFIG=python2-config -j4 -cd $DIR -ln -sf "${DIR}/catboost/catboost/python-package" python-package diff --git a/tests/external_models/catboost/helpers/__init__.py b/tests/external_models/catboost/helpers/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/external_models/catboost/helpers/client.py b/tests/external_models/catboost/helpers/client.py deleted file mode 100644 index ecf44a0b65c..00000000000 --- a/tests/external_models/catboost/helpers/client.py +++ /dev/null @@ -1,42 +0,0 @@ -import subprocess -import threading -import os - - -class ClickHouseClient: - def __init__(self, binary_path, port): - self.binary_path = binary_path - self.port = port - - def query(self, query, timeout=10, pipe=None): - - result = [] - process = [] - - def run(path, port, text, result, in_pipe, process): - - if in_pipe is None: - in_pipe = subprocess.PIPE - - pipe = subprocess.Popen([path, 'client', '--port', str(port), '-q', text], - stdin=in_pipe, stdout=subprocess.PIPE, stderr=subprocess.PIPE, close_fds=True) - process.append(pipe) - stdout_data, stderr_data = pipe.communicate() - - if stderr_data: - raise Exception('Error while executing query: {}\nstdout:\n{}\nstderr:\n{}' - .format(text, stdout_data, stderr_data)) - - result.append(stdout_data) - - thread = threading.Thread(target=run, args=(self.binary_path, self.port, query, result, pipe, process)) - thread.start() - thread.join(timeout) - if thread.isAlive(): - if len(process): - process[0].kill() - thread.join() - raise Exception('timeout exceed for query: ' + query) - - if len(result): - return result[0] diff --git a/tests/external_models/catboost/helpers/generate.py b/tests/external_models/catboost/helpers/generate.py deleted file mode 100644 index a7c1f3d9e98..00000000000 --- a/tests/external_models/catboost/helpers/generate.py +++ /dev/null @@ -1,15 +0,0 @@ -import numpy as np - - -def generate_uniform_int_column(size, low, high, seed=0): - np.random.seed(seed) - return np.random.randint(low, high, size) - - -def generate_uniform_float_column(size, low, high, seed=0): - np.random.seed(seed) - return np.random.random(size) * (high - low) + low - - -def generate_uniform_string_column(size, samples, seed): - return np.array(samples)[generate_uniform_int_column(size, 0, len(samples), seed)] diff --git a/tests/external_models/catboost/helpers/server.py b/tests/external_models/catboost/helpers/server.py deleted file mode 100644 index 8248b16e6df..00000000000 --- a/tests/external_models/catboost/helpers/server.py +++ /dev/null @@ -1,67 +0,0 @@ -import subprocess -import threading -import socket -import time - - -class ClickHouseServer: - def __init__(self, binary_path, config_path, stdout_file=None, stderr_file=None, shutdown_timeout=10): - self.binary_path = binary_path - self.config_path = config_path - self.pipe = None - self.stdout_file = stdout_file - self.stderr_file = stderr_file - self.shutdown_timeout = shutdown_timeout - - def start(self): - cmd = [self.binary_path, 'server', '--config', self.config_path] - out_pipe = None - err_pipe = None - if self.stdout_file is not None: - out_pipe = open(self.stdout_file, 'w') - if self.stderr_file is not None: - err_pipe = open(self.stderr_file, 'w') - self.pipe = subprocess.Popen(cmd, stdout=out_pipe, stderr=err_pipe) - - def wait_for_request(self, port, timeout=1): - try: - s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - # is not working - # s.settimeout(timeout) - - step = 0.01 - for iter in range(int(timeout / step)): - if s.connect_ex(('localhost', port)) == 0: - return - time.sleep(step) - - s.connect(('localhost', port)) - except socket.error as socketerror: - print("Error: ", socketerror) - raise - - def shutdown(self, timeout=10): - - def wait(pipe): - pipe.wait() - - if self.pipe is not None: - self.pipe.terminate() - thread = threading.Thread(target=wait, args=(self.pipe,)) - thread.start() - thread.join(timeout) - if thread.isAlive(): - self.pipe.kill() - thread.join() - - if self.pipe.stdout is not None: - self.pipe.stdout.close() - if self.pipe.stderr is not None: - self.pipe.stderr.close() - - def __enter__(self): - self.start() - return self - - def __exit__(self, type, value, traceback): - self.shutdown(self.shutdown_timeout) diff --git a/tests/external_models/catboost/helpers/server_with_models.py b/tests/external_models/catboost/helpers/server_with_models.py deleted file mode 100644 index e00da7b7027..00000000000 --- a/tests/external_models/catboost/helpers/server_with_models.py +++ /dev/null @@ -1,168 +0,0 @@ -from .server import ClickHouseServer -from .client import ClickHouseClient -from .table import ClickHouseTable -import os -import errno -from shutil import rmtree - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -CATBOOST_ROOT = os.path.dirname(SCRIPT_DIR) - -CLICKHOUSE_CONFIG = \ -''' - - Europe/Moscow - :: - {path} - {tmp_path} - {models_config} - 5368709120 - users.xml - {tcp_port} - {catboost_dynamic_library_path} - - trace - {path}/clickhouse-server.log - {path}/clickhouse-server.err.log - never - 50 - - -''' - -CLICKHOUSE_USERS = \ -''' - - - - - - 1 - - - - - - - readonly - default - - - - - default - default - - ::1 - 127.0.0.1 - - - - - - - - - - -''' - -CATBOOST_MODEL_CONFIG = \ -''' - - - catboost - {name} - {path} - 0 - - -''' - - -class ClickHouseServerWithCatboostModels: - def __init__(self, name, binary_path, port, shutdown_timeout=10, clean_folder=False): - self.models = {} - self.name = name - self.binary_path = binary_path - self.port = port - self.shutdown_timeout = shutdown_timeout - self.clean_folder = clean_folder - self.root = os.path.join(CATBOOST_ROOT, 'data', 'servers') - self.config_path = os.path.join(self.root, 'config.xml') - self.users_path = os.path.join(self.root, 'users.xml') - self.models_dir = os.path.join(self.root, 'models') - self.server = None - - def _get_server(self): - stdout_file = os.path.join(self.root, 'server_stdout.txt') - stderr_file = os.path.join(self.root, 'server_stderr.txt') - return ClickHouseServer(self.binary_path, self.config_path, stdout_file, stderr_file, self.shutdown_timeout) - - def add_model(self, model_name, model): - self.models[model_name] = model - - def apply_model(self, name, df, cat_feature_names): - names = list(df) - float_feature_names = tuple(name for name in names if name not in cat_feature_names) - with ClickHouseTable(self.server, self.port, name, df) as table: - return table.apply_model(name, cat_feature_names, float_feature_names) - - def _create_root(self): - try: - os.makedirs(self.root) - except OSError as exc: # Python >2.5 - if exc.errno == errno.EEXIST and os.path.isdir(self.root): - pass - else: - raise - - def _clean_root(self): - rmtree(self.root) - - def _save_config(self): - params = { - 'tcp_port': self.port, - 'path': os.path.join(self.root, 'clickhouse'), - 'tmp_path': os.path.join(self.root, 'clickhouse', 'tmp'), - 'models_config': os.path.join(self.models_dir, '*_model.xml'), - 'catboost_dynamic_library_path': os.path.join(CATBOOST_ROOT, 'data', 'libcatboostmodel.so') - } - config = CLICKHOUSE_CONFIG.format(**params) - - with open(self.config_path, 'w') as f: - f.write(config) - - with open(self.users_path, 'w') as f: - f.write(CLICKHOUSE_USERS) - - def _save_models(self): - if not os.path.exists(self.models_dir): - os.makedirs(self.models_dir) - - for name, model in list(self.models.items()): - model_path = os.path.join(self.models_dir, name + '.cbm') - config_path = os.path.join(self.models_dir, name + '_model.xml') - params = { - 'name': name, - 'path': model_path - } - config = CATBOOST_MODEL_CONFIG.format(**params) - with open(config_path, 'w') as f: - f.write(config) - - model.save_model(model_path) - - def __enter__(self): - self._create_root() - self._save_config() - self._save_models() - self.server = self._get_server().__enter__() - return self - - def __exit__(self, exc_type, exc_val, exc_tb): - res = self.server.__exit__(exc_type, exc_val, exc_tb) - if self.clean_folder: - self._clean_root() - return res - diff --git a/tests/external_models/catboost/helpers/table.py b/tests/external_models/catboost/helpers/table.py deleted file mode 100644 index 5f9b828c9f3..00000000000 --- a/tests/external_models/catboost/helpers/table.py +++ /dev/null @@ -1,74 +0,0 @@ -from .server import ClickHouseServer -from .client import ClickHouseClient -from pandas import DataFrame -import os -import threading -import tempfile - - -class ClickHouseTable: - def __init__(self, server, port, table_name, df): - self.server = server - self.port = port - self.table_name = table_name - self.df = df - - if not isinstance(self.server, ClickHouseServer): - raise Exception('Expected ClickHouseServer, got ' + repr(self.server)) - if not isinstance(self.df, DataFrame): - raise Exception('Expected DataFrame, got ' + repr(self.df)) - - self.server.wait_for_request(port) - self.client = ClickHouseClient(server.binary_path, port) - - def _convert(self, name): - types_map = { - 'float64': 'Float64', - 'int64': 'Int64', - 'float32': 'Float32', - 'int32': 'Int32' - } - - if name in types_map: - return types_map[name] - return 'String' - - def _create_table_from_df(self): - self.client.query('create database if not exists test') - self.client.query('drop table if exists test.{}'.format(self.table_name)) - - column_types = list(self.df.dtypes) - column_names = list(self.df) - schema = ', '.join((name + ' ' + self._convert(str(t)) for name, t in zip(column_names, column_types))) - print('schema:', schema) - - create_query = 'create table test.{} (date Date DEFAULT today(), {}) engine = MergeTree(date, (date), 8192)' - self.client.query(create_query.format(self.table_name, schema)) - - insert_query = 'insert into test.{} ({}) format CSV' - - with tempfile.TemporaryFile() as tmp_file: - self.df.to_csv(tmp_file, header=False, index=False) - tmp_file.seek(0) - self.client.query(insert_query.format(self.table_name, ', '.join(column_names)), pipe=tmp_file) - - def apply_model(self, model_name, float_columns, cat_columns): - columns = ', '.join(list(float_columns) + list(cat_columns)) - query = "select modelEvaluate('{}', {}) from test.{} format TSV" - result = self.client.query(query.format(model_name, columns, self.table_name)) - - def parse_row(row): - values = tuple(map(float, list(filter(len, list(map(str.strip, row.replace('(', '').replace(')', '').split(','))))))) - return values if len(values) != 1 else values[0] - - return tuple(map(parse_row, list(filter(len, list(map(str.strip, result.split('\n'))))))) - - def _drop_table(self): - self.client.query('drop table test.{}'.format(self.table_name)) - - def __enter__(self): - self._create_table_from_df() - return self - - def __exit__(self, type, value, traceback): - self._drop_table() diff --git a/tests/external_models/catboost/helpers/train.py b/tests/external_models/catboost/helpers/train.py deleted file mode 100644 index 34a6f8e958b..00000000000 --- a/tests/external_models/catboost/helpers/train.py +++ /dev/null @@ -1,28 +0,0 @@ -import os -import sys -from pandas import DataFrame - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -CATBOOST_ROOT = os.path.dirname(SCRIPT_DIR) -CATBOOST_PYTHON_DIR = os.path.join(CATBOOST_ROOT, 'data', 'python-package') - -if CATBOOST_PYTHON_DIR not in sys.path: - sys.path.append(CATBOOST_PYTHON_DIR) - - -import catboost -from catboost import CatBoostClassifier - - -def train_catboost_model(df, target, cat_features, params, verbose=True): - - if not isinstance(df, DataFrame): - raise Exception('DataFrame object expected, but got ' + repr(df)) - - print('features:', df.columns.tolist()) - - cat_features_index = list(df.columns.get_loc(feature) for feature in cat_features) - print('cat features:', cat_features_index) - model = CatBoostClassifier(**params) - model.fit(df, target, cat_features=cat_features_index, verbose=verbose) - return model diff --git a/tests/external_models/catboost/pytest.ini b/tests/external_models/catboost/pytest.ini deleted file mode 100644 index a40472347fb..00000000000 --- a/tests/external_models/catboost/pytest.ini +++ /dev/null @@ -1,3 +0,0 @@ -[pytest] -python_files = test.py -norecursedirs=data diff --git a/tests/external_models/catboost/test_apply_catboost_model/test.py b/tests/external_models/catboost/test_apply_catboost_model/test.py deleted file mode 100644 index d266393bf48..00000000000 --- a/tests/external_models/catboost/test_apply_catboost_model/test.py +++ /dev/null @@ -1,294 +0,0 @@ -from helpers.server_with_models import ClickHouseServerWithCatboostModels -from helpers.generate import generate_uniform_string_column, generate_uniform_float_column, generate_uniform_int_column -from helpers.train import train_catboost_model -import os -import numpy as np -from pandas import DataFrame - - -PORT = int(os.environ.get('CLICKHOUSE_TESTS_PORT', '9000')) -CLICKHOUSE_TESTS_SERVER_BIN_PATH = os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse') - - -def add_noise_to_target(target, seed, threshold=0.05): - col = generate_uniform_float_column(len(target), 0., 1., seed + 1) < threshold - return target * (1 - col) + (1 - target) * col - - -def check_predictions(test_name, target, pred_python, pred_ch, acc_threshold): - ch_class = pred_ch.astype(int) - python_class = pred_python.astype(int) - if not np.array_equal(ch_class, python_class): - raise Exception('Got different results:\npython:\n' + str(python_class) + '\nClickHouse:\n' + str(ch_class)) - - acc = 1 - np.sum(np.abs(ch_class - np.array(target))) / (len(target) + .0) - assert acc >= acc_threshold - print(test_name, 'accuracy: {:.10f}'.format(acc)) - - -def test_apply_float_features_only(): - - name = 'test_apply_float_features_only' - - train_size = 10000 - test_size = 10000 - - def gen_data(size, seed): - data = { - 'a': generate_uniform_float_column(size, 0., 1., seed + 1), - 'b': generate_uniform_float_column(size, 0., 1., seed + 2), - 'c': generate_uniform_float_column(size, 0., 1., seed + 3) - } - return DataFrame.from_dict(data) - - def get_target(df): - def target_filter(row): - return 1 if (row['a'] > .3 and row['b'] > .3) or (row['c'] < .4 and row['a'] * row['b'] > 0.1) else 0 - return df.apply(target_filter, axis=1).as_matrix() - - train_df = gen_data(train_size, 42) - test_df = gen_data(test_size, 43) - - train_target = get_target(train_df) - test_target = get_target(test_df) - - print() - print('train target', train_target) - print('test target', test_target) - - params = { - 'iterations': 4, - 'depth': 2, - 'learning_rate': 1, - 'loss_function': 'Logloss' - } - - model = train_catboost_model(train_df, train_target, [], params) - pred_python = model.predict(test_df) - - server = ClickHouseServerWithCatboostModels(name, CLICKHOUSE_TESTS_SERVER_BIN_PATH, PORT) - server.add_model(name, model) - with server: - pred_ch = (np.array(server.apply_model(name, test_df, [])) > 0).astype(int) - - print('python predictions', pred_python) - print('clickhouse predictions', pred_ch) - - check_predictions(name, test_target, pred_python, pred_ch, 0.9) - - -def test_apply_float_features_with_string_cat_features(): - - name = 'test_apply_float_features_with_string_cat_features' - - train_size = 10000 - test_size = 10000 - - def gen_data(size, seed): - data = { - 'a': generate_uniform_float_column(size, 0., 1., seed + 1), - 'b': generate_uniform_float_column(size, 0., 1., seed + 2), - 'c': generate_uniform_string_column(size, ['a', 'b', 'c'], seed + 3), - 'd': generate_uniform_string_column(size, ['e', 'f', 'g'], seed + 4) - } - return DataFrame.from_dict(data) - - def get_target(df): - def target_filter(row): - return 1 if (row['a'] > .3 and row['b'] > .3 and row['c'] != 'a') \ - or (row['a'] * row['b'] > 0.1 and row['c'] != 'b' and row['d'] != 'e') else 0 - return df.apply(target_filter, axis=1).as_matrix() - - train_df = gen_data(train_size, 42) - test_df = gen_data(test_size, 43) - - train_target = get_target(train_df) - test_target = get_target(test_df) - - print() - print('train target', train_target) - print('test target', test_target) - - params = { - 'iterations': 6, - 'depth': 2, - 'learning_rate': 1, - 'loss_function': 'Logloss' - } - - model = train_catboost_model(train_df, train_target, ['c', 'd'], params) - pred_python = model.predict(test_df) - - server = ClickHouseServerWithCatboostModels(name, CLICKHOUSE_TESTS_SERVER_BIN_PATH, PORT) - server.add_model(name, model) - with server: - pred_ch = (np.array(server.apply_model(name, test_df, [])) > 0).astype(int) - - print('python predictions', pred_python) - print('clickhouse predictions', pred_ch) - - check_predictions(name, test_target, pred_python, pred_ch, 0.9) - - -def test_apply_float_features_with_int_cat_features(): - - name = 'test_apply_float_features_with_int_cat_features' - - train_size = 10000 - test_size = 10000 - - def gen_data(size, seed): - data = { - 'a': generate_uniform_float_column(size, 0., 1., seed + 1), - 'b': generate_uniform_float_column(size, 0., 1., seed + 2), - 'c': generate_uniform_int_column(size, 1, 4, seed + 3), - 'd': generate_uniform_int_column(size, 1, 4, seed + 4) - } - return DataFrame.from_dict(data) - - def get_target(df): - def target_filter(row): - return 1 if (row['a'] > .3 and row['b'] > .3 and row['c'] != 1) \ - or (row['a'] * row['b'] > 0.1 and row['c'] != 2 and row['d'] != 3) else 0 - return df.apply(target_filter, axis=1).as_matrix() - - train_df = gen_data(train_size, 42) - test_df = gen_data(test_size, 43) - - train_target = get_target(train_df) - test_target = get_target(test_df) - - print() - print('train target', train_target) - print('test target', test_target) - - params = { - 'iterations': 6, - 'depth': 4, - 'learning_rate': 1, - 'loss_function': 'Logloss' - } - - model = train_catboost_model(train_df, train_target, ['c', 'd'], params) - pred_python = model.predict(test_df) - - server = ClickHouseServerWithCatboostModels(name, CLICKHOUSE_TESTS_SERVER_BIN_PATH, PORT) - server.add_model(name, model) - with server: - pred_ch = (np.array(server.apply_model(name, test_df, [])) > 0).astype(int) - - print('python predictions', pred_python) - print('clickhouse predictions', pred_ch) - - check_predictions(name, test_target, pred_python, pred_ch, 0.9) - - -def test_apply_float_features_with_mixed_cat_features(): - - name = 'test_apply_float_features_with_mixed_cat_features' - - train_size = 10000 - test_size = 10000 - - def gen_data(size, seed): - data = { - 'a': generate_uniform_float_column(size, 0., 1., seed + 1), - 'b': generate_uniform_float_column(size, 0., 1., seed + 2), - 'c': generate_uniform_string_column(size, ['a', 'b', 'c'], seed + 3), - 'd': generate_uniform_int_column(size, 1, 4, seed + 4) - } - return DataFrame.from_dict(data) - - def get_target(df): - def target_filter(row): - return 1 if (row['a'] > .3 and row['b'] > .3 and row['c'] != 'a') \ - or (row['a'] * row['b'] > 0.1 and row['c'] != 'b' and row['d'] != 2) else 0 - return df.apply(target_filter, axis=1).as_matrix() - - train_df = gen_data(train_size, 42) - test_df = gen_data(test_size, 43) - - train_target = get_target(train_df) - test_target = get_target(test_df) - - print() - print('train target', train_target) - print('test target', test_target) - - params = { - 'iterations': 6, - 'depth': 4, - 'learning_rate': 1, - 'loss_function': 'Logloss' - } - - model = train_catboost_model(train_df, train_target, ['c', 'd'], params) - pred_python = model.predict(test_df) - - server = ClickHouseServerWithCatboostModels(name, CLICKHOUSE_TESTS_SERVER_BIN_PATH, PORT) - server.add_model(name, model) - with server: - pred_ch = (np.array(server.apply_model(name, test_df, [])) > 0).astype(int) - - print('python predictions', pred_python) - print('clickhouse predictions', pred_ch) - - check_predictions(name, test_target, pred_python, pred_ch, 0.9) - - -def test_apply_multiclass(): - - name = 'test_apply_float_features_with_mixed_cat_features' - - train_size = 10000 - test_size = 10000 - - def gen_data(size, seed): - data = { - 'a': generate_uniform_float_column(size, 0., 1., seed + 1), - 'b': generate_uniform_float_column(size, 0., 1., seed + 2), - 'c': generate_uniform_string_column(size, ['a', 'b', 'c'], seed + 3), - 'd': generate_uniform_int_column(size, 1, 4, seed + 4) - } - return DataFrame.from_dict(data) - - def get_target(df): - def target_filter(row): - if row['a'] > .3 and row['b'] > .3 and row['c'] != 'a': - return 0 - elif row['a'] * row['b'] > 0.1 and row['c'] != 'b' and row['d'] != 2: - return 1 - else: - return 2 - - return df.apply(target_filter, axis=1).as_matrix() - - train_df = gen_data(train_size, 42) - test_df = gen_data(test_size, 43) - - train_target = get_target(train_df) - test_target = get_target(test_df) - - print() - print('train target', train_target) - print('test target', test_target) - - params = { - 'iterations': 10, - 'depth': 4, - 'learning_rate': 1, - 'loss_function': 'MultiClass' - } - - model = train_catboost_model(train_df, train_target, ['c', 'd'], params) - pred_python = model.predict(test_df)[:,0].astype(int) - - server = ClickHouseServerWithCatboostModels(name, CLICKHOUSE_TESTS_SERVER_BIN_PATH, PORT) - server.add_model(name, model) - with server: - pred_ch = np.argmax(np.array(server.apply_model(name, test_df, [])), axis=1) - - print('python predictions', pred_python) - print('clickhouse predictions', pred_ch) - - check_predictions(name, test_target, pred_python, pred_ch, 0.9) From 34017c942e5135b12ba758d23c913da1f92f89a8 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 25 Sep 2021 13:38:56 +0800 Subject: [PATCH 160/317] 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 161/317] 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 162/317] 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 30b92d24f1d46b071b519837e9fca697f6d331ef Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 Sep 2021 14:46:03 +0000 Subject: [PATCH 163/317] Fix --- src/Dictionaries/MongoDBDictionarySource.cpp | 16 ++++++++- src/Dictionaries/MySQLDictionarySource.cpp | 26 ++++++++------ .../ExternalDataSourceConfiguration.cpp | 34 ++++++++++++------- .../ExternalDataSourceConfiguration.h | 4 +-- 4 files changed, 53 insertions(+), 27 deletions(-) diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index ebf479d7038..b811a50121b 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -20,7 +20,21 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) bool /* created_from_ddl */) { const auto config_prefix = root_config_prefix + ".mongodb"; - auto configuration = getExternalDataSourceConfiguration(config, config_prefix, context); + ExternalDataSourceConfiguration configuration; + auto named_collection = getExternalDataSourceConfiguration(config, config_prefix, context); + if (named_collection) + { + configuration = *named_collection; + } + else + { + configuration.host = config.getString(config_prefix + ".host", ""); + configuration.port = config.getUInt(config_prefix + ".port", 0); + configuration.username = config.getString(config_prefix + ".user", ""); + configuration.password = config.getString(config_prefix + ".password", ""); + configuration.database = config.getString(config_prefix + ".db", ""); + } + return std::make_unique(dict_struct, config.getString(config_prefix + ".uri", ""), configuration.host, diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 4f805687c26..c740cc7548c 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -44,7 +44,22 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) config.getBool(config_prefix + ".mysql.fail_on_connection_loss", false) ? 1 : default_num_tries_on_connection_loss); auto settings_config_prefix = config_prefix + ".mysql"; - auto configuration = getExternalDataSourceConfiguration(config, settings_config_prefix, global_context); + std::shared_ptr pool; + ExternalDataSourceConfiguration configuration; + auto named_collection = created_from_ddl ? getExternalDataSourceConfiguration(config, settings_config_prefix, global_context) : std::nullopt; + if (named_collection) + { + configuration = *named_collection; + std::vector> addresses{std::make_pair(configuration.host, configuration.port)}; + pool = std::make_shared(configuration.database, addresses, configuration.username, configuration.password); + } + else + { + configuration.database = config.getString(settings_config_prefix + ".db", ""); + configuration.table = config.getString(settings_config_prefix + ".table", ""); + pool = std::make_shared(mysqlxx::PoolFactory::instance().get(config, settings_config_prefix)); + } + auto query = config.getString(settings_config_prefix + ".query", ""); if (query.empty() && configuration.table.empty()) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "MySQL dictionary source configuration must contain table or query field"); @@ -61,15 +76,6 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) .dont_check_update_time = config.getBool(settings_config_prefix + ".dont_check_update_time", false) }; - std::shared_ptr pool; - if (created_from_ddl) - { - std::vector> addresses{std::make_pair(configuration.host, configuration.port)}; - pool = std::make_shared(configuration.database, addresses, configuration.username, configuration.password); - } - else - pool = std::make_shared(mysqlxx::PoolFactory::instance().get(config, settings_config_prefix)); - return std::make_unique(dict_struct, dictionary_configuration, std::move(pool), sample_block, mysql_input_stream_settings); #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 69af9424eaf..00c41069101 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -124,7 +124,7 @@ std::optional getExternalDataSourceConfiguration(const } -ExternalDataSourceConfiguration getExternalDataSourceConfiguration( +std::optional getExternalDataSourceConfiguration( const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context) { ExternalDataSourceConfiguration configuration; @@ -152,25 +152,33 @@ ExternalDataSourceConfiguration getExternalDataSourceConfiguration( throw Exception(ErrorCodes::BAD_ARGUMENTS, "Named collection of connection parameters is missing some of the parameters and dictionary parameters are added"); } + return configuration; } - else - { - configuration.host = dict_config.getString(dict_config_prefix + ".host", ""); - configuration.port = dict_config.getUInt(dict_config_prefix + ".port", 0); - configuration.username = dict_config.getString(dict_config_prefix + ".user", ""); - configuration.password = dict_config.getString(dict_config_prefix + ".password", ""); - configuration.database = dict_config.getString(dict_config_prefix + ".db", ""); - configuration.table = dict_config.getString(fmt::format("{}.table", dict_config_prefix), ""); - configuration.schema = dict_config.getString(fmt::format("{}.schema", dict_config_prefix), ""); - } - return configuration; + return std::nullopt; } ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context) { - auto common_configuration = getExternalDataSourceConfiguration(dict_config, dict_config_prefix, context); + ExternalDataSourceConfiguration common_configuration; + + auto named_collection = getExternalDataSourceConfiguration(dict_config, dict_config_prefix, context); + if (named_collection) + { + common_configuration = *named_collection; + } + else + { + common_configuration.host = dict_config.getString(dict_config_prefix + ".host", ""); + common_configuration.port = dict_config.getUInt(dict_config_prefix + ".port", 0); + common_configuration.username = dict_config.getString(dict_config_prefix + ".user", ""); + common_configuration.password = dict_config.getString(dict_config_prefix + ".password", ""); + common_configuration.database = dict_config.getString(dict_config_prefix + ".db", ""); + common_configuration.table = dict_config.getString(fmt::format("{}.table", dict_config_prefix), ""); + common_configuration.schema = dict_config.getString(fmt::format("{}.schema", dict_config_prefix), ""); + } + ExternalDataSourcesByPriority configuration { .database = common_configuration.database, diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index df3b4f6da1f..5f88653807b 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -25,8 +25,6 @@ struct ExternalDataSourceConfiguration void set(const ExternalDataSourceConfiguration & conf); }; -using ExternalDataSourceConfigurationPtr = std::shared_ptr; - struct StoragePostgreSQLConfiguration : ExternalDataSourceConfiguration { @@ -67,7 +65,7 @@ struct ExternalDataSourceConfig */ std::optional getExternalDataSourceConfiguration(const ASTs & args, ContextPtr context, bool is_database_engine = false); -ExternalDataSourceConfiguration getExternalDataSourceConfiguration( +std::optional getExternalDataSourceConfiguration( const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context); From 2121682882b43c89af3c612d91187b979e47f9c0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 Sep 2021 14:50:58 +0000 Subject: [PATCH 164/317] 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 fb584715e175fef3a4c4373d1bbcebecab1db6d8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 25 Sep 2021 16:32:25 +0300 Subject: [PATCH 165/317] Pass through initial query_id for clickhouse-benchmark --- src/DataStreams/RemoteQueryExecutor.cpp | 6 +++++ ..._benchmark_query_id_pass_through.reference | 1 + ...ckhouse_benchmark_query_id_pass_through.sh | 25 +++++++++++++++++++ 3 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.reference create mode 100755 tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.sh diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index a64c4409cc9..3c78fddfd39 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -208,6 +208,12 @@ void RemoteQueryExecutor::sendQuery() auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(settings); ClientInfo modified_client_info = context->getClientInfo(); modified_client_info.query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; + /// Set initial_query_id to query_id for the clickhouse-benchmark. + /// + /// (since first query of clickhouse-benchmark will be issued as SECONDARY_QUERY, + /// due to it executes queries via RemoteBlockInputStream) + if (modified_client_info.initial_query_id.empty()) + modified_client_info.initial_query_id = query_id; if (CurrentThread::isInitialized()) { modified_client_info.client_trace_context = CurrentThread::get().thread_trace_context; diff --git a/tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.reference b/tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.reference new file mode 100644 index 00000000000..00750edc07d --- /dev/null +++ b/tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.reference @@ -0,0 +1 @@ +3 diff --git a/tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.sh b/tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.sh new file mode 100755 index 00000000000..9cab6db6940 --- /dev/null +++ b/tests/queries/0_stateless/02040_clickhouse_benchmark_query_id_pass_through.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +query_id="${CLICKHOUSE_DATABASE}_$$" +benchmark_args=( + --iterations 1 + --log_queries 1 + --query_id "$query_id" + --log_queries_min_type QUERY_FINISH +) +$CLICKHOUSE_BENCHMARK "${benchmark_args[@]}" --query "SELECT * FROM remote('127.2', 'system.one')" >& /dev/null +$CLICKHOUSE_CLIENT -q "SYSTEM FLUSH LOGS" +# Number of queries: +# - DESC TABLE system.one +# - query on initiator +# - query on shard +# Total: 3 +# +# -- NOTE: this test cannot use 'current_database = $CLICKHOUSE_DATABASE', +# -- because it does not propagated via remote queries, +# -- but it uses query_id, and this is enough. +$CLICKHOUSE_CLIENT --param_query_id="$query_id" -q "SELECT count() FROM system.query_log WHERE event_date >= yesterday() AND initial_query_id = {query_id:String}" From 9dac348893dc32fb9be02122f5474a570788662a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 25 Sep 2021 19:41:50 +0300 Subject: [PATCH 166/317] 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 e40a13bd1f959131eec2f9d8c138036410773059 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 25 Sep 2021 19:47:29 +0300 Subject: [PATCH 167/317] Remove trash --- tests/integration/test_rocksdb_options/test.py | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/tests/integration/test_rocksdb_options/test.py b/tests/integration/test_rocksdb_options/test.py index e9e9cfdf20f..6689c232081 100644 --- a/tests/integration/test_rocksdb_options/test.py +++ b/tests/integration/test_rocksdb_options/test.py @@ -63,13 +63,3 @@ 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 2deb9643af5644d68d62b0808315e0fa853a25c2 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 25 Sep 2021 10:12:32 -0700 Subject: [PATCH 168/317] 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 169/317] 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 170/317] 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 77ab62c821a3a754c4a1d7815ba343dadf20465b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 25 Sep 2021 20:16:57 +0300 Subject: [PATCH 171/317] Update CHANGELOG.md --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0e92fc59509..6e209293e67 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -34,7 +34,6 @@ * New functions `currentProfiles()`, `enabledProfiles()`, `defaultProfiles()`. [#26714](https://github.com/ClickHouse/ClickHouse/pull/26714) ([Vitaly Baranov](https://github.com/vitlibar)). * Add functions that return (initial_)query_id of the current query. This closes [#23682](https://github.com/ClickHouse/ClickHouse/issues/23682). [#26410](https://github.com/ClickHouse/ClickHouse/pull/26410) ([Alexey Boykov](https://github.com/mathalex)). * Add `REPLACE GRANT` feature. [#26384](https://github.com/ClickHouse/ClickHouse/pull/26384) ([Caspian](https://github.com/Cas-pian)). -* Implement window function `nth_value(expr, N)` that returns the value of the Nth row of the window frame. [#26334](https://github.com/ClickHouse/ClickHouse/pull/26334) ([Zuo, RuoYu](https://github.com/ryzuo)). * `EXPLAIN` query now has `EXPLAIN ESTIMATE ...` mode that will show information about read rows, marks and parts from MergeTree tables. Closes [#23941](https://github.com/ClickHouse/ClickHouse/issues/23941). [#26131](https://github.com/ClickHouse/ClickHouse/pull/26131) ([fastio](https://github.com/fastio)). * Added `system.zookeeper_log` table. All actions of ZooKeeper client are logged into this table. Implements [#25449](https://github.com/ClickHouse/ClickHouse/issues/25449). [#26129](https://github.com/ClickHouse/ClickHouse/pull/26129) ([tavplubix](https://github.com/tavplubix)). * Zero-copy replication for `ReplicatedMergeTree` over `HDFS` storage. [#25918](https://github.com/ClickHouse/ClickHouse/pull/25918) ([Zhichang Yu](https://github.com/yuzhichang)). From 7ec4f3339f2b7ba874bd5b8cdc9d90b5f49746c5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 25 Sep 2021 21:04:40 +0300 Subject: [PATCH 172/317] tests: purge unused configs --- tests/clickhouse-client.xml | 3 - tests/client-test.xml | 17 --- tests/decimals_dictionary.xml | 1 - tests/executable_pool_dictionary.xml | 1 - tests/ints_dictionary.xml | 1 - tests/server-test.xml | 150 --------------------------- tests/strings_dictionary.xml | 1 - tests/users.d/access_management.xml | 7 -- tests/users.d/readonly.xml | 1 - tests/users.xml | 1 - 10 files changed, 183 deletions(-) delete mode 100644 tests/clickhouse-client.xml delete mode 100644 tests/client-test.xml delete mode 120000 tests/decimals_dictionary.xml delete mode 120000 tests/executable_pool_dictionary.xml delete mode 120000 tests/ints_dictionary.xml delete mode 100644 tests/server-test.xml delete mode 120000 tests/strings_dictionary.xml delete mode 100644 tests/users.d/access_management.xml delete mode 120000 tests/users.d/readonly.xml delete mode 120000 tests/users.xml diff --git a/tests/clickhouse-client.xml b/tests/clickhouse-client.xml deleted file mode 100644 index b00c16f2c99..00000000000 --- a/tests/clickhouse-client.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 100000 - \ No newline at end of file diff --git a/tests/client-test.xml b/tests/client-test.xml deleted file mode 100644 index ee84d41911f..00000000000 --- a/tests/client-test.xml +++ /dev/null @@ -1,17 +0,0 @@ - - - 59000 - 59440 - - - true - true - sslv2,sslv3 - true - none - - AcceptCertificateHandler - - - - \ No newline at end of file diff --git a/tests/decimals_dictionary.xml b/tests/decimals_dictionary.xml deleted file mode 120000 index 15f5b3800b3..00000000000 --- a/tests/decimals_dictionary.xml +++ /dev/null @@ -1 +0,0 @@ -config/decimals_dictionary.xml \ No newline at end of file diff --git a/tests/executable_pool_dictionary.xml b/tests/executable_pool_dictionary.xml deleted file mode 120000 index 83f18b95f20..00000000000 --- a/tests/executable_pool_dictionary.xml +++ /dev/null @@ -1 +0,0 @@ -config/executable_pool_dictionary.xml \ No newline at end of file diff --git a/tests/ints_dictionary.xml b/tests/ints_dictionary.xml deleted file mode 120000 index 1daa76b8267..00000000000 --- a/tests/ints_dictionary.xml +++ /dev/null @@ -1 +0,0 @@ -config/ints_dictionary.xml \ No newline at end of file diff --git a/tests/server-test.xml b/tests/server-test.xml deleted file mode 100644 index 3f4030e07d1..00000000000 --- a/tests/server-test.xml +++ /dev/null @@ -1,150 +0,0 @@ - - - - - trace - /tmp/clickhouse/log/clickhouse-server.log - /tmp/clickhouse/log/clickhouse-server.err.log - 10M - 1 - 0 - - :: - 0.0.0.0 - 1 - 58123 - 59000 - 58443 - 59440 - 59009 - 10000 - - - - /tmp/clickhouse/etc/server.crt - /tmp/clickhouse/etc/server.key - - /tmp/clickhouse/etc/dhparam.pem - none - true - true - sslv2,sslv3 - true - - - - true - true - sslv2,sslv3 - true - none - - AcceptCertificateHandler - - - - - 3 - /tmp/clickhouse/data/ - /tmp/clickhouse/tmp/ - users.xml - /tmp/clickhouse/data/access/ - custom_ - 5368709120 - default - default - Europe/Moscow - - - - - - localhost - 59000 - - - - - localhost - 1 - - - - - - - localhost - 59000 - - - - - - - 127.0.0.1 - 59000 - - - - - 127.0.0.2 - 59000 - - - - - - - localhost - 59000 - - - - - localhost - 59000 - - - - - - - localhost - 59440 - 1 - - - - - - - - 3600 - 3600 - 60 - - system - query_log
- 7500 -
- *_dictionary.xml - *_function.xml - - - - /clickhouse/task_queue/ddl - - /tmp/clickhouse/data/format_schemas/ - - - TOPSECRET.TOPSECRET - [hidden] - - -
diff --git a/tests/strings_dictionary.xml b/tests/strings_dictionary.xml deleted file mode 120000 index be66c1da224..00000000000 --- a/tests/strings_dictionary.xml +++ /dev/null @@ -1 +0,0 @@ -config/strings_dictionary.xml \ No newline at end of file diff --git a/tests/users.d/access_management.xml b/tests/users.d/access_management.xml deleted file mode 100644 index 7e799cb7b10..00000000000 --- a/tests/users.d/access_management.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - - 1 - - - diff --git a/tests/users.d/readonly.xml b/tests/users.d/readonly.xml deleted file mode 120000 index 023bb5bf9f5..00000000000 --- a/tests/users.d/readonly.xml +++ /dev/null @@ -1 +0,0 @@ -../../programs/server/users.d/readonly.xml \ No newline at end of file diff --git a/tests/users.xml b/tests/users.xml deleted file mode 120000 index 32a83e5806f..00000000000 --- a/tests/users.xml +++ /dev/null @@ -1 +0,0 @@ -../programs/server/users.xml \ No newline at end of file From 5e1d180d6452f0b79b567a1de14d74179e2c4246 Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 25 Sep 2021 19:32:42 +0000 Subject: [PATCH 173/317] reload and grant stub --- docs/en/sql-reference/statements/grant.md | 2 ++ docs/en/sql-reference/statements/system.md | 13 +++++++++++++ 2 files changed, 15 insertions(+) diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index 2b3cd68fbb2..f04952746a6 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -155,6 +155,8 @@ Hierarchy of privileges: - `SYSTEM RELOAD CONFIG` - `SYSTEM RELOAD DICTIONARY` - `SYSTEM RELOAD EMBEDDED DICTIONARIES` + - `SYSTEM RELOAD FUNCTION` + - `SYSTEM RELOAD FUNCTIONS` - `SYSTEM MERGES` - `SYSTEM TTL MERGES` - `SYSTEM FETCHES` diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index cf2a99a4c5f..5b2d0175786 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -12,6 +12,8 @@ The list of available `SYSTEM` statements: - [RELOAD DICTIONARY](#query_language-system-reload-dictionary) - [RELOAD MODELS](#query_language-system-reload-models) - [RELOAD MODEL](#query_language-system-reload-model) +- [RELOAD FUNCTIONS](#query_language-system-reload-functions) +- [RELOAD FUNCTION](#query_language-system-reload-function) - [DROP DNS CACHE](#query_language-system-drop-dns-cache) - [DROP MARK CACHE](#query_language-system-drop-mark-cache) - [DROP UNCOMPRESSED CACHE](#query_language-system-drop-uncompressed-cache) @@ -83,6 +85,17 @@ Completely reloads a CatBoost model `model_name` if the configuration was update SYSTEM RELOAD MODEL ``` +## RELOAD FUNCTIONS {#query_language-system-reload-functions} + +**Syntax** + +```sql +[SYSTEM] RELOAD FUNCTIONS +``` + +## RELOAD FUNCTION {#query_language-system-reload-function} + + ## DROP DNS CACHE {#query_language-system-drop-dns-cache} Resets ClickHouse’s internal DNS cache. Sometimes (for old ClickHouse versions) it is necessary to use this command when changing the infrastructure (changing the IP address of another ClickHouse server or the server used by dictionaries). From cd178496079287e540c2c36fc2ccf11eb5a90183 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 25 Sep 2021 13:11:23 -0700 Subject: [PATCH 174/317] 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 dc7eb5c1aebefd130bb3200f700e1494b676f143 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 26 Sep 2021 03:29:07 +0300 Subject: [PATCH 175/317] Fix ugly typo --- docker/test/coverage/s3downloader | 2 +- docker/test/stateful/s3downloader | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/coverage/s3downloader b/docker/test/coverage/s3downloader index 363ece8dac6..eb3b3cd9faf 100755 --- a/docker/test/coverage/s3downloader +++ b/docker/test/coverage/s3downloader @@ -92,7 +92,7 @@ if __name__ == "__main__": logging.info("Some exception occured %s", str(ex)) raise finally: - logging.info("Will remove dowloaded file %s from filesystem if it exists", temp_archive_path) + logging.info("Will remove downloaded file %s from filesystem if it exists", temp_archive_path) if os.path.exists(temp_archive_path): os.remove(temp_archive_path) logging.info("Processing of %s finished", dataset) diff --git a/docker/test/stateful/s3downloader b/docker/test/stateful/s3downloader index 363ece8dac6..eb3b3cd9faf 100755 --- a/docker/test/stateful/s3downloader +++ b/docker/test/stateful/s3downloader @@ -92,7 +92,7 @@ if __name__ == "__main__": logging.info("Some exception occured %s", str(ex)) raise finally: - logging.info("Will remove dowloaded file %s from filesystem if it exists", temp_archive_path) + logging.info("Will remove downloaded file %s from filesystem if it exists", temp_archive_path) if os.path.exists(temp_archive_path): os.remove(temp_archive_path) logging.info("Processing of %s finished", dataset) From 23b6ac82a7fd85d6c37c9d5d472899665133515d Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 25 Sep 2021 18:06:49 -0700 Subject: [PATCH 176/317] 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 177/317] 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 3ef5d13a4aaa5a9ec84775d10a3f6bd0e6edfd3d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=B8=87=E5=BA=B7?= Date: Sat, 25 Sep 2021 23:20:16 +0800 Subject: [PATCH 178/317] add parameter --- utils/keeper-data-dumper/main.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 11db6fc61bc..842e6b09775 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -68,7 +68,7 @@ int main(int argc, char *argv[]) LOG_INFO(logger, "Last committed index: {}", last_commited_index); - DB::KeeperLogStore changelog(argv[2], 10000000, true); + DB::KeeperLogStore changelog(argv[2], 10000000, true, settings->compress_logs); changelog.init(last_commited_index, 10000000000UL); /// collect all logs if (changelog.size() == 0) LOG_INFO(logger, "Changelog empty"); From 121dfe26b6a8ec181bba91182111a61a8cf0bbce Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Sun, 26 Sep 2021 08:13:58 +0300 Subject: [PATCH 179/317] 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 180/317] 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 181/317] 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 182/317] 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 ed7617554f2807c24285cf5952d1c317d4eed494 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 25 Sep 2021 22:44:44 -0700 Subject: [PATCH 183/317] Add review fixes --- docs/en/sql-reference/functions/geo/s2.md | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/s2.md b/docs/en/sql-reference/functions/geo/s2.md index 532266ea983..592ccedec0e 100644 --- a/docs/en/sql-reference/functions/geo/s2.md +++ b/docs/en/sql-reference/functions/geo/s2.md @@ -80,8 +80,6 @@ Result: └──────────────────────────────────────┘ ``` - - ## s2GetNeighbors {#s2GetNeighbors} Returns S2 neighbor indices corresponding to the provided [S2](#s2index)). Each cell in the S2 system is a quadrilateral bounded by four geodesics. So, each cell has 4 neighbors. @@ -89,7 +87,7 @@ Returns S2 neighbor indices corresponding to the provided [S2](#s2index)). Each **Syntax** ``` sql -s2ToGeo(s2index) +s2GetNeighbors(s2index) ``` **Arguments** @@ -156,7 +154,6 @@ Result: └───────────┘ ``` - ## s2CapContains {#s2CapContains} A cap represents a portion of the sphere that has been cut off by a plane. It is defined by a point on a sphere and a radius in degrees. @@ -166,7 +163,7 @@ Determines if a cap contains a s2 point index. **Syntax** ``` sql -s2CellsIntersect(center, degrees, point) +s2CapContains(center, degrees, point) ``` **Arguments** @@ -198,7 +195,6 @@ Result: └─────────────┘ ``` - ## s2CapUnion {#s2CapUnion} A cap represents a portion of the sphere that has been cut off by a plane. It is defined by a point on a sphere and a radius in degrees. @@ -279,7 +275,6 @@ Result: └───────────────────────────────────────────┘ ``` - ## s2RectContains{#s2RectContains} In the S2 system, a rectangle is represented by a type of S2Region called a S2LatLngRect that represents a rectangle in latitude-longitude space. From ada1baff74477c1fc792efb8e387bb7b5d0d4459 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sat, 25 Sep 2021 22:59:50 -0700 Subject: [PATCH 184/317] Add more review fixes --- docs/en/sql-reference/functions/geo/s2.md | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/functions/geo/s2.md b/docs/en/sql-reference/functions/geo/s2.md index 592ccedec0e..d669b1c8b32 100644 --- a/docs/en/sql-reference/functions/geo/s2.md +++ b/docs/en/sql-reference/functions/geo/s2.md @@ -6,6 +6,8 @@ toc_title: S2 Geometry [S2](https://s2geometry.io/) is a geographical indexing system where all geographical data is represented on a three-dimensional sphere (similar to a globe). +In the S2 library points are represented as unit length vectors called S2 point indices (points on the surface of a three dimensional unit sphere) as opposed to traditional (latitude, longitude) pairs. + ## geoToS2 {#geoToS2} Returns [S2](#s2index) point index corresponding to the provided coordinates `(longitude, latitude)`. @@ -63,7 +65,6 @@ s2ToGeo(s2index) Type: `lon` - [Float64](../../../sql-reference/data-types/float.md). `lat` — [Float64](../../../sql-reference/data-types/float.md). - **Example** Query: @@ -98,8 +99,7 @@ s2GetNeighbors(s2index) - An array consisting of the 4 neighbor indices: `array[s2index1, s2index3, s2index2, s2index4]`. -Type: Each S2 index is [UInt64](../../../sql-reference/data-types/int-uint.md). - +Type: Each S2 index is [UInt64](../../../sql-reference/data-types/int-uint.md). **Example** @@ -217,7 +217,6 @@ s2CapUnion(center1, radius1, center2, radius2) - `center` - S2 point index corresponding the center of the smallest cap containing the two input caps. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). - `radius` - Radius of the smallest cap containing the two input caps. Type: [Float64](../../../sql-reference/data-types/float.md). - **Example** Query: @@ -240,7 +239,6 @@ In the S2 system, a rectangle is represented by a type of S2Region called a S2La Increases the size of the bounding rectangle to include the given S2 point index. - **Syntax** ``` sql @@ -258,7 +256,6 @@ s2RectAdd(s2pointLow, s2pointHigh, s2Point) - `s2PointLow` - Low S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). - `s2PointHigh` - Hight S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/float.md). - **Example** Query: @@ -336,7 +333,6 @@ s2RectUnion(s2Rect1PointLow, s2Rect1PointHi, s2Rect2PointLow, s2Rect2PointHi) - `s2UnionRect2PointLow` - Low S2 cell id corresponding to the union rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). - `s2UnionRect2PointHi` - High S2 cell id corresponding to the union rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). - **Example** Query: @@ -373,7 +369,6 @@ s2RectIntersection(s2Rect1PointLow, s2Rect1PointHi, s2Rect2PointLow, s2Rect2Poin - `s2UnionRect2PointLow` - Low S2 cell id corresponding to the rectangle containing the intersection of the given rectangles. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). - `s2UnionRect2PointHi` - Hi S2 cell id corresponding to the rectangle containing the intersection of the given rectangles. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). - **Example** Query: From 332a2ac990844d3406c3cd3874470c6f67f032ed Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 26 Sep 2021 14:58:55 +0300 Subject: [PATCH 185/317] temporarily fix tests --- tests/queries/0_stateless/02015_async_inserts_stress_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh index f9a58818404..f6e8a2dce14 100755 --- a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh +++ b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh @@ -55,7 +55,7 @@ export -f insert1 export -f insert2 export -f select1 export -f select2 -export -f truncate1 +# export -f truncate1 for _ in {1..5}; do timeout $TIMEOUT bash -c insert1 & From 01757a85a7d9acb21a0d08c2ac1151ff7bc04443 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 26 Sep 2021 14:59:39 +0300 Subject: [PATCH 186/317] temporarily fix tests --- tests/queries/0_stateless/02015_async_inserts_stress_long.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh index f6e8a2dce14..f01b3010d9c 100755 --- a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh +++ b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh @@ -64,7 +64,7 @@ done timeout $TIMEOUT bash -c select1 & timeout $TIMEOUT bash -c select2 & -timeout $TIMEOUT bash -c truncate1 & +# timeout $TIMEOUT bash -c truncate1 & wait echo "OK" From 874aef3e9a916ab31432d86c49778ac0e94bfd40 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sun, 26 Sep 2021 15:57:11 +0300 Subject: [PATCH 187/317] 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 188/317] 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 189/317] 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 190/317] 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 191/317] 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 192/317] 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 193/317] 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 59a69b64e00a16f997840f735633465454c59c01 Mon Sep 17 00:00:00 2001 From: Alexey Date: Sun, 26 Sep 2021 19:44:53 +0000 Subject: [PATCH 194/317] Improved text --- docs/en/sql-reference/functions/index.md | 31 +++++++++++++--------- docs/en/sql-reference/statements/system.md | 10 +++---- 2 files changed, 23 insertions(+), 18 deletions(-) diff --git a/docs/en/sql-reference/functions/index.md b/docs/en/sql-reference/functions/index.md index 643ba660253..3069a596721 100644 --- a/docs/en/sql-reference/functions/index.md +++ b/docs/en/sql-reference/functions/index.md @@ -59,41 +59,46 @@ A lambda function that accepts multiple arguments can also be passed to a higher For some functions the first argument (the lambda function) can be omitted. In this case, identical mapping is assumed. -## User Defined Functions {#user-defined-functions} +## SQL User Defined Functions {#user-defined-functions} Custom functions from lambda expressions can be created using the [CREATE FUNCTION](../statements/create/function.md) statement. To delete these functions use the [DROP FUNCTION](../statements/drop.md#drop-function) statement. -The other option is to create functions using XML configuration. Add the path to a function configuration file inside `user_defined_executable_functions_config` tag. Wildcard symbol `*` may be used inside the path. +## Executable User Defined Functions {#executable-user-defined-functions} +ClickHouse can call any external executable program or script to process data. Describe such functions in a [configuration file](../../operations/configuration-files.md) and add the path of that file to the main configuration in `user_defined_executable_functions_config` setting. If a wildcard symbol `*` is used in the path, then all files matching a pattern are loaded. Example: ``` xml *_function.xml ``` -Function configuration files are searched inside path specified by `user_files_path` setting. +User defined function configurations are searched relative to a path specified in the `user_files_path` setting. -Function configuration contains: +A function configuration contains the following settings: - `name` - a function name. - `command` - a command or a script to execute. -- `argument` - argument description with the `type` of an argument. Each argument is described in a separate tag. -- `format` - The format in which arguments are passed to a command. -- `return_type` - the type of a value returned by the function. -- `type` - a function type. If it is set to `executable` then single command is started. If it is set to `executable_pool` then several commands are started. -- `lifetime` - reload interval in seconds. +- `argument` - argument description with the `type` of an argument. Each argument is described in a separate setting. +- `format` - a [format](../../interfaces/formats.md) in which arguments are passed to the command. +- `return_type` - the type of a returned value. +- `max_command_execution_time` - the maximum number of seconds the function is allowed to process arguments. Optional. Default value is `10`. +- `command_termination_timeout` - ??? Optional. Default value is `10`. +- `type` - an executable type. If `type` is set to `executable` then single command is started. If it is set to `executable_pool` then a pool of commands is created. +- `pool_size` - a size of the command pool. Optional. Default value is `16`. +- `lifetime` - reload interval of the function in seconds. If it is set to `0` then function is not reloaded. +- `send_chunk_header` - ??? Optional. Default value is `false`. -A function command must read arguments from STDIN and must output result to STDOUT. It must process arguments in a loop. +The command must read arguments from STDIN and must output the result to STDOUT. The command must process arguments iteratively. That is after processing a set of arguments it must wait for the next set of arguments. **Example** The following example creates `my_function`. It gets single argument of type String. `xargs` command listens to STDIN and calls `echo` for every argument. ``` - executable my_function + xargs -I arg echo Processing arg String - String TabSeparated - xargs -I arg echo Processing arg + String + executable 0 diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index 5b2d0175786..bbe90013d11 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -13,7 +13,7 @@ The list of available `SYSTEM` statements: - [RELOAD MODELS](#query_language-system-reload-models) - [RELOAD MODEL](#query_language-system-reload-model) - [RELOAD FUNCTIONS](#query_language-system-reload-functions) -- [RELOAD FUNCTION](#query_language-system-reload-function) +- [RELOAD FUNCTION](#query_language-system-reload-functions) - [DROP DNS CACHE](#query_language-system-drop-dns-cache) - [DROP MARK CACHE](#query_language-system-drop-mark-cache) - [DROP UNCOMPRESSED CACHE](#query_language-system-drop-uncompressed-cache) @@ -87,15 +87,15 @@ SYSTEM RELOAD MODEL ## RELOAD FUNCTIONS {#query_language-system-reload-functions} +Reloads all registered [executable user defined functions](../functions/index.md#executable-user-defined-functions) or one of them from an XML configuration. + **Syntax** ```sql -[SYSTEM] RELOAD FUNCTIONS +RELOAD FUNCTIONS +RELOAD FUNCTION function_name ``` -## RELOAD FUNCTION {#query_language-system-reload-function} - - ## DROP DNS CACHE {#query_language-system-drop-dns-cache} Resets ClickHouse’s internal DNS cache. Sometimes (for old ClickHouse versions) it is necessary to use this command when changing the infrastructure (changing the IP address of another ClickHouse server or the server used by dictionaries). From e06115d77f2020315c2252e717edc46da5335d97 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sun, 26 Sep 2021 17:24:05 -0300 Subject: [PATCH 195/317] 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 196/317] 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 197/317] 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 198/317] 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 199/317] 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 c495297e65c93c4bf3d1af45df6434e639929a09 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 26 Sep 2021 22:18:38 +0300 Subject: [PATCH 200/317] Remove unused reset_timeout/access_timeout from AsynchronousInsertQueue --- src/Interpreters/AsynchronousInsertQueue.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index e92a7b04d74..f1cdb04758f 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -106,13 +106,10 @@ private: QueryIdToEntry currently_processing_queries; /// Logic and events behind queue are as follows: - /// - reset_timeout: if queue is empty for some time, then we delete the queue and free all associated resources, e.g. tables. /// - busy_timeout: if queue is active for too long and there are a lot of rapid inserts, then we dump the data, so it doesn't /// grow for a long period of time and users will be able to select new data in deterministic manner. /// - stale_timeout: if queue is stale for too long, then we dump the data too, so that users will be able to select the last /// piece of inserted data. - /// - access_timeout: also we have to check if user still has access to the tables periodically, and if the access is lost, then - /// we dump pending data and delete queue immediately. /// - max_data_size: if the maximum size of data is reached, then again we dump the data. const size_t max_data_size; /// in bytes From ccf5050f7b8cdd0ee60a8074068ccfe0a20b92da Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 26 Sep 2021 22:19:20 +0300 Subject: [PATCH 201/317] Increase async_insert_busy_timeout_ms to avoid 02015_async_inserts_2 flakiness It failed few times on CI [1], and it seems that 200ms is not enough: 2021.09.26 02:43:16.063992 [ 706 ] {d5144a51-f217-48f9-8630-29492f81aae5} ContextAccess (default): Access granted: INSERT(id, s) ON test_fptde5.async_inserts 2021.09.26 02:43:16.069188 [ 706 ] {d5144a51-f217-48f9-8630-29492f81aae5} AsynchronousInsertQueue: Have 1 pending inserts with total 11 bytes of data for query 'INSERT INTO test_fptde5.async_inserts FORMAT CSV' 2021.09.26 02:43:16.209661 [ 1940 ] {} DynamicQueryHandler: Request URI: /?database=test_fptde5&log_comment=/usr/share/clickhouse-test/queries/0_stateless/02015_async_inserts_2.sh&async_insert=1&wait_for_async_insert=1 2021.09.26 02:43:16.340496 [ 1938 ] {e01d640f-90a8-4306-808f-d732fb144a11} ContextAccess (default): Access granted: INSERT(id, s) ON test_fptde5.async_inserts 2021.09.26 02:43:16.401966 [ 1938 ] {e01d640f-90a8-4306-808f-d732fb144a11} AsynchronousInsertQueue: Have 1 pending inserts with total 11 bytes of data for query 'INSERT INTO test_fptde5.async_inserts FORMAT CSV' 2021.09.26 02:43:16.667513 [ 1940 ] {5c059e03-397f-4f9a-95ca-0fcb3c479235} ContextAccess (default): Access granted: INSERT(id, s) ON test_fptde5.async_inserts 2021.09.26 02:43:16.707577 [ 1392 ] {} AsynchronousInsertQueue: Failed parsing for query 'INSERT INTO test_fptde5.async_inserts FORMAT CSV' with query id d5144a51-f217-48f9-8630-29492f81aae5. DB::ParsingException: Cannot parse input: expected ',' before: 'qqqqqqqqqqq': 2021.09.26 02:43:16.789697 [ 1392 ] {} AsynchronousInsertQueue: Flushed 0 rows, 0 bytes for query 'INSERT INTO test_fptde5.async_inserts FORMAT CSV' 2021.09.26 02:43:16.872731 [ 1940 ] {5c059e03-397f-4f9a-95ca-0fcb3c479235} AsynchronousInsertQueue: Have 1 pending inserts with total 11 bytes of data for query 'INSERT INTO test_fptde5.async_inserts FORMAT CSV' 2021.09.26 02:43:17.472223 [ 1392 ] {} test_fptde5.async_inserts (b877377f-9706-488b-b877-377f9706f88b): Renaming temporary part tmp_insert_all_1_1_0 to all_1_1_0. 2021.09.26 02:43:17.527828 [ 1392 ] {} AsynchronousInsertQueue: Flushed 2 rows, 28 bytes for query 'INSERT INTO test_fptde5.async_inserts FORMAT CSV' 2021.09.26 02:43:18.344135 [ 1859 ] {} test_fptde5.async_inserts (b877377f-9706-488b-b877-377f9706f88b): Renaming temporary part tmp_insert_all_2_2_0 to all_2_2_0. 2021.09.26 02:43:18.373230 [ 1859 ] {} AsynchronousInsertQueue: Flushed 2 rows, 28 bytes for query 'INSERT INTO test_fptde5.async_inserts FORMAT CSV' [1]: https://clickhouse-test-reports.s3.yandex.net/29364/fb584715e175fef3a4c4373d1bbcebecab1db6d8/functional_stateless_tests_(thread).html#fail1 --- tests/config/users.d/timeouts.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/config/users.d/timeouts.xml b/tests/config/users.d/timeouts.xml index 7d58315bd94..8f7f127e452 100644 --- a/tests/config/users.d/timeouts.xml +++ b/tests/config/users.d/timeouts.xml @@ -8,6 +8,9 @@ 60000 30 + + + 5000 From 09232788c090c56fc488a5960e938b6354747e61 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 27 Sep 2021 10:55:33 +0300 Subject: [PATCH 202/317] 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 203/317] 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 204/317] 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 222ab7893144581cc18d37f7b7a9a8a26aaab0cf Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 23 Sep 2021 15:21:02 +0300 Subject: [PATCH 205/317] Convert EmbeddedRocksDBBlockInputStream to ISource --- .../EmbeddedRocksDBBlockInputStream.cpp | 23 +++++++++---------- .../RocksDB/EmbeddedRocksDBBlockInputStream.h | 21 +++++++++-------- .../RocksDB/StorageEmbeddedRocksDB.cpp | 9 +++----- 3 files changed, 25 insertions(+), 28 deletions(-) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp index 4900e17ad91..2e740b275b2 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -17,27 +16,27 @@ namespace ErrorCodes EmbeddedRocksDBBlockInputStream::EmbeddedRocksDBBlockInputStream( StorageEmbeddedRocksDB & storage_, - const StorageMetadataPtr & metadata_snapshot_, + const Block & header, size_t max_block_size_) - : storage(storage_) - , metadata_snapshot(metadata_snapshot_) + : SourceWithProgress(header) + , storage(storage_) , max_block_size(max_block_size_) { - sample_block = metadata_snapshot->getSampleBlock(); - primary_key_pos = sample_block.getPositionByName(storage.primary_key); + primary_key_pos = header.getPositionByName(storage.primary_key); } -Block EmbeddedRocksDBBlockInputStream::readImpl() +Chunk EmbeddedRocksDBBlockInputStream::generate() { - if (finished) - return {}; - if (!iterator) { iterator = std::unique_ptr(storage.rocksdb_ptr->NewIterator(rocksdb::ReadOptions())); iterator->SeekToFirst(); } + if (!iterator->Valid()) + return {}; + + const auto & sample_block = getPort().getHeader(); MutableColumns columns = sample_block.cloneEmptyColumns(); for (size_t rows = 0; iterator->Valid() && rows < max_block_size; ++rows, iterator->Next()) @@ -54,13 +53,13 @@ Block EmbeddedRocksDBBlockInputStream::readImpl() } } - finished = !iterator->Valid(); if (!iterator->status().ok()) { throw Exception("Engine " + getName() + " got error while seeking key value data: " + iterator->status().ToString(), ErrorCodes::ROCKSDB_ERROR); } - return sample_block.cloneWithColumns(std::move(columns)); + Block block = sample_block.cloneWithColumns(std::move(columns)); + return Chunk(block.getColumns(), block.rows()); } } diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h index ddff1fc2e84..16760f4fe90 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h @@ -1,6 +1,7 @@ #pragma once -#include +#include +#include namespace rocksdb @@ -12,28 +13,28 @@ namespace DB { class StorageEmbeddedRocksDB; -struct StorageInMemoryMetadata; -using StorageMetadataPtr = std::shared_ptr; -class EmbeddedRocksDBBlockInputStream : public IBlockInputStream +class EmbeddedRocksDBBlockInputStream : public SourceWithProgress { public: EmbeddedRocksDBBlockInputStream( - StorageEmbeddedRocksDB & storage_, const StorageMetadataPtr & metadata_snapshot_, size_t max_block_size_); + StorageEmbeddedRocksDB & storage_, + const Block & header, + size_t max_block_size_); String getName() const override { return "EmbeddedRocksDB"; } - Block getHeader() const override { return sample_block; } - Block readImpl() override; + +protected: + Chunk generate() override; private: StorageEmbeddedRocksDB & storage; - StorageMetadataPtr metadata_snapshot; const size_t max_block_size; - Block sample_block; std::unique_ptr iterator; size_t primary_key_pos; - bool finished = false; + }; + } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 0b09a1f94d5..cd4bbc10bc3 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -379,7 +379,6 @@ void StorageEmbeddedRocksDB::initDB() rocksdb_ptr = std::unique_ptr(db); } - Pipe StorageEmbeddedRocksDB::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -394,13 +393,12 @@ Pipe StorageEmbeddedRocksDB::read( FieldVectorPtr keys; bool all_scan = false; - auto primary_key_data_type = metadata_snapshot->getSampleBlock().getByName(primary_key).type; + Block sample_block = metadata_snapshot->getSampleBlock(); + auto primary_key_data_type = sample_block.getByName(primary_key).type; std::tie(keys, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info); if (all_scan) { - auto reader = std::make_shared( - *this, metadata_snapshot, max_block_size); - return Pipe(std::make_shared(reader)); + return Pipe(std::make_shared(*this, sample_block, max_block_size)); } else { @@ -436,7 +434,6 @@ SinkToStoragePtr StorageEmbeddedRocksDB::write( return std::make_shared(*this, metadata_snapshot); } - static StoragePtr create(const StorageFactory::Arguments & args) { // TODO custom RocksDBSettings, table function From dc7393e810c43fb7b6246cfc540604663a88462c Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 23 Sep 2021 15:22:53 +0300 Subject: [PATCH 206/317] Rename EmbeddedRocksDBBlockInputStream -> EmbeddedRocksDBSource --- ...ocksDBBlockInputStream.cpp => EmbeddedRocksDBSource.cpp} | 6 +++--- ...dedRocksDBBlockInputStream.h => EmbeddedRocksDBSource.h} | 4 ++-- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 4 ++-- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) rename src/Storages/RocksDB/{EmbeddedRocksDBBlockInputStream.cpp => EmbeddedRocksDBSource.cpp} (90%) rename src/Storages/RocksDB/{EmbeddedRocksDBBlockInputStream.h => EmbeddedRocksDBSource.h} (84%) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBSource.cpp similarity index 90% rename from src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp rename to src/Storages/RocksDB/EmbeddedRocksDBSource.cpp index 2e740b275b2..f4bf20a6517 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBSource.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include @@ -14,7 +14,7 @@ namespace ErrorCodes extern const int ROCKSDB_ERROR; } -EmbeddedRocksDBBlockInputStream::EmbeddedRocksDBBlockInputStream( +EmbeddedRocksDBSource::EmbeddedRocksDBSource( StorageEmbeddedRocksDB & storage_, const Block & header, size_t max_block_size_) @@ -25,7 +25,7 @@ EmbeddedRocksDBBlockInputStream::EmbeddedRocksDBBlockInputStream( primary_key_pos = header.getPositionByName(storage.primary_key); } -Chunk EmbeddedRocksDBBlockInputStream::generate() +Chunk EmbeddedRocksDBSource::generate() { if (!iterator) { diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBSource.h similarity index 84% rename from src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h rename to src/Storages/RocksDB/EmbeddedRocksDBSource.h index 16760f4fe90..c873de6ff79 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBSource.h @@ -14,11 +14,11 @@ namespace DB class StorageEmbeddedRocksDB; -class EmbeddedRocksDBBlockInputStream : public SourceWithProgress +class EmbeddedRocksDBSource : public SourceWithProgress { public: - EmbeddedRocksDBBlockInputStream( + EmbeddedRocksDBSource( StorageEmbeddedRocksDB & storage_, const Block & header, size_t max_block_size_); diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index cd4bbc10bc3..2c31ca71420 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include @@ -398,7 +398,7 @@ Pipe StorageEmbeddedRocksDB::read( std::tie(keys, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info); if (all_scan) { - return Pipe(std::make_shared(*this, sample_block, max_block_size)); + return Pipe(std::make_shared(*this, sample_block, max_block_size)); } else { diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index b095673a6f5..9d897b7085c 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -23,7 +23,7 @@ class StorageEmbeddedRocksDB final : public shared_ptr_helper; friend class EmbeddedRocksDBSink; - friend class EmbeddedRocksDBBlockInputStream; + friend class EmbeddedRocksDBSource; public: std::string getName() const override { return "EmbeddedRocksDB"; } From 8fe17dd3d1b05e13436a494ed4de985a4f58681e Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 23 Sep 2021 15:24:46 +0300 Subject: [PATCH 207/317] Revert "Rename EmbeddedRocksDBBlockInputStream -> EmbeddedRocksDBSource" This reverts commit 3ab217031e50a0b4d19fb24d72088e353cdfa510. --- ...ocksDBSource.cpp => EmbeddedRocksDBBlockInputStream.cpp} | 6 +++--- ...dedRocksDBSource.h => EmbeddedRocksDBBlockInputStream.h} | 4 ++-- src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp | 4 ++-- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) rename src/Storages/RocksDB/{EmbeddedRocksDBSource.cpp => EmbeddedRocksDBBlockInputStream.cpp} (90%) rename src/Storages/RocksDB/{EmbeddedRocksDBSource.h => EmbeddedRocksDBBlockInputStream.h} (84%) diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSource.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp similarity index 90% rename from src/Storages/RocksDB/EmbeddedRocksDBSource.cpp rename to src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp index f4bf20a6517..2e740b275b2 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSource.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include @@ -14,7 +14,7 @@ namespace ErrorCodes extern const int ROCKSDB_ERROR; } -EmbeddedRocksDBSource::EmbeddedRocksDBSource( +EmbeddedRocksDBBlockInputStream::EmbeddedRocksDBBlockInputStream( StorageEmbeddedRocksDB & storage_, const Block & header, size_t max_block_size_) @@ -25,7 +25,7 @@ EmbeddedRocksDBSource::EmbeddedRocksDBSource( primary_key_pos = header.getPositionByName(storage.primary_key); } -Chunk EmbeddedRocksDBSource::generate() +Chunk EmbeddedRocksDBBlockInputStream::generate() { if (!iterator) { diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSource.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h similarity index 84% rename from src/Storages/RocksDB/EmbeddedRocksDBSource.h rename to src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h index c873de6ff79..16760f4fe90 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSource.h +++ b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h @@ -14,11 +14,11 @@ namespace DB class StorageEmbeddedRocksDB; -class EmbeddedRocksDBSource : public SourceWithProgress +class EmbeddedRocksDBBlockInputStream : public SourceWithProgress { public: - EmbeddedRocksDBSource( + EmbeddedRocksDBBlockInputStream( StorageEmbeddedRocksDB & storage_, const Block & header, size_t max_block_size_); diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index 2c31ca71420..cd4bbc10bc3 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include @@ -398,7 +398,7 @@ Pipe StorageEmbeddedRocksDB::read( std::tie(keys, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info); if (all_scan) { - return Pipe(std::make_shared(*this, sample_block, max_block_size)); + return Pipe(std::make_shared(*this, sample_block, max_block_size)); } else { diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index 9d897b7085c..b095673a6f5 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -23,7 +23,7 @@ class StorageEmbeddedRocksDB final : public shared_ptr_helper; friend class EmbeddedRocksDBSink; - friend class EmbeddedRocksDBSource; + friend class EmbeddedRocksDBBlockInputStream; public: std::string getName() const override { return "EmbeddedRocksDB"; } From 462d36195c94d50e9c2438d3ab18b93823dd4df2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 27 Sep 2021 12:32:35 +0200 Subject: [PATCH 208/317] Allow reloading max_concurrent_queries without restart --- programs/server/Server.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 5804d28d337..11b8c2b2540 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -859,6 +859,9 @@ if (ThreadFuzzer::instance().isEffective()) if (config->has("max_partition_size_to_drop")) global_context->setMaxPartitionSizeToDrop(config->getUInt64("max_partition_size_to_drop")); + if (config->has("max_concurrent_queries")) + global_context->getProcessList().setMaxSize(config->getInt("max_concurrent_queries", 0)); + if (!initial_loading) { /// We do not load ZooKeeper configuration on the first config loading From a73064679dc77a7c35111da87ded9a8f9d38cf72 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 27 Sep 2021 14:54:04 +0300 Subject: [PATCH 209/317] Compress keeper snapshots with default ZSTD codec --- src/Coordination/CoordinationSettings.h | 4 +- src/Coordination/KeeperSnapshotManager.cpp | 57 ++++++++++--- src/Coordination/KeeperSnapshotManager.h | 52 +++++++++++- src/Coordination/KeeperStateMachine.cpp | 5 +- src/Coordination/tests/gtest_coordination.cpp | 85 +++++++++++++++---- 5 files changed, 170 insertions(+), 33 deletions(-) diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 8b0d036ec8e..805dedab89c 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -16,6 +16,7 @@ struct Settings; * and should not be changed by the user without a reason. */ + #define LIST_OF_COORDINATION_SETTINGS(M) \ M(Milliseconds, session_timeout_ms, Coordination::DEFAULT_SESSION_TIMEOUT_MS, "Default client session timeout", 0) \ M(Milliseconds, operation_timeout_ms, Coordination::DEFAULT_OPERATION_TIMEOUT_MS, "Default client operation timeout", 0) \ @@ -36,7 +37,8 @@ struct Settings; M(UInt64, max_requests_batch_size, 100, "Max size of batch in requests count before it will be sent to RAFT", 0) \ M(Bool, quorum_reads, false, "Execute read requests as writes through whole RAFT consesus with similar speed", 0) \ M(Bool, force_sync, true, "Call fsync on each change in RAFT changelog", 0) \ - M(Bool, compress_logs, true, "Write compressed coordination logs in ZSTD format", 0) + M(Bool, compress_logs, true, "Write compressed coordination logs in ZSTD format", 0) \ + M(Bool, compress_snapshots_with_zstd_format, true, "Write compressed snapshots in ZSTD format (instead of custom LZ4)", 0) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 40c898efdb5..a4e0f9fe937 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB { @@ -32,9 +33,12 @@ namespace return parse(name_parts[1]); } - std::string getSnapshotFileName(uint64_t up_to_log_idx) + std::string getSnapshotFileName(uint64_t up_to_log_idx, bool compress_zstd) { - return std::string{"snapshot_"} + std::to_string(up_to_log_idx) + ".bin"; + auto base = std::string{"snapshot_"} + std::to_string(up_to_log_idx) + ".bin"; + if (compress_zstd) + base += ".zstd"; + return base; } std::string getBaseName(const String & path) @@ -218,6 +222,7 @@ SnapshotMetadataPtr KeeperStorageSnapshot::deserialize(KeeperStorage & storage, storage.zxid = result->get_last_log_idx(); storage.session_id_counter = session_id; + /// Before V1 we serialized ACL without acl_map if (current_version >= SnapshotVersion::V1) { size_t acls_map_size; @@ -338,9 +343,13 @@ KeeperStorageSnapshot::~KeeperStorageSnapshot() storage->disableSnapshotMode(); } -KeeperSnapshotManager::KeeperSnapshotManager(const std::string & snapshots_path_, size_t snapshots_to_keep_, const std::string & superdigest_, size_t storage_tick_time_) +KeeperSnapshotManager::KeeperSnapshotManager( + const std::string & snapshots_path_, size_t snapshots_to_keep_, + bool compress_snapshots_zstd_, + const std::string & superdigest_, size_t storage_tick_time_) : snapshots_path(snapshots_path_) , snapshots_to_keep(snapshots_to_keep_) + , compress_snapshots_zstd(compress_snapshots_zstd_) , superdigest(superdigest_) , storage_tick_time(storage_tick_time_) { @@ -380,7 +389,7 @@ std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer { ReadBufferFromNuraftBuffer reader(buffer); - auto snapshot_file_name = getSnapshotFileName(up_to_log_idx); + auto snapshot_file_name = getSnapshotFileName(up_to_log_idx, compress_snapshots_zstd); auto tmp_snapshot_file_name = "tmp_" + snapshot_file_name; std::string tmp_snapshot_path = std::filesystem::path{snapshots_path} / tmp_snapshot_file_name; std::string new_snapshot_path = std::filesystem::path{snapshots_path} / snapshot_file_name; @@ -428,20 +437,44 @@ nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFrom nuraft::ptr KeeperSnapshotManager::serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) { - WriteBufferFromNuraftBuffer writer; - CompressedWriteBuffer compressed_writer(writer); + std::unique_ptr writer = std::make_unique(); + auto buffer_raw_ptr = writer.get(); + std::unique_ptr compressed_writer; + if (compress_snapshots_zstd) + compressed_writer = wrapWriteBufferWithCompressionMethod(std::move(writer), CompressionMethod::Zstd, 3); + else + compressed_writer = std::make_unique(*writer); - KeeperStorageSnapshot::serialize(snapshot, compressed_writer); - compressed_writer.finalize(); - return writer.getBuffer(); + KeeperStorageSnapshot::serialize(snapshot, *compressed_writer); + compressed_writer->finalize(); + return buffer_raw_ptr->getBuffer(); +} + + +bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr buffer) const +{ + static constexpr uint32_t ZSTD_COMPRESSED_MAGIC = 0xFD2FB528; + ReadBufferFromNuraftBuffer reader(buffer); + uint32_t magic_from_buffer; + reader.readStrict(reinterpret_cast(&magic_from_buffer), sizeof(magic_from_buffer)); + buffer->pos(0); + return magic_from_buffer == ZSTD_COMPRESSED_MAGIC; } SnapshotMetaAndStorage KeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr buffer) const { - ReadBufferFromNuraftBuffer reader(buffer); - CompressedReadBuffer compressed_reader(reader); + bool is_zstd_compressed = isZstdCompressed(buffer); + + std::unique_ptr reader = std::make_unique(buffer); + std::unique_ptr compressed_reader; + + if (is_zstd_compressed) + compressed_reader = wrapReadBufferWithCompressionMethod(std::move(reader), CompressionMethod::Zstd); + else + compressed_reader = std::make_unique(*reader); + auto storage = std::make_unique(storage_tick_time, superdigest); - auto snapshot_metadata = KeeperStorageSnapshot::deserialize(*storage, compressed_reader); + auto snapshot_metadata = KeeperStorageSnapshot::deserialize(*storage, *compressed_reader); return std::make_pair(snapshot_metadata, std::move(storage)); } diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 57347c37b9f..10030d7daca 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -15,10 +15,19 @@ enum SnapshotVersion : uint8_t V0 = 0, V1 = 1, /// with ACL map V2 = 2, /// with 64 bit buffer header + V3 = 3, /// compress snapshots with ZSTD codec }; -static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V2; +static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V3; +/// In memory keeper snapshot. Keeper Storage based on a hash map which can be +/// turned into snapshot mode. This operation is fast and KeeperStorageSnapshot +/// class do it in constructor. It also copies iterators from storage hash table +/// up to some log index with lock. In destructor this class turn off snapshot +/// mode for KeeperStorage. +/// +/// This representation of snapshot have to be serialized into NuRaft +/// buffer and send over network or saved to file. struct KeeperStorageSnapshot { public: @@ -34,12 +43,20 @@ public: KeeperStorage * storage; SnapshotVersion version = CURRENT_SNAPSHOT_VERSION; + /// Snapshot metadata SnapshotMetadataPtr snapshot_meta; + /// Max session id int64_t session_id; + /// Size of snapshot container in amount of nodes after begin iterator + /// so we have for loop for (i = 0; i < snapshot_container_size; ++i) { doSmth(begin + i); } size_t snapshot_container_size; + /// Iterator to the start of the storage KeeperStorage::Container::const_iterator begin; + /// Active sessions and their timeouts SessionAndTimeout session_and_timeout; + /// Sessions credentials KeeperStorage::SessionAndAuth session_and_auth; + /// ACLs cache for better performance. Without we cannot deserialize storage. std::unordered_map acl_map; }; @@ -49,28 +66,42 @@ using CreateSnapshotCallback = std::function; using SnapshotMetaAndStorage = std::pair; +/// Class responsible for snapshots serialization and deserialization. Each snapshot +/// has it's path on disk and log index. class KeeperSnapshotManager { public: - KeeperSnapshotManager(const std::string & snapshots_path_, size_t snapshots_to_keep_, const std::string & superdigest_ = "", size_t storage_tick_time_ = 500); + KeeperSnapshotManager( + const std::string & snapshots_path_, size_t snapshots_to_keep_, + bool compress_snapshots_zstd_ = true, const std::string & superdigest_ = "", size_t storage_tick_time_ = 500); + /// Restore storage from latest available snapshot SnapshotMetaAndStorage restoreFromLatestSnapshot(); - static nuraft::ptr serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot); + /// Compress snapshot and serialize it to buffer + nuraft::ptr serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot); + + /// Serialize already compressed snapshot to disk (return path) std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx); SnapshotMetaAndStorage deserializeSnapshotFromBuffer(nuraft::ptr buffer) const; + /// Deserialize snapshot with log index up_to_log_idx from disk into compressed nuraft buffer. nuraft::ptr deserializeSnapshotBufferFromDisk(uint64_t up_to_log_idx) const; + + /// Deserialize latest snapshot from disk into compressed nuraft buffer. nuraft::ptr deserializeLatestSnapshotBufferFromDisk(); + /// Remove snapshot with this log_index void removeSnapshot(uint64_t log_idx); + /// Total amount of snapshots size_t totalSnapshots() const { return existing_snapshots.size(); } + /// The most fresh snapshot log index we have size_t getLatestSnapshotIndex() const { if (!existing_snapshots.empty()) @@ -80,13 +111,28 @@ public: private: void removeOutdatedSnapshotsIfNeeded(); + + /// Checks first 4 buffer bytes to became sure that snapshot compressed with + /// ZSTD codec. + bool isZstdCompressed(nuraft::ptr buffer) const; + const std::string snapshots_path; + /// How many snapshots to keep before remove const size_t snapshots_to_keep; + /// All existing snapshots in our path (log_index -> path) std::map existing_snapshots; + /// Compress snapshots in common ZSTD format instead of custom ClickHouse block LZ4 format + const bool compress_snapshots_zstd; + /// Superdigest for deserialization of storage const std::string superdigest; + /// Storage sessions timeout check interval (also for deserializatopn) size_t storage_tick_time; }; +/// Keeper create snapshots in background thread. KeeperStateMachine just create +/// in-memory snapshot from storage and push task for it serialization into +/// special tasks queue. Background thread check this queue and after snapshot +/// successfully serialized notify state machine. struct CreateSnapshotTask { KeeperStorageSnapshotPtr snapshot; diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index c10303062ef..be4f73cf5ae 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -46,7 +46,10 @@ KeeperStateMachine::KeeperStateMachine( const CoordinationSettingsPtr & coordination_settings_, const std::string & superdigest_) : coordination_settings(coordination_settings_) - , snapshot_manager(snapshots_path_, coordination_settings->snapshots_to_keep, superdigest_, coordination_settings->dead_session_check_period_ms.totalMicroseconds()) + , snapshot_manager( + snapshots_path_, coordination_settings->snapshots_to_keep, + coordination_settings->compress_snapshots_with_zstd_format, superdigest_, + coordination_settings->dead_session_check_period_ms.totalMicroseconds()) , responses_queue(responses_queue_) , snapshots_queue(snapshots_queue_) , last_committed_idx(0) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 256afe56d36..3ef0ebf99df 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -934,8 +934,9 @@ void addNode(DB::KeeperStorage & storage, const std::string & path, const std::s TEST_P(CoordinationTest, TestStorageSnapshotSimple) { + auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3); + DB::KeeperSnapshotManager manager("./snapshots", 3, params.enable_compression); DB::KeeperStorage storage(500, ""); addNode(storage, "/hello", "world", 1); @@ -956,7 +957,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 2); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin")); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin" + params.extension)); auto debuf = manager.deserializeSnapshotBufferFromDisk(2); @@ -981,8 +982,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotSimple) TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) { + auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3); + DB::KeeperSnapshotManager manager("./snapshots", 3, params.enable_compression); DB::KeeperStorage storage(500, ""); storage.getSessionID(130); @@ -1005,7 +1007,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin")); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); auto debuf = manager.deserializeSnapshotBufferFromDisk(50); @@ -1021,8 +1023,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotMoreWrites) TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) { + auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3); + DB::KeeperSnapshotManager manager("./snapshots", 3, params.enable_compression); DB::KeeperStorage storage(500, ""); storage.getSessionID(130); @@ -1037,14 +1040,14 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) DB::KeeperStorageSnapshot snapshot(&storage, j * 50); auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, j * 50); - EXPECT_TRUE(fs::exists(std::string{"./snapshots/snapshot_"} + std::to_string(j * 50) + ".bin")); + EXPECT_TRUE(fs::exists(std::string{"./snapshots/snapshot_"} + std::to_string(j * 50) + ".bin" + params.extension)); } - EXPECT_FALSE(fs::exists("./snapshots/snapshot_50.bin")); - EXPECT_FALSE(fs::exists("./snapshots/snapshot_100.bin")); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_150.bin")); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_200.bin")); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_250.bin")); + EXPECT_FALSE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); + EXPECT_FALSE(fs::exists("./snapshots/snapshot_100.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_150.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_200.bin" + params.extension)); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_250.bin" + params.extension)); auto [meta, restored_storage] = manager.restoreFromLatestSnapshot(); @@ -1059,8 +1062,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotManySnapshots) TEST_P(CoordinationTest, TestStorageSnapshotMode) { + auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3); + DB::KeeperSnapshotManager manager("./snapshots", 3, params.enable_compression); DB::KeeperStorage storage(500, ""); for (size_t i = 0; i < 50; ++i) { @@ -1087,7 +1091,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); } - EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin")); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); EXPECT_EQ(storage.container.size(), 26); storage.clearGarbageAfterSnapshot(); EXPECT_EQ(storage.container.snapshotSize(), 26); @@ -1110,8 +1114,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) TEST_P(CoordinationTest, TestStorageSnapshotBroken) { + auto params = GetParam(); ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3); + DB::KeeperSnapshotManager manager("./snapshots", 3, params.enable_compression); DB::KeeperStorage storage(500, ""); for (size_t i = 0; i < 50; ++i) { @@ -1122,10 +1127,10 @@ TEST_P(CoordinationTest, TestStorageSnapshotBroken) auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 50); } - EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin")); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); /// Let's corrupt file - DB::WriteBufferFromFile plain_buf("./snapshots/snapshot_50.bin", DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + DB::WriteBufferFromFile plain_buf("./snapshots/snapshot_50.bin" + params.extension, DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(34); plain_buf.sync(); @@ -1464,6 +1469,54 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) } +TEST_P(CoordinationTest, TestStorageSnapshotDifferenVersions) +{ + auto params = GetParam(); + if (!params.enable_compression) + return; + + ChangelogDirTest test("./snapshots"); + DB::KeeperSnapshotManager manager("./snapshots", 3, false); + + DB::KeeperStorage storage(500, ""); + addNode(storage, "/hello", "world", 1); + addNode(storage, "/hello/somepath", "somedata", 3); + storage.session_id_counter = 5; + storage.zxid = 2; + storage.ephemerals[3] = {"/hello"}; + storage.ephemerals[1] = {"/hello/somepath"}; + storage.getSessionID(130); + storage.getSessionID(130); + + DB::KeeperStorageSnapshot snapshot(&storage, 2); + + auto buf = manager.serializeSnapshotToBuffer(snapshot); + manager.serializeSnapshotBufferToDisk(*buf, 2); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin")); + + DB::KeeperSnapshotManager new_manager("./snapshots", 3, true); + + auto debuf = new_manager.deserializeSnapshotBufferFromDisk(2); + + auto [snapshot_meta, restored_storage] = new_manager.deserializeSnapshotFromBuffer(debuf); + + EXPECT_EQ(restored_storage->container.size(), 3); + EXPECT_EQ(restored_storage->container.getValue("/").children.size(), 1); + EXPECT_EQ(restored_storage->container.getValue("/hello").children.size(), 1); + EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").children.size(), 0); + + EXPECT_EQ(restored_storage->container.getValue("/").data, ""); + EXPECT_EQ(restored_storage->container.getValue("/hello").data, "world"); + EXPECT_EQ(restored_storage->container.getValue("/hello/somepath").data, "somedata"); + EXPECT_EQ(restored_storage->session_id_counter, 7); + EXPECT_EQ(restored_storage->zxid, 2); + EXPECT_EQ(restored_storage->ephemerals.size(), 2); + EXPECT_EQ(restored_storage->ephemerals[3].size(), 1); + EXPECT_EQ(restored_storage->ephemerals[1].size(), 1); + EXPECT_EQ(restored_storage->session_and_timeout.size(), 2); +} + + INSTANTIATE_TEST_SUITE_P(CoordinationTestSuite, CoordinationTest, ::testing::ValuesIn(std::initializer_list{ From 8ce21a70bba0d3fc3958d16d054fc7980c404c04 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 27 Sep 2021 16:06:35 +0300 Subject: [PATCH 210/317] 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 aea2b952fd4b5e917e2c7a865354c9315a56e88e Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 27 Sep 2021 15:10:03 +0300 Subject: [PATCH 211/317] Merge EmbeddedRocksDBBlockInputStream and EmbeddedRocksDBSource --- .../EmbeddedRocksDBBlockInputStream.cpp | 65 ------------ .../RocksDB/EmbeddedRocksDBBlockInputStream.h | 40 -------- .../RocksDB/StorageEmbeddedRocksDB.cpp | 98 ++++++++++++++----- src/Storages/RocksDB/StorageEmbeddedRocksDB.h | 1 - 4 files changed, 75 insertions(+), 129 deletions(-) delete mode 100644 src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp delete mode 100644 src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp deleted file mode 100644 index 2e740b275b2..00000000000 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.cpp +++ /dev/null @@ -1,65 +0,0 @@ -#include -#include -#include -#include - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ROCKSDB_ERROR; -} - -EmbeddedRocksDBBlockInputStream::EmbeddedRocksDBBlockInputStream( - StorageEmbeddedRocksDB & storage_, - const Block & header, - size_t max_block_size_) - : SourceWithProgress(header) - , storage(storage_) - , max_block_size(max_block_size_) -{ - primary_key_pos = header.getPositionByName(storage.primary_key); -} - -Chunk EmbeddedRocksDBBlockInputStream::generate() -{ - if (!iterator) - { - iterator = std::unique_ptr(storage.rocksdb_ptr->NewIterator(rocksdb::ReadOptions())); - iterator->SeekToFirst(); - } - - if (!iterator->Valid()) - return {}; - - const auto & sample_block = getPort().getHeader(); - MutableColumns columns = sample_block.cloneEmptyColumns(); - - for (size_t rows = 0; iterator->Valid() && rows < max_block_size; ++rows, iterator->Next()) - { - ReadBufferFromString key_buffer(iterator->key()); - ReadBufferFromString value_buffer(iterator->value()); - - size_t idx = 0; - for (const auto & elem : sample_block) - { - auto serialization = elem.type->getDefaultSerialization(); - serialization->deserializeBinary(*columns[idx], idx == primary_key_pos ? key_buffer : value_buffer); - ++idx; - } - } - - if (!iterator->status().ok()) - { - throw Exception("Engine " + getName() + " got error while seeking key value data: " + iterator->status().ToString(), - ErrorCodes::ROCKSDB_ERROR); - } - Block block = sample_block.cloneWithColumns(std::move(columns)); - return Chunk(block.getColumns(), block.rows()); -} - -} diff --git a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h b/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h deleted file mode 100644 index 16760f4fe90..00000000000 --- a/src/Storages/RocksDB/EmbeddedRocksDBBlockInputStream.h +++ /dev/null @@ -1,40 +0,0 @@ -#pragma once - -#include -#include - - -namespace rocksdb -{ - class Iterator; -} - -namespace DB -{ - -class StorageEmbeddedRocksDB; - -class EmbeddedRocksDBBlockInputStream : public SourceWithProgress -{ - -public: - EmbeddedRocksDBBlockInputStream( - StorageEmbeddedRocksDB & storage_, - const Block & header, - size_t max_block_size_); - - String getName() const override { return "EmbeddedRocksDB"; } - -protected: - Chunk generate() override; - -private: - StorageEmbeddedRocksDB & storage; - const size_t max_block_size; - - std::unique_ptr iterator; - size_t primary_key_pos; - -}; - -} diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp index cd4bbc10bc3..820b7c94ebd 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.cpp @@ -1,6 +1,5 @@ #include #include -#include #include @@ -182,15 +181,15 @@ class EmbeddedRocksDBSource : public SourceWithProgress public: EmbeddedRocksDBSource( const StorageEmbeddedRocksDB & storage_, - const StorageMetadataPtr & metadata_snapshot_, + const Block & header, FieldVectorPtr keys_, FieldVector::const_iterator begin_, FieldVector::const_iterator end_, const size_t max_block_size_) - : SourceWithProgress(metadata_snapshot_->getSampleBlock()) + : SourceWithProgress(header) , storage(storage_) - , metadata_snapshot(metadata_snapshot_) - , keys(std::move(keys_)) + , primary_key_pos(header.getPositionByName(storage.getPrimaryKey())) + , keys(keys_) , begin(begin_) , end(end_) , it(begin) @@ -198,12 +197,29 @@ public: { } - String getName() const override + EmbeddedRocksDBSource( + const StorageEmbeddedRocksDB & storage_, + const Block & header, + std::unique_ptr iterator_, + const size_t max_block_size_) + : SourceWithProgress(header) + , storage(storage_) + , primary_key_pos(header.getPositionByName(storage.getPrimaryKey())) + , iterator(std::move(iterator_)) + , max_block_size(max_block_size_) { - return storage.getName(); } + String getName() const override { return storage.getName(); } + Chunk generate() override + { + if (keys) + return generateWithKeys(); + return generateFullScan(); + } + + Chunk generateWithKeys() { if (it >= end) return {}; @@ -213,16 +229,15 @@ public: std::vector serialized_keys(num_keys); std::vector slices_keys(num_keys); - const auto & sample_block = metadata_snapshot->getSampleBlock(); - const auto & key_column = sample_block.getByName(storage.getPrimaryKey()); - auto columns = sample_block.cloneEmptyColumns(); - size_t primary_key_pos = sample_block.getPositionByName(storage.getPrimaryKey()); + const auto & sample_block = getPort().getHeader(); + + const auto & key_column_type = sample_block.getByName(storage.getPrimaryKey()).type; size_t rows_processed = 0; while (it < end && rows_processed < max_block_size) { WriteBufferFromString wb(serialized_keys[rows_processed]); - key_column.type->getDefaultSerialization()->serializeBinary(*it, wb); + key_column_type->getDefaultSerialization()->serializeBinary(*it, wb); wb.finalize(); slices_keys[rows_processed] = std::move(serialized_keys[rows_processed]); @@ -230,6 +245,7 @@ public: ++rows_processed; } + MutableColumns columns = sample_block.cloneEmptyColumns(); std::vector values; auto statuses = storage.multiGet(slices_keys, values); for (size_t i = 0; i < statuses.size(); ++i) @@ -238,13 +254,7 @@ public: { ReadBufferFromString key_buffer(slices_keys[i]); ReadBufferFromString value_buffer(values[i]); - - size_t idx = 0; - for (const auto & elem : sample_block) - { - elem.type->getDefaultSerialization()->deserializeBinary(*columns[idx], idx == primary_key_pos ? key_buffer : value_buffer); - ++idx; - } + fillColumns(key_buffer, value_buffer, columns); } } @@ -252,14 +262,54 @@ public: return Chunk(std::move(columns), num_rows); } + Chunk generateFullScan() + { + if (!iterator->Valid()) + return {}; + + const auto & sample_block = getPort().getHeader(); + MutableColumns columns = sample_block.cloneEmptyColumns(); + + for (size_t rows = 0; iterator->Valid() && rows < max_block_size; ++rows, iterator->Next()) + { + ReadBufferFromString key_buffer(iterator->key()); + ReadBufferFromString value_buffer(iterator->value()); + fillColumns(key_buffer, value_buffer, columns); + } + + if (!iterator->status().ok()) + { + throw Exception("Engine " + getName() + " got error while seeking key value data: " + iterator->status().ToString(), + ErrorCodes::ROCKSDB_ERROR); + } + Block block = sample_block.cloneWithColumns(std::move(columns)); + return Chunk(block.getColumns(), block.rows()); + } + + void fillColumns(ReadBufferFromString & key_buffer, ReadBufferFromString & value_buffer, MutableColumns & columns) + { + size_t idx = 0; + for (const auto & elem : getPort().getHeader()) + { + elem.type->getDefaultSerialization()->deserializeBinary(*columns[idx], idx == primary_key_pos ? key_buffer : value_buffer); + ++idx; + } + } + private: const StorageEmbeddedRocksDB & storage; - const StorageMetadataPtr metadata_snapshot; - FieldVectorPtr keys; + size_t primary_key_pos; + + /// For key scan + FieldVectorPtr keys = nullptr; FieldVector::const_iterator begin; FieldVector::const_iterator end; FieldVector::const_iterator it; + + /// For full scan + std::unique_ptr iterator = nullptr; + const size_t max_block_size; }; @@ -398,7 +448,9 @@ Pipe StorageEmbeddedRocksDB::read( std::tie(keys, all_scan) = getFilterKeys(primary_key, primary_key_data_type, query_info); if (all_scan) { - return Pipe(std::make_shared(*this, sample_block, max_block_size)); + auto iterator = std::unique_ptr(rocksdb_ptr->NewIterator(rocksdb::ReadOptions())); + iterator->SeekToFirst(); + return Pipe(std::make_shared(*this, sample_block, std::move(iterator), max_block_size)); } else { @@ -422,7 +474,7 @@ Pipe StorageEmbeddedRocksDB::read( size_t end = num_keys * (thread_idx + 1) / num_threads; pipes.emplace_back(std::make_shared( - *this, metadata_snapshot, keys, keys->begin() + begin, keys->begin() + end, max_block_size)); + *this, sample_block, keys, keys->begin() + begin, keys->begin() + end, max_block_size)); } return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h index b095673a6f5..ade4bd58bf4 100644 --- a/src/Storages/RocksDB/StorageEmbeddedRocksDB.h +++ b/src/Storages/RocksDB/StorageEmbeddedRocksDB.h @@ -23,7 +23,6 @@ class StorageEmbeddedRocksDB final : public shared_ptr_helper; friend class EmbeddedRocksDBSink; - friend class EmbeddedRocksDBBlockInputStream; public: std::string getName() const override { return "EmbeddedRocksDB"; } From 608644436cd7cb715478ff8fdd936f22460e50e9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 27 Sep 2021 17:21:10 +0300 Subject: [PATCH 212/317] Fix tidy --- src/Coordination/KeeperSnapshotManager.cpp | 6 +++--- src/Coordination/KeeperSnapshotManager.h | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index a4e0f9fe937..0a31878879c 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -435,10 +435,10 @@ nuraft::ptr KeeperSnapshotManager::deserializeSnapshotBufferFrom return writer.getBuffer(); } -nuraft::ptr KeeperSnapshotManager::serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) +nuraft::ptr KeeperSnapshotManager::serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const { std::unique_ptr writer = std::make_unique(); - auto buffer_raw_ptr = writer.get(); + auto * buffer_raw_ptr = writer.get(); std::unique_ptr compressed_writer; if (compress_snapshots_zstd) compressed_writer = wrapWriteBufferWithCompressionMethod(std::move(writer), CompressionMethod::Zstd, 3); @@ -451,7 +451,7 @@ nuraft::ptr KeeperSnapshotManager::serializeSnapshotToBuffer(con } -bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr buffer) const +bool KeeperSnapshotManager::isZstdCompressed(nuraft::ptr buffer) { static constexpr uint32_t ZSTD_COMPRESSED_MAGIC = 0xFD2FB528; ReadBufferFromNuraftBuffer reader(buffer); diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 10030d7daca..1d14409a574 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -79,7 +79,7 @@ public: SnapshotMetaAndStorage restoreFromLatestSnapshot(); /// Compress snapshot and serialize it to buffer - nuraft::ptr serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot); + nuraft::ptr serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot) const; /// Serialize already compressed snapshot to disk (return path) std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, uint64_t up_to_log_idx); @@ -114,7 +114,7 @@ private: /// Checks first 4 buffer bytes to became sure that snapshot compressed with /// ZSTD codec. - bool isZstdCompressed(nuraft::ptr buffer) const; + static bool isZstdCompressed(nuraft::ptr buffer); const std::string snapshots_path; /// How many snapshots to keep before remove From cd9c796e6fe1c8aa1a7760fda337a343b80123dd Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 27 Sep 2021 17:23:56 +0300 Subject: [PATCH 213/317] 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 b28d6e76ccbed8bbbec43ef42fefe61071668d0b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 27 Sep 2021 17:26:25 +0300 Subject: [PATCH 214/317] Map bloom filter full text index added support for mapKeys, mapValues --- .../MergeTree/MergeTreeIndexFullText.cpp | 364 ++++++++++-------- .../MergeTree/MergeTreeIndexFullText.h | 9 +- ...ap_full_text_bloom_filter_index.reference} | 0 ...2000_map_full_text_bloom_filter_index.sql} | 4 +- 4 files changed, 219 insertions(+), 158 deletions(-) rename tests/queries/0_stateless/{02000_bloom_filter_map.reference => 02000_map_full_text_bloom_filter_index.reference} (100%) rename tests/queries/0_stateless/{02000_bloom_filter_map.sql => 02000_map_full_text_bloom_filter_index.sql} (90%) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index c428d6ffef6..d76f19d6304 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -214,7 +214,7 @@ MergeTreeConditionFullText::MergeTreeConditionFullText( query_info, context, [this] (const ASTPtr & node, ContextPtr /* context */, Block & block_with_constants, RPNElement & out) -> bool { - return this->atomFromAST(node, block_with_constants, out); + return this->traverseAtomAST(node, block_with_constants, out); }).extractRPN()); } @@ -367,187 +367,243 @@ bool MergeTreeConditionFullText::getKey(const std::string & key_column_name, siz return true; } -bool MergeTreeConditionFullText::atomFromAST( - const ASTPtr & node, Block & block_with_constants, RPNElement & out) +bool MergeTreeConditionFullText::traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out) { - Field const_value; - DataTypePtr const_type; - if (const auto * func = typeid_cast(node.get())) { - const ASTs & args = typeid_cast(*func->arguments).children; + Field const_value; + DataTypePtr const_type; - if (args.size() != 2) + if (KeyCondition::getConstant(node, block_with_constants, const_value, const_type)) + { + /// Check constant like in KeyCondition + if (const_value.getType() == Field::Types::UInt64 + || const_value.getType() == Field::Types::Int64 + || const_value.getType() == Field::Types::Float64) + { + /// Zero in all types is represented in memory the same way as in UInt64. + out.function = const_value.get() + ? RPNElement::ALWAYS_TRUE + : RPNElement::ALWAYS_FALSE; + + return true; + } + } + } + + if (const auto * function = node->as()) + { + if (!function->arguments) return false; - size_t key_arg_pos; /// Position of argument with key column (non-const argument) - size_t key_column_num = -1; /// Number of a key column (inside key_column_names array) - const auto & func_name = func->name; + const ASTs & arguments = function->arguments->children; - if (functionIsInOrGlobalInOperator(func_name) && tryPrepareSetBloomFilter(args, out)) + if (arguments.size() != 2) + return false; + + if (functionIsInOrGlobalInOperator(function->name)) { - key_arg_pos = 0; - } - else if (KeyCondition::getConstant(args[1], block_with_constants, const_value, const_type) && getKey(args[0]->getColumnName(), key_column_num)) - { - key_arg_pos = 0; - } - else if (KeyCondition::getConstant(args[0], block_with_constants, const_value, const_type) && getKey(args[1]->getColumnName(), key_column_num)) - { - key_arg_pos = 1; - } - else if (const auto * index_function = args[0].get()->as()) - { - if (index_function->name == "arrayElement") + if (tryPrepareSetBloomFilter(arguments, out)) { - auto column_name = assert_cast(index_function->arguments.get()->children[0].get())->name(); + if (function->name == "notIn") + { + out.function = RPNElement::FUNCTION_NOT_IN; + return true; + } + else if (function->name == "in") + { + out.function = RPNElement::FUNCTION_IN; + return true; + } + } + } + else if (function->name == "equals" || + function->name == "notEquals" || + function->name == "has" || + function->name == "mapContains" || + function->name == "like" || + function->name == "notLike" || + function->name == "hasToken" || + function->name == "startsWith" || + function->name == "endsWith" || + function->name == "multiSearchAny") + { + Field const_value; + DataTypePtr const_type; + if (KeyCondition::getConstant(arguments[1], block_with_constants, const_value, const_type)) + { + if (traverseASTEquals(function->name, arguments[0], const_type, const_value, out)) + return true; + } + else if (KeyCondition::getConstant(arguments[0], block_with_constants, const_value, const_type) && (function->name == "equals" || function->name == "notEquals")) + { + if (traverseASTEquals(function->name, arguments[1], const_type, const_value, out)) + return true; + } + } + } - if (!getKey(column_name, key_column_num)) - return false; + return false; +} - key_arg_pos = 0; +bool MergeTreeConditionFullText::traverseASTEquals( + const String & function_name, + const ASTPtr & key_ast, + const DataTypePtr & value_type, + const Field & value_field, + RPNElement & out) +{ + auto value_data_type = WhichDataType(value_type); + if (!value_data_type.isStringOrFixedString() && !value_data_type.isArray()) + return false; - auto & argument = index_function->arguments.get()->children[1]; + if (!token_extractor->supportLike() && (function_name == "like" || function_name == "notLike")) + return false; + + Field const_value = value_field; + + size_t key_column_num = 0; + bool key_exists = getKey(key_ast->getColumnName(), key_column_num); + bool map_key_exists = getKey(fmt::format("mapKeys({})", key_ast->getColumnName()), key_column_num); + + if (const auto * function = key_ast->as()) + { + if (function->name == "arrayElement") + { + /** 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 + * we return default value for arrayElement. + * + * We cannot skip keys that does not exist in map if comparison is with default type value because + * that way we skip necessary granules where map key does not exists. + */ + if (value_field == value_type->getDefault()) + return false; + + const auto & map_column_name = assert_cast(function->arguments.get()->children[0].get())->name(); + + size_t map_keys_key_column_num = 0; + auto map_keys_index_column_name = fmt::format("mapKeys({})", map_column_name); + bool map_keys_exists = getKey(map_keys_index_column_name, map_keys_key_column_num); + + size_t map_values_key_column_num = 0; + auto map_values_index_column_name = fmt::format("mapValues({})", map_column_name); + bool map_values_exists = getKey(map_values_index_column_name, map_values_key_column_num); + + if (map_keys_exists) + { + auto & argument = function->arguments.get()->children[1]; if (const auto * literal = argument->as()) { - const_value = literal->value; - - if (const_value.getType() != Field::Types::String) - return false; - - const_type = std::make_shared(); + auto element_key = literal->value; + const_value = element_key; + key_column_num = map_keys_key_column_num; + key_exists = true; } else { return false; } } + else if (map_values_exists) + { + key_column_num = map_values_key_column_num; + key_exists = true; + } else { return false; } } - else - { - return false; - } - - if (const_type && const_type->getTypeId() != TypeIndex::String - && const_type->getTypeId() != TypeIndex::FixedString - && const_type->getTypeId() != TypeIndex::Array) - { - return false; - } - - if (key_arg_pos == 1 && (func_name != "equals" && func_name != "notEquals")) - return false; - else if (!token_extractor->supportLike() && (func_name == "like" || func_name == "notLike")) - return false; - - if (func_name == "has") - { - out.key_column = key_column_num; - out.function = RPNElement::FUNCTION_HAS; - out.bloom_filter = std::make_unique(params); - stringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); - - return true; - } - if (func_name == "notEquals") - { - out.key_column = key_column_num; - out.function = RPNElement::FUNCTION_NOT_EQUALS; - out.bloom_filter = std::make_unique(params); - stringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); - return true; - } - else if (func_name == "equals") - { - out.key_column = key_column_num; - return createFunctionEqualsCondition(out, const_value, params, token_extractor); - } - else if (func_name == "like") - { - out.key_column = key_column_num; - out.function = RPNElement::FUNCTION_EQUALS; - out.bloom_filter = std::make_unique(params); - likeStringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); - return true; - } - else if (func_name == "notLike") - { - out.key_column = key_column_num; - out.function = RPNElement::FUNCTION_NOT_EQUALS; - out.bloom_filter = std::make_unique(params); - likeStringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); - return true; - } - else if (func_name == "hasToken") - { - out.key_column = key_column_num; - out.function = RPNElement::FUNCTION_EQUALS; - out.bloom_filter = std::make_unique(params); - stringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); - return true; - } - else if (func_name == "startsWith") - { - out.key_column = key_column_num; - return createFunctionEqualsCondition(out, const_value, params, token_extractor); - } - else if (func_name == "endsWith") - { - out.key_column = key_column_num; - return createFunctionEqualsCondition(out, const_value, params, token_extractor); - } - else if (func_name == "multiSearchAny") - { - out.key_column = key_column_num; - out.function = RPNElement::FUNCTION_MULTI_SEARCH; - - /// 2d vector is not needed here but is used because already exists for FUNCTION_IN - std::vector> bloom_filters; - bloom_filters.emplace_back(); - for (const auto & element : const_value.get()) - { - if (element.getType() != Field::Types::String) - return false; - - bloom_filters.back().emplace_back(params); - stringToBloomFilter(element.get(), token_extractor, bloom_filters.back().back()); - } - out.set_bloom_filters = std::move(bloom_filters); - return true; - } - else if (func_name == "notIn") - { - out.key_column = key_column_num; - out.function = RPNElement::FUNCTION_NOT_IN; - return true; - } - else if (func_name == "in") - { - out.key_column = key_column_num; - out.function = RPNElement::FUNCTION_IN; - return true; - } - - return false; } - else if (KeyCondition::getConstant(node, block_with_constants, const_value, const_type)) - { - /// Check constant like in KeyCondition - if (const_value.getType() == Field::Types::UInt64 - || const_value.getType() == Field::Types::Int64 - || const_value.getType() == Field::Types::Float64) - { - /// Zero in all types is represented in memory the same way as in UInt64. - out.function = const_value.get() - ? RPNElement::ALWAYS_TRUE - : RPNElement::ALWAYS_FALSE; - return true; + if (!key_exists && !map_key_exists) + return false; + + if (map_key_exists && (function_name == "has" || function_name == "mapContains")) + { + out.key_column = key_column_num; + out.function = RPNElement::FUNCTION_HAS; + out.bloom_filter = std::make_unique(params); + stringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); + + return true; + } + else if (function_name == "has") + { + out.key_column = key_column_num; + out.function = RPNElement::FUNCTION_HAS; + out.bloom_filter = std::make_unique(params); + stringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); + + return true; + } + + if (function_name == "notEquals") + { + out.key_column = key_column_num; + out.function = RPNElement::FUNCTION_NOT_EQUALS; + out.bloom_filter = std::make_unique(params); + stringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); + return true; + } + else if (function_name == "equals") + { + out.key_column = key_column_num; + return createFunctionEqualsCondition(out, value_field, params, token_extractor); + } + else if (function_name == "like") + { + out.key_column = key_column_num; + out.function = RPNElement::FUNCTION_EQUALS; + out.bloom_filter = std::make_unique(params); + likeStringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); + return true; + } + else if (function_name == "notLike") + { + out.key_column = key_column_num; + out.function = RPNElement::FUNCTION_NOT_EQUALS; + out.bloom_filter = std::make_unique(params); + likeStringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); + return true; + } + else if (function_name == "hasToken") + { + out.key_column = key_column_num; + out.function = RPNElement::FUNCTION_EQUALS; + out.bloom_filter = std::make_unique(params); + stringToBloomFilter(const_value.get(), token_extractor, *out.bloom_filter); + return true; + } + else if (function_name == "startsWith") + { + out.key_column = key_column_num; + return createFunctionEqualsCondition(out, const_value, params, token_extractor); + } + else if (function_name == "endsWith") + { + out.key_column = key_column_num; + return createFunctionEqualsCondition(out, const_value, params, token_extractor); + } + else if (function_name == "multiSearchAny") + { + out.key_column = key_column_num; + out.function = RPNElement::FUNCTION_MULTI_SEARCH; + + /// 2d vector is not needed here but is used because already exists for FUNCTION_IN + std::vector> bloom_filters; + bloom_filters.emplace_back(); + for (const auto & element : const_value.get()) + { + if (element.getType() != Field::Types::String) + return false; + + bloom_filters.back().emplace_back(params); + stringToBloomFilter(element.get(), token_extractor, bloom_filters.back().back()); } + out.set_bloom_filters = std::move(bloom_filters); + return true; } return false; diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.h b/src/Storages/MergeTree/MergeTreeIndexFullText.h index 80b42420a02..f8e35fd84da 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.h +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.h @@ -146,7 +146,14 @@ private: using RPN = std::vector; - bool atomFromAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out); + bool traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out); + + bool traverseASTEquals( + const String & function_name, + const ASTPtr & key_ast, + const DataTypePtr & value_type, + const Field & value_field, + RPNElement & out); bool getKey(const std::string & key_column_name, size_t & key_column_num); bool tryPrepareSetBloomFilter(const ASTs & args, RPNElement & out); diff --git a/tests/queries/0_stateless/02000_bloom_filter_map.reference b/tests/queries/0_stateless/02000_map_full_text_bloom_filter_index.reference similarity index 100% rename from tests/queries/0_stateless/02000_bloom_filter_map.reference rename to tests/queries/0_stateless/02000_map_full_text_bloom_filter_index.reference diff --git a/tests/queries/0_stateless/02000_bloom_filter_map.sql b/tests/queries/0_stateless/02000_map_full_text_bloom_filter_index.sql similarity index 90% rename from tests/queries/0_stateless/02000_bloom_filter_map.sql rename to tests/queries/0_stateless/02000_map_full_text_bloom_filter_index.sql index 488c369ec85..ca262411eb8 100644 --- a/tests/queries/0_stateless/02000_bloom_filter_map.sql +++ b/tests/queries/0_stateless/02000_map_full_text_bloom_filter_index.sql @@ -5,9 +5,7 @@ CREATE TABLE bf_tokenbf_map_test ( row_id UInt32, map Map(String, String), - map_fixed Map(FixedString(2), String), - INDEX map_tokenbf map TYPE tokenbf_v1(256,2,0) GRANULARITY 1, - INDEX map_fixed_tokenbf map_fixed TYPE tokenbf_v1(256,2,0) GRANULARITY 1 + INDEX map_tokenbf map TYPE tokenbf_v1(256,2,0) GRANULARITY 1 ) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 2; CREATE TABLE bf_ngram_map_test From bd499e5b7dc1c8ff4fbcfff8e957281ce0a1be42 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 27 Sep 2021 17:44:34 +0300 Subject: [PATCH 215/317] 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 216/317] 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 217/317] 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 9a58e4a8baedf9ed8c790c24a8a9f6fb1f02f08a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 27 Sep 2021 19:39:32 +0300 Subject: [PATCH 218/317] fix deadlock in concurrent async inserts and truncates --- src/Common/RWLock.h | 4 ++++ src/Interpreters/AsynchronousInsertQueue.cpp | 16 ++++++++++++---- src/Interpreters/AsynchronousInsertQueue.h | 2 +- .../02015_async_inserts_stress_long.sh | 4 ++-- 4 files changed, 19 insertions(+), 7 deletions(-) diff --git a/src/Common/RWLock.h b/src/Common/RWLock.h index 1e4f9b9bd5e..c0bc01c5460 100644 --- a/src/Common/RWLock.h +++ b/src/Common/RWLock.h @@ -34,6 +34,10 @@ using RWLock = std::shared_ptr; /// - SELECT thread 1 locks in the Read mode /// - ALTER tries to lock in the Write mode (waits for SELECT thread 1) /// - SELECT thread 2 tries to lock in the Read mode (waits for ALTER) +/// +/// NOTE: it is dangerous to acquire lock with NO_QUERY, because FastPath doesn't +/// exist for this case and deadlock, described in previous note, +/// may accur in case of recursive locking. class RWLockImpl : public std::enable_shared_from_this { public: diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index da41eb82d5e..736d35f07e8 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -148,7 +148,7 @@ AsynchronousInsertQueue::~AsynchronousInsertQueue() } } -void AsynchronousInsertQueue::scheduleProcessDataJob(const InsertQuery & key, InsertDataPtr data, ContextPtr global_context) +void AsynchronousInsertQueue::scheduleDataProcessingJob(const InsertQuery & key, InsertDataPtr data, ContextPtr global_context) { /// Wrap 'unique_ptr' with 'shared_ptr' to make this /// lambda copyable and allow to save it to the thread pool. @@ -219,7 +219,7 @@ void AsynchronousInsertQueue::pushImpl(InsertData::EntryPtr entry, QueueIterator data->entries.size(), data->size, queryToString(it->first.query)); if (data->size > max_data_size) - scheduleProcessDataJob(it->first, std::move(data), getContext()); + scheduleDataProcessingJob(it->first, std::move(data), getContext()); } void AsynchronousInsertQueue::waitForProcessingQuery(const String & query_id, const Milliseconds & timeout) @@ -264,7 +264,7 @@ void AsynchronousInsertQueue::busyCheck() auto lag = std::chrono::steady_clock::now() - elem->data->first_update; if (lag >= busy_timeout) - scheduleProcessDataJob(key, std::move(elem->data), getContext()); + scheduleDataProcessingJob(key, std::move(elem->data), getContext()); else timeout = std::min(timeout, std::chrono::ceil(busy_timeout - lag)); } @@ -286,7 +286,7 @@ void AsynchronousInsertQueue::staleCheck() auto lag = std::chrono::steady_clock::now() - elem->data->last_update; if (lag >= stale_timeout) - scheduleProcessDataJob(key, std::move(elem->data), getContext()); + scheduleDataProcessingJob(key, std::move(elem->data), getContext()); } } } @@ -367,6 +367,10 @@ try insert_context->makeQueryContext(); insert_context->setSettings(key.settings); + /// Set initial_query_id, because it's used in InterpreterInsertQuery for table lock. + insert_context->getClientInfo().query_kind = ClientInfo::QueryKind::INITIAL_QUERY; + insert_context->setCurrentQueryId(""); + InterpreterInsertQuery interpreter(key.query, insert_context, key.settings.insert_allow_materialized_columns); auto sinks = interpreter.getSinks(); assert(sinks.size() == 1); @@ -434,6 +438,10 @@ catch (const Exception & e) { finishWithException(key.query, data->entries, e); } +catch (const Poco::Exception & e) +{ + finishWithException(key.query, data->entries, e); +} catch (const std::exception & e) { finishWithException(key.query, data->entries, e); diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index e92a7b04d74..e5361a1fc04 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -134,7 +134,7 @@ private: /// Should be called with shared or exclusively locked 'rwlock'. void pushImpl(InsertData::EntryPtr entry, QueueIterator it); - void scheduleProcessDataJob(const InsertQuery & key, InsertDataPtr data, ContextPtr global_context); + void scheduleDataProcessingJob(const InsertQuery & key, InsertDataPtr data, ContextPtr global_context); static void processData(InsertQuery key, InsertDataPtr data, ContextPtr global_context); template diff --git a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh index f01b3010d9c..f9a58818404 100755 --- a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh +++ b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh @@ -55,7 +55,7 @@ export -f insert1 export -f insert2 export -f select1 export -f select2 -# export -f truncate1 +export -f truncate1 for _ in {1..5}; do timeout $TIMEOUT bash -c insert1 & @@ -64,7 +64,7 @@ done timeout $TIMEOUT bash -c select1 & timeout $TIMEOUT bash -c select2 & -# timeout $TIMEOUT bash -c truncate1 & +timeout $TIMEOUT bash -c truncate1 & wait echo "OK" From 23d899fb604430f92accf7d55ecc9ccd0f4b0f5e Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 27 Sep 2021 19:54:59 +0300 Subject: [PATCH 219/317] 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); From 8be6a59e658d003f4f9d1da73283ec7f4879afc4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 27 Sep 2021 21:08:41 +0300 Subject: [PATCH 220/317] clickhouse-test: fix long tag check for flaky check (--test-runs > 1) Otherwise it will mark the test as failed, like in [1]. [1]: https://clickhouse-test-reports.s3.yandex.net/29369/2320be204397bbb9f9a673cf093b7967fd37a107/functional_stateless_tests_flaky_check_(address).html#fail1 Follow-up for: #28909 (Cc: @vitlibar) --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 7ac434e03cd..7bab99b957f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -465,7 +465,7 @@ class TestCase: description += "\n{}\n".format(diff) return TestResult(self.name, TestStatus.FAIL, FailureReason.RESULT_DIFF, total_time, description) - if self.testcase_args.test_runs > 1 and total_time > 60 and 'long' not in self.name: + if self.testcase_args.test_runs > 1 and total_time > 60 and 'long' not in self.tags: # We're in Flaky Check mode, check the run time as well while we're at it. return TestResult(self.name, TestStatus.FAIL, FailureReason.TOO_LONG, total_time, description) From 985e8ee06102ccbecdd9e706daa768b8b28ef7ef Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 27 Sep 2021 21:10:59 +0300 Subject: [PATCH 221/317] clickhouse-test: fix shared list object (by fixing manager lifetime) Right now it is possible to get the following error: Having 20 errors! 0 tests passed. 0 tests skipped. 57.37 s elapsed (MainProcess). Won't run stateful tests because test data wasn't loaded. Traceback (most recent call last): File "/usr/lib/python3.9/multiprocessing/managers.py", line 802, in _callmethod conn = self._tls.connection AttributeError: 'ForkAwareLocal' object has no attribute 'connection' During handling of the above exception, another exception occurred: Traceback (most recent call last): File "/src/ch/clickhouse/.cmake/../tests/clickhouse-test", line 1462, in main(args) File "/src/ch/clickhouse/.cmake/../tests/clickhouse-test", line 1261, in main if len(restarted_tests) > 0: File "", line 2, in __len__ File "/usr/lib/python3.9/multiprocessing/managers.py", line 806, in _callmethod self._connect() File "/usr/lib/python3.9/multiprocessing/managers.py", line 793, in _connect conn = self._Client(self._token.address, authkey=self._authkey) File "/usr/lib/python3.9/multiprocessing/connection.py", line 507, in Client c = SocketClient(address) File "/usr/lib/python3.9/multiprocessing/connection.py", line 635, in SocketClient s.connect(address) ConnectionRefusedError: [Errno 111] Connection refused The reason behind this is that manager's thread got terminated: ipdb> p restarted_tests._manager._process Refs: #29259 (cc: @vdimir) Follow-up for: #29197 (cc: @tavplubix) --- tests/clickhouse-test | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 7ac434e03cd..d443e9186f8 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -65,11 +65,20 @@ def signal_handler(sig, frame): def stop_tests(): global stop_tests_triggered_lock global stop_tests_triggered + global restarted_tests with stop_tests_triggered_lock: if not stop_tests_triggered.is_set(): stop_tests_triggered.set() + # materialize multiprocessing.Manager().list() object before + # sending SIGTERM since this object is a proxy, that requires + # communicating with manager thread, but after SIGTERM will be + # send, this thread will die, and you will get + # ConnectionRefusedError error for any access to "restarted_tests" + # variable. + restarted_tests = [*restarted_tests] + # send signal to all processes in group to avoid hung check triggering # (to avoid terminating clickhouse-test itself, the signal should be ignored) signal.signal(signal.SIGTERM, signal.SIG_IGN) From 2378491c04d079532919934bc40a1bcce8ce4ccb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 27 Sep 2021 21:04:12 +0300 Subject: [PATCH 222/317] Transform isNull/isNotNull to IS NULL/IS NOT NULL (for external dbs) This is required to support queries (like `foo IS NULL`) to external databases (i.e. MySQL). This patch had been submitted separately, due to: - avoid mixing patches - run CI (and update tests references) --- src/Parsers/ASTFunction.cpp | 33 +++++++++++++++++-- .../00826_cross_to_inner_join.reference | 4 +-- .../01872_functions_to_subcolumns.reference | 4 +-- .../02035_isNull_isNotNull_format.reference | 9 +++++ .../02035_isNull_isNotNull_format.sql | 5 +++ 5 files changed, 49 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02035_isNull_isNotNull_format.reference create mode 100644 tests/queries/0_stateless/02035_isNull_isNotNull_format.sql diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 6a3e9d19484..058cd8daa41 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -217,16 +217,19 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format settings.ostr << nl_or_nothing << indent_str << ")"; return; } + /// Should this function to be written as operator? bool written = false; + if (arguments && !parameters) { + /// Unary prefix operators. if (arguments->children.size() == 1) { const char * operators[] = { - "negate", "-", - "not", "NOT ", + "negate", "-", + "not", "NOT ", nullptr }; @@ -270,6 +273,32 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format } } + /// Unary postfix operators. + if (!written && arguments->children.size() == 1) + { + const char * operators[] = + { + "isNull", " IS NULL", + "isNotNull", " IS NOT NULL", + nullptr + }; + + for (const char ** func = operators; *func; func += 2) + { + if (strcasecmp(name.c_str(), func[0]) != 0) + { + continue; + } + + arguments->formatImpl(settings, state, nested_need_parens); + settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : ""); + + written = true; + + break; + } + } + /** need_parens - do we need parentheses around the expression with the operator. * They are needed only if this expression is included in another expression with the operator. */ diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.reference b/tests/queries/0_stateless/00826_cross_to_inner_join.reference index 973c5b078a3..c152a92fb83 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.reference +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.reference @@ -146,7 +146,7 @@ SELECT t2_00826.b FROM t1_00826 ALL INNER JOIN t2_00826 ON b = t2_00826.a -WHERE (b = t2_00826.a) AND (isNull(t2_00826.b) OR (t2_00826.b > t2_00826.a)) +WHERE (b = t2_00826.a) AND (t2_00826.b IS NULL OR (t2_00826.b > t2_00826.a)) --- do not rewrite alias --- SELECT a AS b FROM t1_00826 @@ -178,4 +178,4 @@ SELECT t2_00826.b FROM t1_00826 ALL INNER JOIN t2_00826 ON a = t2_00826.a -WHERE (a = t2_00826.a) AND (isNull(t2_00826.b) OR (t2_00826.b < 2)) +WHERE (a = t2_00826.a) AND (t2_00826.b IS NULL OR (t2_00826.b < 2)) diff --git a/tests/queries/0_stateless/01872_functions_to_subcolumns.reference b/tests/queries/0_stateless/01872_functions_to_subcolumns.reference index 8c5ca4df602..a1cd31e2dc9 100644 --- a/tests/queries/0_stateless/01872_functions_to_subcolumns.reference +++ b/tests/queries/0_stateless/01872_functions_to_subcolumns.reference @@ -1,7 +1,7 @@ 0 0 1 0 1 0 SELECT - isNull(id), + id IS NULL, `n.null`, NOT `n.null` FROM t_func_to_subcolumns @@ -31,7 +31,7 @@ FROM t_func_to_subcolumns SELECT id, `n.null`, - isNull(right.n) + right.n IS NULL FROM t_func_to_subcolumns AS left ALL FULL OUTER JOIN ( diff --git a/tests/queries/0_stateless/02035_isNull_isNotNull_format.reference b/tests/queries/0_stateless/02035_isNull_isNotNull_format.reference new file mode 100644 index 00000000000..74fc4a44b5f --- /dev/null +++ b/tests/queries/0_stateless/02035_isNull_isNotNull_format.reference @@ -0,0 +1,9 @@ +-- { echo } +explain syntax select null is null; +SELECT NULL IS NULL +explain syntax select null is not null; +SELECT NULL IS NOT NULL +explain syntax select isNull(null); +SELECT NULL IS NULL +explain syntax select isNotNull(null); +SELECT NULL IS NOT NULL diff --git a/tests/queries/0_stateless/02035_isNull_isNotNull_format.sql b/tests/queries/0_stateless/02035_isNull_isNotNull_format.sql new file mode 100644 index 00000000000..eff658445a8 --- /dev/null +++ b/tests/queries/0_stateless/02035_isNull_isNotNull_format.sql @@ -0,0 +1,5 @@ +-- { echo } +explain syntax select null is null; +explain syntax select null is not null; +explain syntax select isNull(null); +explain syntax select isNotNull(null); From dcde7fc9453d9b407b68ef30f9c95f9aa30fe5ab Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 25 Sep 2021 20:50:14 +0300 Subject: [PATCH 223/317] Enable memory profiler in tests This is to catch possible allocations for mutations, for 01200_mutations_memory_consumption --- tests/config/install.sh | 1 + tests/config/users.d/memory_profiler.xml | 8 ++++++++ 2 files changed, 9 insertions(+) create mode 100644 tests/config/users.d/memory_profiler.xml diff --git a/tests/config/install.sh b/tests/config/install.sh index df62cba0ea9..05db883c691 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -46,6 +46,7 @@ ln -sf $SRC_PATH/users.d/database_atomic_drop_detach_sync.xml $DEST_SERVER_PATH/ ln -sf $SRC_PATH/users.d/opentelemetry.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/remote_queries.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), diff --git a/tests/config/users.d/memory_profiler.xml b/tests/config/users.d/memory_profiler.xml new file mode 100644 index 00000000000..56e9ba28347 --- /dev/null +++ b/tests/config/users.d/memory_profiler.xml @@ -0,0 +1,8 @@ + + + + 1Mi + 1Mi + + + From e44ade9281f5dc756d18686bde6d8699567edd21 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 25 Sep 2021 22:12:42 +0300 Subject: [PATCH 224/317] Update test references in 01293_show_settings --- tests/queries/0_stateless/01293_show_settings.reference | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01293_show_settings.reference b/tests/queries/0_stateless/01293_show_settings.reference index 6fcbf194614..aa27ef83f52 100644 --- a/tests/queries/0_stateless/01293_show_settings.reference +++ b/tests/queries/0_stateless/01293_show_settings.reference @@ -3,3 +3,5 @@ connect_timeout Seconds 10 connect_timeout_with_failover_ms Milliseconds 2000 connect_timeout_with_failover_secure_ms Milliseconds 3000 max_memory_usage UInt64 10000000000 +max_untracked_memory UInt64 1048576 +memory_profiler_step UInt64 1048576 From 9eb400bc1627fb614f1d942be682122654b00ef0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 25 Sep 2021 22:14:19 +0300 Subject: [PATCH 225/317] Fix 01656_test_query_log_factories_info flakiness --- .../queries/0_stateless/01656_test_query_log_factories_info.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01656_test_query_log_factories_info.sql b/tests/queries/0_stateless/01656_test_query_log_factories_info.sql index 7205d547df9..020d7cc5e72 100644 --- a/tests/queries/0_stateless/01656_test_query_log_factories_info.sql +++ b/tests/queries/0_stateless/01656_test_query_log_factories_info.sql @@ -17,7 +17,7 @@ SELECT uniqArray([1, 1, 2]), toDayOfYear(d) % 2) FROM numbers(100); -SELECT repeat('a', number) +SELECT repeat('aa', number) FROM numbers(10e3) SETTINGS max_memory_usage=4e6, max_block_size=100 FORMAT Null; -- { serverError 241 } From a218c3a28c769302acbe1b82568665eb7a012b16 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 27 Sep 2021 22:41:57 +0300 Subject: [PATCH 226/317] Enable per-query memory profiler (set to memory_profiler_step=4MiB) globally Let's see at performance tests. And yes, they should show the difference, if any, since they uses clickhouse python driver, which does not knows about default settings, so new server will have this setting set, while old server - upstream/master, will not. --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9dfa4d911b5..e5c3622ef0e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -350,7 +350,7 @@ class IColumn; M(UInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \ M(UInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \ M(UInt64, max_untracked_memory, (4 * 1024 * 1024), "Small allocations and deallocations are grouped in thread local variable and tracked or profiled only when amount (in absolute value) becomes larger than specified value. If the value is higher than 'memory_profiler_step' it will be effectively lowered to 'memory_profiler_step'.", 0) \ - M(UInt64, memory_profiler_step, 0, "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ + M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ \ M(UInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \ From cc40e36514ea395e4367a39c375e5dbc225baff1 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 27 Sep 2021 22:39:50 +0300 Subject: [PATCH 227/317] Fix logical error in function greatest/least --- src/Functions/LeastGreatestGeneric.h | 2 ++ .../0_stateless/01822_short_circuit.sql | 1 + ...short_circuit_least_greatest_bug.reference | 20 +++++++++++++++++++ ...02032_short_circuit_least_greatest_bug.sql | 2 ++ 4 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/02032_short_circuit_least_greatest_bug.reference create mode 100644 tests/queries/0_stateless/02032_short_circuit_least_greatest_bug.sql diff --git a/src/Functions/LeastGreatestGeneric.h b/src/Functions/LeastGreatestGeneric.h index a8bab0efd54..df44ff87762 100644 --- a/src/Functions/LeastGreatestGeneric.h +++ b/src/Functions/LeastGreatestGeneric.h @@ -107,6 +107,8 @@ public: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override { DataTypes argument_types; + for (const auto & argument : arguments) + argument_types.push_back(argument.type); /// More efficient specialization for two numeric arguments. if (arguments.size() == 2 && isNumber(arguments[0].type) && isNumber(arguments[1].type)) diff --git a/tests/queries/0_stateless/01822_short_circuit.sql b/tests/queries/0_stateless/01822_short_circuit.sql index 1f0e04cb4b5..48fff04921b 100644 --- a/tests/queries/0_stateless/01822_short_circuit.sql +++ b/tests/queries/0_stateless/01822_short_circuit.sql @@ -153,3 +153,4 @@ select number % 2 and toLowCardinality(number) from numbers(5); select number % 2 or toLowCardinality(number) from numbers(5); select if(toLowCardinality(number) % 2, number, number + 1) from numbers(10); select multiIf(toLowCardinality(number) % 2, number, number + 1) from numbers(10); + diff --git a/tests/queries/0_stateless/02032_short_circuit_least_greatest_bug.reference b/tests/queries/0_stateless/02032_short_circuit_least_greatest_bug.reference new file mode 100644 index 00000000000..57c88cc489e --- /dev/null +++ b/tests/queries/0_stateless/02032_short_circuit_least_greatest_bug.reference @@ -0,0 +1,20 @@ +0 +1 +1 +1 +1 +1 +0 +1 +1 +1 +0 +1 +0 +0 +0 +1 +0 +1 +0 +0 diff --git a/tests/queries/0_stateless/02032_short_circuit_least_greatest_bug.sql b/tests/queries/0_stateless/02032_short_circuit_least_greatest_bug.sql new file mode 100644 index 00000000000..e7dca0bde91 --- /dev/null +++ b/tests/queries/0_stateless/02032_short_circuit_least_greatest_bug.sql @@ -0,0 +1,2 @@ +select 1 and greatest(number % 2, number % 3) from numbers(10); +select 1 and least(number % 2, number % 3) from numbers(10); From af1fbb1d6eceab361245e13b0f80f0892455c3b1 Mon Sep 17 00:00:00 2001 From: Alexey Date: Mon, 27 Sep 2021 20:15:15 +0000 Subject: [PATCH 228/317] en fixes --- docs/en/sql-reference/functions/index.md | 41 ++++++++++++------------ 1 file changed, 21 insertions(+), 20 deletions(-) diff --git a/docs/en/sql-reference/functions/index.md b/docs/en/sql-reference/functions/index.md index 3069a596721..d1958333a81 100644 --- a/docs/en/sql-reference/functions/index.md +++ b/docs/en/sql-reference/functions/index.md @@ -64,11 +64,11 @@ For some functions the first argument (the lambda function) can be omitted. In t Custom functions from lambda expressions can be created using the [CREATE FUNCTION](../statements/create/function.md) statement. To delete these functions use the [DROP FUNCTION](../statements/drop.md#drop-function) statement. ## Executable User Defined Functions {#executable-user-defined-functions} -ClickHouse can call any external executable program or script to process data. Describe such functions in a [configuration file](../../operations/configuration-files.md) and add the path of that file to the main configuration in `user_defined_executable_functions_config` setting. If a wildcard symbol `*` is used in the path, then all files matching a pattern are loaded. Example: +ClickHouse can call any external executable program or script to process data. Describe such functions in a [configuration file](../../operations/configuration-files.md) and add the path of that file to the main configuration in `user_defined_executable_functions_config` setting. If a wildcard symbol `*` is used in the path, then all files matching the pattern are loaded. Example: ``` xml *_function.xml ``` -User defined function configurations are searched relative to a path specified in the `user_files_path` setting. +User defined function configurations are searched relative to the path specified in the `user_files_path` setting. A function configuration contains the following settings: @@ -77,47 +77,48 @@ A function configuration contains the following settings: - `argument` - argument description with the `type` of an argument. Each argument is described in a separate setting. - `format` - a [format](../../interfaces/formats.md) in which arguments are passed to the command. - `return_type` - the type of a returned value. -- `max_command_execution_time` - the maximum number of seconds the function is allowed to process arguments. Optional. Default value is `10`. -- `command_termination_timeout` - ??? Optional. Default value is `10`. - `type` - an executable type. If `type` is set to `executable` then single command is started. If it is set to `executable_pool` then a pool of commands is created. -- `pool_size` - a size of the command pool. Optional. Default value is `16`. -- `lifetime` - reload interval of the function in seconds. If it is set to `0` then function is not reloaded. -- `send_chunk_header` - ??? Optional. Default value is `false`. +- `max_command_execution_time` - maximum execution time in seconds for processing block of data. This setting is valid for `executable_pool` fuctions only. Optional. Default value is `10`. +- `command_termination_timeout` - time in seconds during which a command should finish after its pipe is closed. After that time SIGTERM is sent to the process executing the command. This setting is valid for `executable_pool` fuctions only. Optional. Default value is `10`. +- `pool_size` - the size of a command pool. Optional. Default value is `16`. +- `lifetime` - the reload interval of a function in seconds. If it is set to `0` then the function is not reloaded. +- `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. -The command must read arguments from STDIN and must output the result to STDOUT. The command must process arguments iteratively. That is after processing a set of arguments it must wait for the next set of arguments. +The command must read arguments from STDIN and must output the result to STDOUT. The command must process arguments iteratively. That is after processing a chunk of arguments it must wait for the next chunk. **Example** -The following example creates `my_function`. It gets single argument of type String. `xargs` command listens to STDIN and calls `echo` for every argument. +The following example creates `test_function` using XML configuration. ``` - my_function - xargs -I arg echo Processing arg + executable + test_function + UInt64 - String + UInt64 + + + UInt64 TabSeparated - String - executable + cd /; clickhouse-local --input-format TabSeparated --output-format TabSeparated --structure 'x UInt64, y UInt64' --query "SELECT x + y FROM table" 0 ``` Query: -`my_function` is available in queries. ``` sql -SELECT number, my_function(toString(number)) FROM numbers(2); +SELECT test_function(toUInt64(2), toUInt64(2)); ``` Result: ``` text -┌─number─┬─my_function(toString(number))─┐ -│ 0 │ Processing 0 │ -│ 1 │ Processing 1 │ -└────────┴───────────────────────────────┘ +┌─test_function(toUInt64(2), toUInt64(2))─┐ +│ 4 │ +└─────────────────────────────────────────┘ ``` From a30537ca9829d63ae1fc0180c672daf54e90740c Mon Sep 17 00:00:00 2001 From: Alexey Date: Mon, 27 Sep 2021 20:15:39 +0000 Subject: [PATCH 229/317] ru translation --- docs/ru/sql-reference/functions/index.md | 63 +++++++++++++++++++++++- 1 file changed, 61 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/index.md b/docs/ru/sql-reference/functions/index.md index 92bd1c1c2f8..caee4e5c672 100644 --- a/docs/ru/sql-reference/functions/index.md +++ b/docs/ru/sql-reference/functions/index.md @@ -58,9 +58,68 @@ str -> str != Referer Для некоторых функций первый аргумент (лямбда-функция) может отсутствовать. В этом случае подразумевается тождественное отображение. -## Пользовательские функции {#user-defined-functions} +## Пользовательские функции SQL {#user-defined-functions} -Функции можно создавать с помощью выражения [CREATE FUNCTION](../statements/create/function.md). Для удаления таких функций используется выражение [DROP FUNCTION](../statements/drop.md#drop-function). +Функции можно создавать из лямбда выражений с помощью [CREATE FUNCTION](../statements/create/function.md). Для удаления таких функций используется выражение [DROP FUNCTION](../statements/drop.md#drop-function). + +## Исполняемые пользовательские функции {#executable-user-defined-functions} +ClickHouse может вызывать внешнюю программу или скрипт для обработки данных. Такие функции описываются в [конфигурационном файле](../../operations/configuration-files.md). А путь к нему должен быть указан в настройке `user_defined_executable_functions_config` в основной конфигурации. В пути можно использовать символ подстановки `*`, тогда будут загружены все файлы, соответствующие шаблону. Пример: +``` xml +*_function.xml +``` +Файлы с описанием функций ищутся относительно каталога, заданного в настройке `user_files_path`. + +Конфигурация функции содержит следующие нстройки: + +- `name` - имя функции. +- `command` - исполняемая команда или скрипт. +- `argument` - описание аргумента, содержащее его тип во вложенной настройке `type`. Каждый аргумент описывается отдельно. +- `format` - [формат](../../interfaces/formats.md), в котором аргументы передаются команде. +- `return_type` - тип возвращаемого значения. +- `type` - вариант запуска команды. Если задан вариант `executable`, то запускается одна команда. При указании `executable_pool` создается пул команд. +- `max_command_execution_time` - максимальное время в секундах, отводимое на обработку блока данных. Эта настройка применима только для команд с вариантом запуска `executable_pool`. Необязательная настройка. Значение по умолчанию `10`. +- `command_termination_timeout` - максимальное время завершения команды в секундах после закрытия конвейера. Если команда не завершается, то процессу отправляется сигнал SIGTERM. Эта настройка применима только для команд с вариантом запуска `executable_pool`. Необязательная настройка. Значение по умолчанию `10`. +- `pool_size` - размер пула команд. Необязательная настройка. Значение по умолчанию `16`. +- `lifetime` - интервал перезагрузки функций в секундах. Если задан `0`, то функция не перезагружается. +- `send_chunk_header` - управляет отправкой количества строк перед отправкой блока данных для обработки. Необязательная настройка. Значение по умолчанию `false`. + +Команд должна читать аргументы из STDIN и выводить результат в STDOUT. Обработка должна выполняться в цикле. То есть после обработки группы аргументов команда должна ожидать следующую группу. + +**Пример** + +XML конфигурация описывает функцию `test_function`. +``` + + + executable + test_function + UInt64 + + UInt64 + + + UInt64 + + TabSeparated + cd /; clickhouse-local --input-format TabSeparated --output-format TabSeparated --structure 'x UInt64, y UInt64' --query "SELECT x + y FROM table" + 0 + + +``` + +Запрос: + +``` sql +SELECT test_function(toUInt64(2), toUInt64(2)); +``` + +Результат: + +``` text +┌─test_function(toUInt64(2), toUInt64(2))─┐ +│ 4 │ +└─────────────────────────────────────────┘ +``` ## Обработка ошибок {#obrabotka-oshibok} From 8f576096ad0f9aa76e7d8b4d0623a5d9e4e726e4 Mon Sep 17 00:00:00 2001 From: Alexey Date: Mon, 27 Sep 2021 20:24:12 +0000 Subject: [PATCH 230/317] ru translation --- docs/en/sql-reference/statements/system.md | 2 +- docs/ru/sql-reference/statements/grant.md | 2 ++ docs/ru/sql-reference/statements/system.md | 13 +++++++++++++ 3 files changed, 16 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index bbe90013d11..2009d23ec54 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -87,7 +87,7 @@ SYSTEM RELOAD MODEL ## RELOAD FUNCTIONS {#query_language-system-reload-functions} -Reloads all registered [executable user defined functions](../functions/index.md#executable-user-defined-functions) or one of them from an XML configuration. +Reloads all registered [executable user defined functions](../functions/index.md#executable-user-defined-functions) or one of them from a configuration file. **Syntax** diff --git a/docs/ru/sql-reference/statements/grant.md b/docs/ru/sql-reference/statements/grant.md index 45ba9bb0343..c970d4d24f3 100644 --- a/docs/ru/sql-reference/statements/grant.md +++ b/docs/ru/sql-reference/statements/grant.md @@ -157,6 +157,8 @@ GRANT SELECT(x,y) ON db.table TO john WITH GRANT OPTION - `SYSTEM RELOAD CONFIG` - `SYSTEM RELOAD DICTIONARY` - `SYSTEM RELOAD EMBEDDED DICTIONARIES` + - `SYSTEM RELOAD FUNCTION` + - `SYSTEM RELOAD FUNCTIONS` - `SYSTEM MERGES` - `SYSTEM TTL MERGES` - `SYSTEM FETCHES` diff --git a/docs/ru/sql-reference/statements/system.md b/docs/ru/sql-reference/statements/system.md index e123f506d46..40f503937cf 100644 --- a/docs/ru/sql-reference/statements/system.md +++ b/docs/ru/sql-reference/statements/system.md @@ -10,6 +10,8 @@ toc_title: SYSTEM - [RELOAD DICTIONARY](#query_language-system-reload-dictionary) - [RELOAD MODELS](#query_language-system-reload-models) - [RELOAD MODEL](#query_language-system-reload-model) +- [RELOAD FUNCTIONS](#query_language-system-reload-functions) +- [RELOAD FUNCTION](#query_language-system-reload-functions) - [DROP DNS CACHE](#query_language-system-drop-dns-cache) - [DROP MARK CACHE](#query_language-system-drop-mark-cache) - [DROP UNCOMPRESSED CACHE](#query_language-system-drop-uncompressed-cache) @@ -80,6 +82,17 @@ SYSTEM RELOAD MODELS SYSTEM RELOAD MODEL ``` +## RELOAD FUNCTIONS {#query_language-system-reload-functions} + +Перезагружает все зарегистрированные [исполняемые пользовательские функции](../functions/index.md#executable-user-defined-functions) или одну из них из файла конфигурации. + +**Синтаксис** + +```sql +RELOAD FUNCTIONS +RELOAD FUNCTION function_name + + ## DROP DNS CACHE {#query_language-system-drop-dns-cache} Сбрасывает внутренний DNS кеш ClickHouse. Иногда (для старых версий ClickHouse) необходимо использовать эту команду при изменении инфраструктуры (смене IP адреса у другого ClickHouse сервера или сервера, используемого словарями). From c9aab06ab83539a455577783bf33888224f608bf Mon Sep 17 00:00:00 2001 From: Alexey Date: Mon, 27 Sep 2021 20:27:11 +0000 Subject: [PATCH 231/317] markup fixed --- docs/ru/sql-reference/statements/system.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/system.md b/docs/ru/sql-reference/statements/system.md index 40f503937cf..0eb0a5326f2 100644 --- a/docs/ru/sql-reference/statements/system.md +++ b/docs/ru/sql-reference/statements/system.md @@ -91,7 +91,7 @@ SYSTEM RELOAD MODEL ```sql RELOAD FUNCTIONS RELOAD FUNCTION function_name - +``` ## DROP DNS CACHE {#query_language-system-drop-dns-cache} From eeea3caf4988c171fbbd773fcc442340c17c71ab Mon Sep 17 00:00:00 2001 From: Alexey Date: Mon, 27 Sep 2021 20:32:38 +0000 Subject: [PATCH 232/317] typos --- docs/en/sql-reference/functions/index.md | 4 ++-- docs/ru/sql-reference/functions/index.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/index.md b/docs/en/sql-reference/functions/index.md index d1958333a81..0749fd9e2d7 100644 --- a/docs/en/sql-reference/functions/index.md +++ b/docs/en/sql-reference/functions/index.md @@ -78,8 +78,8 @@ A function configuration contains the following settings: - `format` - a [format](../../interfaces/formats.md) in which arguments are passed to the command. - `return_type` - the type of a returned value. - `type` - an executable type. If `type` is set to `executable` then single command is started. If it is set to `executable_pool` then a pool of commands is created. -- `max_command_execution_time` - maximum execution time in seconds for processing block of data. This setting is valid for `executable_pool` fuctions only. Optional. Default value is `10`. -- `command_termination_timeout` - time in seconds during which a command should finish after its pipe is closed. After that time SIGTERM is sent to the process executing the command. This setting is valid for `executable_pool` fuctions only. Optional. Default value is `10`. +- `max_command_execution_time` - maximum execution time in seconds for processing block of data. This setting is valid for `executable_pool` commands only. Optional. Default value is `10`. +- `command_termination_timeout` - time in seconds during which a command should finish after its pipe is closed. After that time SIGTERM is sent to the process executing the command. This setting is valid for `executable_pool` commands only. Optional. Default value is `10`. - `pool_size` - the size of a command pool. Optional. Default value is `16`. - `lifetime` - the reload interval of a function in seconds. If it is set to `0` then the function is not reloaded. - `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. diff --git a/docs/ru/sql-reference/functions/index.md b/docs/ru/sql-reference/functions/index.md index caee4e5c672..49af0e14dd2 100644 --- a/docs/ru/sql-reference/functions/index.md +++ b/docs/ru/sql-reference/functions/index.md @@ -69,7 +69,7 @@ ClickHouse может вызывать внешнюю программу или ``` Файлы с описанием функций ищутся относительно каталога, заданного в настройке `user_files_path`. -Конфигурация функции содержит следующие нстройки: +Конфигурация функции содержит следующие настройки: - `name` - имя функции. - `command` - исполняемая команда или скрипт. From 108b76e650e324b6d000949ed57e0c334e52202e Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Mon, 27 Sep 2021 20:42:11 -0300 Subject: [PATCH 233/317] added tests for tuples --- ...llable_key_and_index_negate_cond.reference | 13 ++++++++++ ...410_nullable_key_and_index_negate_cond.sql | 24 +++++++++++++++++++ 2 files changed, 37 insertions(+) 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 index f735f1378b6..9bd4af1986f 100644 --- 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 @@ -1,3 +1,16 @@ +---Q1--- s s s +---Q2--- s s s +---Q3--- s s s +---Q4--- +s s s +\N s1 s1 +\N \N s2 +---Q5--- +---Q6--- +---Q7--- +---Q8--- +s s s +---Q9--- 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 index 3d1e3177173..0a1e3464596 100644 --- 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 @@ -7,10 +7,34 @@ 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 '---Q1---'; select * from test_23634 where id !=''; +select '---Q2---'; select * from test_23634 where id !='' and s != ''; +select '---Q3---'; select * from test_23634 where id !='' and s != '' and s1 != ''; +set force_primary_key=0; + +select '---Q4---'; +select * from test_23634 where (id, s, s1) != ('', '', '') order by id, s1, s1; + +select '---Q5---'; +select * from test_23634 where (id, s, s1) = ('', '', '') order by id, s1, s1; + +select '---Q6---'; +select * from test_23634 where (id, s, s1) = ('', '', 's2') order by id, s1, s1; + +select '---Q7---'; +select * from test_23634 where (id, s, s1) = ('', 's1', 's1') order by id, s1, s1; + +select '---Q8---'; +select * from test_23634 where (id, s, s1) = ('s', 's', 's') order by id, s1, s1; + +select '---Q9---'; +select * from test_23634 where (id, s, s1) = (null::Nullable(String), null::Nullable(String), null::Nullable(String)) order by id, s1, s1; + drop table test_23634; + From b25c3fb659c466b272076260c8e091e9a0209c55 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 10 Sep 2021 18:27:58 +0800 Subject: [PATCH 234/317] Handle null column correctly in JSONFunction --- src/Functions/FunctionsJSON.cpp | 30 ++-- src/Functions/FunctionsJSON.h | 140 +++++++++++++++--- src/Functions/IFunction.h | 11 +- .../02013_json_function_null_column.reference | 1 + .../02013_json_function_null_column.sql | 8 + 5 files changed, 153 insertions(+), 37 deletions(-) create mode 100644 tests/queries/0_stateless/02013_json_function_null_column.reference create mode 100644 tests/queries/0_stateless/02013_json_function_null_column.sql diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index b220a59f8eb..47f485e9d6c 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -58,21 +58,21 @@ size_t FunctionJSONHelpers::calculateMaxSize(const ColumnString::Offsets & offse void registerFunctionsJSON(FunctionFactory & factory) { - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); - factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); + factory.registerFunction>(); } } diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 68a29615fa8..4e0b8f291ca 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -26,9 +26,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -276,37 +278,139 @@ private: template typename Impl> -class FunctionJSON : public IFunction, WithContext +class ExecutableFunctionJSON : public IExecutableFunction, WithContext { + public: - static FunctionPtr create(ContextPtr context_) { return std::make_shared(context_); } - FunctionJSON(ContextPtr context_) : WithContext(context_) {} - - static constexpr auto name = Name::name; - String getName() const override { return Name::name; } - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - bool useDefaultImplementationForConstants() const override { return true; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + explicit ExecutableFunctionJSON(const NullPresence & null_presence_, bool allow_simdjson_) + : null_presence(null_presence_), allow_simdjson(allow_simdjson_) { - return Impl::getReturnType(Name::name, arguments); } + String getName() const override { return Name::name; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { + if (null_presence.has_null_constant) + return result_type->createColumnConstWithDefaultValue(input_rows_count); + + ColumnsWithTypeAndName temporary_columns; + if (null_presence.has_nullable) + temporary_columns = createBlockWithNestedColumns(arguments); + else + temporary_columns = arguments; + + ColumnPtr temporary_result; + /// Choose JSONParser. #if USE_SIMDJSON - if (getContext()->getSettingsRef().allow_simdjson) - return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); + if (allow_simdjson) + temporary_result = FunctionJSONHelpers::Executor::run( + temporary_columns, result_type, input_rows_count); + else #endif - + { #if USE_RAPIDJSON - return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); + temporary_result = FunctionJSONHelpers::Executor::run( + temporary_columns, result_type, input_rows_count); #else - return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); + temporary_result + = FunctionJSONHelpers::Executor::run(temporary_columns, result_type, input_rows_count); #endif + } + + if (null_presence.has_nullable) + return wrapInNullable(temporary_result, arguments, result_type, input_rows_count); + return temporary_result; + } + +private: + NullPresence null_presence; + bool allow_simdjson; +}; + + +template typename Impl> +class FunctionBaseFunctionJSON : public IFunctionBase +{ +public: + explicit FunctionBaseFunctionJSON( + const NullPresence & null_presence_, bool allow_simdjson_, DataTypes argument_types_, DataTypePtr return_type_) + : null_presence(null_presence_) + , allow_simdjson(allow_simdjson_) + , argument_types(std::move(argument_types_)) + , return_type(std::move(return_type_)) + { + } + + String getName() const override { return Name::name; } + + const DataTypes & getArgumentTypes() const override + { + return argument_types; + } + + const DataTypePtr & getResultType() const override + { + return return_type; + } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override + { + return std::make_unique>(null_presence, allow_simdjson); + } + +private: + NullPresence null_presence; + bool allow_simdjson; + DataTypes argument_types; + DataTypePtr return_type; +}; + + +template typename Impl> +class JSONOverloadResolver : public IFunctionOverloadResolver, WithContext +{ +public: + static constexpr auto name = Name::name; + + String getName() const override { return name; } + + static FunctionOverloadResolverPtr create(ContextPtr context_) + { + return std::make_unique(context_); + } + + explicit JSONOverloadResolver(ContextPtr context_) : WithContext(context_) {} + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + // Both NULL and JSON NULL should generate NULL value. + // If any argument is NULL, return NULL. + bool useDefaultImplementationForNulls() const override { return false; } + + FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const override + { + NullPresence null_presence = getNullPresense(arguments); + DataTypePtr return_type; + if (null_presence.has_null_constant) + return_type = makeNullable(std::make_shared()); + else if (null_presence.has_nullable) + return_type = makeNullable(Impl::getReturnType(Name::name, createBlockWithNestedColumns(arguments))); + else + return_type = Impl::getReturnType(Name::name, arguments); + + DataTypes argument_types; + argument_types.reserve(arguments.size()); + for (const auto & argument : arguments) + argument_types.emplace_back(argument.type); + return std::make_unique>( + null_presence, getContext()->getSettingsRef().allow_simdjson, argument_types, return_type); } }; diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index db8a449f2b1..db09351a92e 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -278,9 +278,7 @@ class IFunctionOverloadResolver public: virtual ~IFunctionOverloadResolver() = default; - FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const; - - DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const; + virtual FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const; void getLambdaArgumentTypes(DataTypes & arguments) const; @@ -322,7 +320,10 @@ public: protected: - virtual FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const = 0; + virtual FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & /* arguments */, const DataTypePtr & /* result_type */) const + { + throw Exception("buildImpl is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } virtual DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const { @@ -360,6 +361,8 @@ protected: private: + DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const; + DataTypePtr getReturnTypeWithoutLowCardinality(const ColumnsWithTypeAndName & arguments) const; }; diff --git a/tests/queries/0_stateless/02013_json_function_null_column.reference b/tests/queries/0_stateless/02013_json_function_null_column.reference new file mode 100644 index 00000000000..fdf3cea38fd --- /dev/null +++ b/tests/queries/0_stateless/02013_json_function_null_column.reference @@ -0,0 +1 @@ +\N \N \N diff --git a/tests/queries/0_stateless/02013_json_function_null_column.sql b/tests/queries/0_stateless/02013_json_function_null_column.sql new file mode 100644 index 00000000000..bb2eed84b8f --- /dev/null +++ b/tests/queries/0_stateless/02013_json_function_null_column.sql @@ -0,0 +1,8 @@ +drop table if exists test_table; + +create table test_table (col String, col_nullable Nullable(String)) engine MergeTree order by col; +insert into test_table select '{"string_value":null}' as col, '{"string_value":null}' as col_nullable; + +select JSONExtract(col, 'string_value', 'Nullable(String)') as res1, JSONExtract(col_nullable, 'string_value', 'Nullable(String)') as res2, JSONExtract(assumeNotNull(col_nullable), 'string_value', 'Nullable(String)') as res3 from test_table; + +drop table test_table; From 540956112d1da74b89dffdb7b9d3ced72e19501d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 24 Sep 2021 01:20:59 +0800 Subject: [PATCH 235/317] skip fasttest --- tests/queries/0_stateless/02013_json_function_null_column.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02013_json_function_null_column.sql b/tests/queries/0_stateless/02013_json_function_null_column.sql index bb2eed84b8f..58005a05647 100644 --- a/tests/queries/0_stateless/02013_json_function_null_column.sql +++ b/tests/queries/0_stateless/02013_json_function_null_column.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + drop table if exists test_table; create table test_table (col String, col_nullable Nullable(String)) engine MergeTree order by col; From 07a278e4266083de23e1ba0fd6b7143e68720399 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 28 Sep 2021 11:38:08 +0800 Subject: [PATCH 236/317] Better --- src/Functions/FunctionsJSON.h | 48 ++++++++----------- .../02013_json_function_null_column.reference | 7 ++- .../02013_json_function_null_column.sql | 16 +++---- 3 files changed, 32 insertions(+), 39 deletions(-) diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index 4e0b8f291ca..b3b34c9b352 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -296,37 +296,30 @@ public: if (null_presence.has_null_constant) return result_type->createColumnConstWithDefaultValue(input_rows_count); - ColumnsWithTypeAndName temporary_columns; - if (null_presence.has_nullable) - temporary_columns = createBlockWithNestedColumns(arguments); - else - temporary_columns = arguments; - - ColumnPtr temporary_result; - - /// Choose JSONParser. -#if USE_SIMDJSON - if (allow_simdjson) - temporary_result = FunctionJSONHelpers::Executor::run( - temporary_columns, result_type, input_rows_count); - else -#endif - { -#if USE_RAPIDJSON - temporary_result = FunctionJSONHelpers::Executor::run( - temporary_columns, result_type, input_rows_count); -#else - temporary_result - = FunctionJSONHelpers::Executor::run(temporary_columns, result_type, input_rows_count); -#endif - } - + ColumnsWithTypeAndName temporary_columns = null_presence.has_nullable ? createBlockWithNestedColumns(arguments) : arguments; + ColumnPtr temporary_result = chooseAndRunJSONParser(temporary_columns, result_type, input_rows_count); if (null_presence.has_nullable) return wrapInNullable(temporary_result, arguments, result_type, input_rows_count); return temporary_result; } private: + + ColumnPtr + chooseAndRunJSONParser(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const + { +#if USE_SIMDJSON + if (allow_simdjson) + return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); +#endif + +#if USE_RAPIDJSON + return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); +#else + return FunctionJSONHelpers::Executor::run(arguments, result_type, input_rows_count); +#endif + } + NullPresence null_presence; bool allow_simdjson; }; @@ -372,6 +365,8 @@ private: }; +/// We use IFunctionOverloadResolver instead of IFunction to handle non-default NULL processing. +/// Both NULL and JSON NULL should generate NULL value. If any argument is NULL, return NULL. template typename Impl> class JSONOverloadResolver : public IFunctionOverloadResolver, WithContext { @@ -389,9 +384,6 @@ public: bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } - - // Both NULL and JSON NULL should generate NULL value. - // If any argument is NULL, return NULL. bool useDefaultImplementationForNulls() const override { return false; } FunctionBasePtr build(const ColumnsWithTypeAndName & arguments) const override diff --git a/tests/queries/0_stateless/02013_json_function_null_column.reference b/tests/queries/0_stateless/02013_json_function_null_column.reference index fdf3cea38fd..0cfae73a705 100644 --- a/tests/queries/0_stateless/02013_json_function_null_column.reference +++ b/tests/queries/0_stateless/02013_json_function_null_column.reference @@ -1 +1,6 @@ -\N \N \N +\N Nullable(String) + String +\N Nullable(String) +\N Nullable(String) +\N Nullable(Nothing) +\N Nullable(Nothing) diff --git a/tests/queries/0_stateless/02013_json_function_null_column.sql b/tests/queries/0_stateless/02013_json_function_null_column.sql index 58005a05647..43ba5c04dcd 100644 --- a/tests/queries/0_stateless/02013_json_function_null_column.sql +++ b/tests/queries/0_stateless/02013_json_function_null_column.sql @@ -1,10 +1,6 @@ --- Tags: no-fasttest - -drop table if exists test_table; - -create table test_table (col String, col_nullable Nullable(String)) engine MergeTree order by col; -insert into test_table select '{"string_value":null}' as col, '{"string_value":null}' as col_nullable; - -select JSONExtract(col, 'string_value', 'Nullable(String)') as res1, JSONExtract(col_nullable, 'string_value', 'Nullable(String)') as res2, JSONExtract(assumeNotNull(col_nullable), 'string_value', 'Nullable(String)') as res3 from test_table; - -drop table test_table; +SELECT JSONExtract('{"string_value":null}', 'string_value', 'Nullable(String)') as x, toTypeName(x); +SELECT JSONExtract('{"string_value":null}', 'string_value', 'String') as x, toTypeName(x); +SELECT JSONExtract(toNullable('{"string_value":null}'), 'string_value', 'Nullable(String)') as x, toTypeName(x); +SELECT JSONExtract(toNullable('{"string_value":null}'), 'string_value', 'String') as x, toTypeName(x); +SELECT JSONExtract(NULL, 'string_value', 'Nullable(String)') as x, toTypeName(x); +SELECT JSONExtract(NULL, 'string_value', 'String') as x, toTypeName(x); From 8163b3f7f6916d04dbd1ddba07ac7fb0ab55eb53 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 28 Sep 2021 09:05:36 +0300 Subject: [PATCH 237/317] Fix 2024_merge_regexp_assert --- tests/queries/0_stateless/2024_merge_regexp_assert.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/2024_merge_regexp_assert.sql b/tests/queries/0_stateless/2024_merge_regexp_assert.sql index 8ea4a77bbd8..fed26b08ad9 100644 --- a/tests/queries/0_stateless/2024_merge_regexp_assert.sql +++ b/tests/queries/0_stateless/2024_merge_regexp_assert.sql @@ -1,3 +1,6 @@ +-- Tags: no-parallel +-- (databases can be removed in background, so this test should not be run in parallel) + DROP TABLE IF EXISTS t; CREATE TABLE t (b UInt8) ENGINE = Memory; SELECT a FROM merge(REGEXP('.'), '^t$'); -- { serverError 47 } From d6cd057e2c0c8b04d8a712a7b2a83526507ef512 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 28 Sep 2021 09:05:57 +0300 Subject: [PATCH 238/317] Rename 2024_merge_regexp_assert to 02024_merge_regexp_assert --- ...egexp_assert.reference => 02024_merge_regexp_assert.reference} | 0 ...2024_merge_regexp_assert.sql => 02024_merge_regexp_assert.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{2024_merge_regexp_assert.reference => 02024_merge_regexp_assert.reference} (100%) rename tests/queries/0_stateless/{2024_merge_regexp_assert.sql => 02024_merge_regexp_assert.sql} (100%) diff --git a/tests/queries/0_stateless/2024_merge_regexp_assert.reference b/tests/queries/0_stateless/02024_merge_regexp_assert.reference similarity index 100% rename from tests/queries/0_stateless/2024_merge_regexp_assert.reference rename to tests/queries/0_stateless/02024_merge_regexp_assert.reference diff --git a/tests/queries/0_stateless/2024_merge_regexp_assert.sql b/tests/queries/0_stateless/02024_merge_regexp_assert.sql similarity index 100% rename from tests/queries/0_stateless/2024_merge_regexp_assert.sql rename to tests/queries/0_stateless/02024_merge_regexp_assert.sql From eb5402622ea5149e7e750e14e58e99a89e87090c Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Sep 2021 10:24:01 +0300 Subject: [PATCH 239/317] Better test --- src/Coordination/tests/gtest_coordination.cpp | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 3ef0ebf99df..fa0f549f479 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1469,14 +1469,12 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) } -TEST_P(CoordinationTest, TestStorageSnapshotDifferenVersions) +TEST_P(CoordinationTest, TestStorageSnapshotDifferenCompressions) { auto params = GetParam(); - if (!params.enable_compression) - return; ChangelogDirTest test("./snapshots"); - DB::KeeperSnapshotManager manager("./snapshots", 3, false); + DB::KeeperSnapshotManager manager("./snapshots", 3, params.enable_compression); DB::KeeperStorage storage(500, ""); addNode(storage, "/hello", "world", 1); @@ -1492,9 +1490,9 @@ TEST_P(CoordinationTest, TestStorageSnapshotDifferenVersions) auto buf = manager.serializeSnapshotToBuffer(snapshot); manager.serializeSnapshotBufferToDisk(*buf, 2); - EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin")); + EXPECT_TRUE(fs::exists("./snapshots/snapshot_2.bin" + params.extension)); - DB::KeeperSnapshotManager new_manager("./snapshots", 3, true); + DB::KeeperSnapshotManager new_manager("./snapshots", 3, !params.enable_compression); auto debuf = new_manager.deserializeSnapshotBufferFromDisk(2); From 9a23b209654db2c867cf2e9bf9c4bdcb77267ad4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 28 Sep 2021 10:28:10 +0300 Subject: [PATCH 240/317] Remove debug from MergeTreeIndexConditionBloomFilter::traverseASTEquals() --- src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index a11eaa9a4be..5bc69761f49 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -511,8 +511,6 @@ 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()); From e709ead11ab5556114f1fc540d099f44dab8590e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 28 Sep 2021 10:23:12 +0300 Subject: [PATCH 241/317] Add std::cerr/std::cout style check --- utils/check-style/check-style | 32 ++++++++++++++++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 8c1d56c90d8..efdc5f488d2 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -195,6 +195,38 @@ find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' | grep -vP $EXCLUDE_DIRS | xargs grep -P 'std::[io]?stringstream' | grep -v "STYLE_CHECK_ALLOW_STD_STRING_STREAM" && echo "Use WriteBufferFromOwnString or ReadBufferFromString instead of std::stringstream" +# Forbid std::cerr/std::cout in src (fine in programs/utils) +std_cerr_cout_excludes=( + /examples/ + /tests/ + _fuzzer + # OK + src/Common/ProgressIndication.cpp + # only under #ifdef DBMS_HASH_MAP_DEBUG_RESIZES, that is used only in tests + src/Common/HashTable/HashTable.h + # SensitiveDataMasker::printStats() + src/Common/SensitiveDataMasker.cpp + # StreamStatistics::print() + src/Compression/LZ4_decompress_faster.cpp + # ContextSharedPart with subsequent std::terminate() + src/Interpreters/Context.cpp + # IProcessor::dump() + src/Processors/IProcessor.cpp +) +sources_with_std_cerr_cout=( $( + find $ROOT_PATH/src -name '*.h' -or -name '*.cpp' | \ + grep -vP $EXCLUDE_DIRS | \ + grep -F -v $(printf -- "-e %s " "${std_cerr_cout_excludes[@]}") | \ + xargs grep -F --with-filename -e std::cerr -e std::cout | cut -d: -f1 | sort -u +) ) +# Exclude comments +for src in "${sources_with_std_cerr_cout[@]}"; do + # suppress stderr, since it may contain warning for #pargma once in headers + if gcc -fpreprocessed -dD -E "$src" 2>/dev/null | grep -F -q -e std::cerr -e std::cout; then + echo "$src: uses std::cerr/std::cout" + fi +done + # Conflict markers find $ROOT_PATH/{src,base,programs,utils,tests,docs,website,cmake} -name '*.md' -or -name '*.cpp' -or -name '*.h' | xargs grep -P '^(<<<<<<<|=======|>>>>>>>)$' | grep -P '.' && echo "Conflict markers are found in files" From a5ad6f7f9062a556d933f8246a6aaebf168a6044 Mon Sep 17 00:00:00 2001 From: ClickHouse Admin <89963708+clickhouse-admin@users.noreply.github.com> Date: Tue, 28 Sep 2021 10:38:53 +0300 Subject: [PATCH 242/317] [github] we're switching to CLA based on Apache CLA CLA signing is automated with CLA Assistant. --- .github/PULL_REQUEST_TEMPLATE.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index a2930beb89f..82a16d0589f 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -1,5 +1,3 @@ -I hereby agree to the terms of the CLA available at: https://yandex.ru/legal/cla/?lang=en - Changelog category (leave one): - New Feature - Improvement From ec26768dcea08a9a75a1c5ab8ad49353fde5dd41 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Sep 2021 10:49:06 +0300 Subject: [PATCH 243/317] Temporary remove 01200_mutations_memory_consumption --- .../01200_mutations_memory_consumption.reference | 0 .../{0_stateless => bugs}/01200_mutations_memory_consumption.sql | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/{0_stateless => bugs}/01200_mutations_memory_consumption.reference (100%) rename tests/queries/{0_stateless => bugs}/01200_mutations_memory_consumption.sql (100%) diff --git a/tests/queries/0_stateless/01200_mutations_memory_consumption.reference b/tests/queries/bugs/01200_mutations_memory_consumption.reference similarity index 100% rename from tests/queries/0_stateless/01200_mutations_memory_consumption.reference rename to tests/queries/bugs/01200_mutations_memory_consumption.reference diff --git a/tests/queries/0_stateless/01200_mutations_memory_consumption.sql b/tests/queries/bugs/01200_mutations_memory_consumption.sql similarity index 100% rename from tests/queries/0_stateless/01200_mutations_memory_consumption.sql rename to tests/queries/bugs/01200_mutations_memory_consumption.sql From e7a233d9adc5241f557ff46f9276bde785cf2ab7 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 28 Sep 2021 11:03:50 +0300 Subject: [PATCH 244/317] switch to repo.clickhouse.com --- docs/_includes/install/deb.sh | 2 +- docs/_includes/install/rpm.sh | 4 ++-- docs/_includes/install/tgz.sh | 10 +++++----- docs/en/getting-started/install.md | 18 ++++++++--------- docs/ja/getting-started/install.md | 18 ++++++++--------- docs/ru/getting-started/install.md | 20 +++++++++---------- docs/zh/getting-started/install.md | 18 ++++++++--------- .../en/2020/package-repository-behind-cdn.md | 14 ++++++------- 8 files changed, 52 insertions(+), 52 deletions(-) diff --git a/docs/_includes/install/deb.sh b/docs/_includes/install/deb.sh index 6686a1f64cd..7dcca601d33 100644 --- a/docs/_includes/install/deb.sh +++ b/docs/_includes/install/deb.sh @@ -1,7 +1,7 @@ sudo apt-get install apt-transport-https ca-certificates dirmngr sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4 -echo "deb https://repo.clickhouse.tech/deb/stable/ main/" | sudo tee \ +echo "deb https://repo.clickhouse.com/deb/stable/ main/" | sudo tee \ /etc/apt/sources.list.d/clickhouse.list sudo apt-get update diff --git a/docs/_includes/install/rpm.sh b/docs/_includes/install/rpm.sh index 972d50124c4..de4a07420f7 100644 --- a/docs/_includes/install/rpm.sh +++ b/docs/_includes/install/rpm.sh @@ -1,6 +1,6 @@ sudo yum install yum-utils -sudo rpm --import https://repo.clickhouse.tech/CLICKHOUSE-KEY.GPG -sudo yum-config-manager --add-repo https://repo.clickhouse.tech/rpm/clickhouse.repo +sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/clickhouse.repo sudo yum install clickhouse-server clickhouse-client sudo /etc/init.d/clickhouse-server start diff --git a/docs/_includes/install/tgz.sh b/docs/_includes/install/tgz.sh index 52ea45bdf90..0994510755b 100644 --- a/docs/_includes/install/tgz.sh +++ b/docs/_includes/install/tgz.sh @@ -1,9 +1,9 @@ -export LATEST_VERSION=$(curl -s https://repo.clickhouse.tech/tgz/stable/ | \ +export LATEST_VERSION=$(curl -s https://repo.clickhouse.com/tgz/stable/ | \ grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | sort -V -r | head -n 1) -curl -O https://repo.clickhouse.tech/tgz/stable/clickhouse-common-static-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/stable/clickhouse-common-static-dbg-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/stable/clickhouse-server-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/stable/clickhouse-client-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-common-static-dbg-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-server-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/stable/clickhouse-client-$LATEST_VERSION.tgz tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 26b44962dca..7728be45c89 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -29,7 +29,7 @@ It is recommended to use official pre-compiled `deb` packages for Debian or Ubun If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). -You can also download and install packages manually from [here](https://repo.clickhouse.tech/deb/stable/main/). +You can also download and install packages manually from [here](https://repo.clickhouse.com/deb/stable/main/). #### Packages {#packages} @@ -50,8 +50,8 @@ First, you need to add the official repository: ``` bash sudo yum install yum-utils -sudo rpm --import https://repo.clickhouse.tech/CLICKHOUSE-KEY.GPG -sudo yum-config-manager --add-repo https://repo.clickhouse.tech/rpm/stable/x86_64 +sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64 ``` If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). `prestable` is sometimes also available. @@ -62,21 +62,21 @@ Then run these commands to install packages: sudo yum install clickhouse-server clickhouse-client ``` -You can also download and install packages manually from [here](https://repo.clickhouse.tech/rpm/stable/x86_64). +You can also download and install packages manually from [here](https://repo.clickhouse.com/rpm/stable/x86_64). ### From Tgz Archives {#from-tgz-archives} It is recommended to use official pre-compiled `tgz` archives for all Linux distributions, where installation of `deb` or `rpm` packages is not possible. -The required version can be downloaded with `curl` or `wget` from repository https://repo.clickhouse.tech/tgz/. +The required version can be downloaded with `curl` or `wget` from repository https://repo.clickhouse.com/tgz/. After that downloaded archives should be unpacked and installed with installation scripts. Example for the latest version: ``` bash export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1` -curl -O https://repo.clickhouse.tech/tgz/clickhouse-common-static-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-common-static-dbg-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-server-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-client-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-dbg-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-server-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-client-$LATEST_VERSION.tgz tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh diff --git a/docs/ja/getting-started/install.md b/docs/ja/getting-started/install.md index 189cd5119b4..1201d25acba 100644 --- a/docs/ja/getting-started/install.md +++ b/docs/ja/getting-started/install.md @@ -30,7 +30,7 @@ Debian や Ubuntu 用にコンパイル済みの公式パッケージ `deb` を 最新版を使いたい場合は、`stable`を`testing`に置き換えてください。(テスト環境ではこれを推奨します) -同様に、[こちら](https://repo.clickhouse.tech/deb/stable/main/)からパッケージをダウンロードして、手動でインストールすることもできます。 +同様に、[こちら](https://repo.clickhouse.com/deb/stable/main/)からパッケージをダウンロードして、手動でインストールすることもできます。 #### パッケージ {#packages} @@ -47,8 +47,8 @@ CentOS、RedHat、その他すべてのrpmベースのLinuxディストリビュ ``` bash sudo yum install yum-utils -sudo rpm --import https://repo.clickhouse.tech/CLICKHOUSE-KEY.GPG -sudo yum-config-manager --add-repo https://repo.clickhouse.tech/rpm/stable/x86_64 +sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64 ``` 最新版を使いたい場合は `stable` を `testing` に置き換えてください。(テスト環境ではこれが推奨されています)。`prestable` もしばしば同様に利用できます。 @@ -59,20 +59,20 @@ sudo yum-config-manager --add-repo https://repo.clickhouse.tech/rpm/stable/x86_6 sudo yum install clickhouse-server clickhouse-client ``` -同様に、[こちら](https://repo.clickhouse.tech/rpm/stable/x86_64) からパッケージをダウンロードして、手動でインストールすることもできます。 +同様に、[こちら](https://repo.clickhouse.com/rpm/stable/x86_64) からパッケージをダウンロードして、手動でインストールすることもできます。 ### Tgzアーカイブから {#from-tgz-archives} すべての Linux ディストリビューションで、`deb` や `rpm` パッケージがインストールできない場合は、公式のコンパイル済み `tgz` アーカイブを使用することをお勧めします。 -必要なバージョンは、リポジトリ https://repo.clickhouse.tech/tgz/ から `curl` または `wget` でダウンロードできます。その後、ダウンロードしたアーカイブを解凍し、インストールスクリプトでインストールしてください。最新版の例は以下です: +必要なバージョンは、リポジトリ https://repo.clickhouse.com/tgz/ から `curl` または `wget` でダウンロードできます。その後、ダウンロードしたアーカイブを解凍し、インストールスクリプトでインストールしてください。最新版の例は以下です: ``` bash export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1` -curl -O https://repo.clickhouse.tech/tgz/clickhouse-common-static-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-common-static-dbg-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-server-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-client-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-dbg-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-server-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-client-$LATEST_VERSION.tgz tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index 92f76c4951c..d2a5a4d5232 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -27,11 +27,11 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su {% include 'install/deb.sh' %} ``` -Также эти пакеты можно скачать и установить вручную отсюда: https://repo.clickhouse.tech/deb/stable/main/. +Также эти пакеты можно скачать и установить вручную отсюда: https://repo.clickhouse.com/deb/stable/main/. Если вы хотите использовать наиболее свежую версию, замените `stable` на `testing` (рекомендуется для тестовых окружений). -Также вы можете вручную скачать и установить пакеты из [репозитория](https://repo.clickhouse.tech/deb/stable/main/). +Также вы можете вручную скачать и установить пакеты из [репозитория](https://repo.clickhouse.com/deb/stable/main/). #### Пакеты {#packages} @@ -52,8 +52,8 @@ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not su ``` bash sudo yum install yum-utils -sudo rpm --import https://repo.clickhouse.tech/CLICKHOUSE-KEY.GPG -sudo yum-config-manager --add-repo https://repo.clickhouse.tech/rpm/stable/x86_64 +sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64 ``` Для использования наиболее свежих версий нужно заменить `stable` на `testing` (рекомендуется для тестовых окружений). Также иногда доступен `prestable`. @@ -64,21 +64,21 @@ sudo yum-config-manager --add-repo https://repo.clickhouse.tech/rpm/stable/x86_6 sudo yum install clickhouse-server clickhouse-client ``` -Также есть возможность установить пакеты вручную, скачав отсюда: https://repo.clickhouse.tech/rpm/stable/x86_64. +Также есть возможность установить пакеты вручную, скачав отсюда: https://repo.clickhouse.com/rpm/stable/x86_64. ### Из Tgz архивов {#from-tgz-archives} Команда ClickHouse в Яндексе рекомендует использовать предкомпилированные бинарники из `tgz` архивов для всех дистрибутивов, где невозможна установка `deb` и `rpm` пакетов. -Интересующую версию архивов можно скачать вручную с помощью `curl` или `wget` из репозитория https://repo.clickhouse.tech/tgz/. +Интересующую версию архивов можно скачать вручную с помощью `curl` или `wget` из репозитория https://repo.clickhouse.com/tgz/. После этого архивы нужно распаковать и воспользоваться скриптами установки. Пример установки самой свежей версии: ``` bash export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1` -curl -O https://repo.clickhouse.tech/tgz/clickhouse-common-static-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-common-static-dbg-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-server-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-client-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-dbg-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-server-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-client-$LATEST_VERSION.tgz tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh diff --git a/docs/zh/getting-started/install.md b/docs/zh/getting-started/install.md index 27ed0a40a12..204c8342e42 100644 --- a/docs/zh/getting-started/install.md +++ b/docs/zh/getting-started/install.md @@ -29,7 +29,7 @@ $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not 如果您想使用最新的版本,请用`testing`替代`stable`(我们只推荐您用于测试环境)。 -你也可以从这里手动下载安装包:[下载](https://repo.clickhouse.tech/deb/stable/main/)。 +你也可以从这里手动下载安装包:[下载](https://repo.clickhouse.com/deb/stable/main/)。 安装包列表: @@ -46,8 +46,8 @@ $ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not ``` bash sudo yum install yum-utils -sudo rpm --import https://repo.clickhouse.tech/CLICKHOUSE-KEY.GPG -sudo yum-config-manager --add-repo https://repo.clickhouse.tech/rpm/stable/x86_64 +sudo rpm --import https://repo.clickhouse.com/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.clickhouse.com/rpm/stable/x86_64 ``` 如果您想使用最新的版本,请用`testing`替代`stable`(我们只推荐您用于测试环境)。`prestable`有时也可用。 @@ -58,22 +58,22 @@ sudo yum-config-manager --add-repo https://repo.clickhouse.tech/rpm/stable/x86_6 sudo yum install clickhouse-server clickhouse-client ``` -你也可以从这里手动下载安装包:[下载](https://repo.clickhouse.tech/rpm/stable/x86_64)。 +你也可以从这里手动下载安装包:[下载](https://repo.clickhouse.com/rpm/stable/x86_64)。 ### `Tgz`安装包 {#from-tgz-archives} 如果您的操作系统不支持安装`deb`或`rpm`包,建议使用官方预编译的`tgz`软件包。 -所需的版本可以通过`curl`或`wget`从存储库`https://repo.clickhouse.tech/tgz/`下载。 +所需的版本可以通过`curl`或`wget`从存储库`https://repo.clickhouse.com/tgz/`下载。 下载后解压缩下载资源文件并使用安装脚本进行安装。以下是一个最新版本的安装示例: ``` bash export LATEST_VERSION=`curl https://api.github.com/repos/ClickHouse/ClickHouse/tags 2>/dev/null | grep -Eo '[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+' | head -n 1` -curl -O https://repo.clickhouse.tech/tgz/clickhouse-common-static-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-common-static-dbg-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-server-$LATEST_VERSION.tgz -curl -O https://repo.clickhouse.tech/tgz/clickhouse-client-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-common-static-dbg-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-server-$LATEST_VERSION.tgz +curl -O https://repo.clickhouse.com/tgz/clickhouse-client-$LATEST_VERSION.tgz tar -xzvf clickhouse-common-static-$LATEST_VERSION.tgz sudo clickhouse-common-static-$LATEST_VERSION/install/doinst.sh diff --git a/website/blog/en/2020/package-repository-behind-cdn.md b/website/blog/en/2020/package-repository-behind-cdn.md index fb724d18cb3..5b7992d16d8 100644 --- a/website/blog/en/2020/package-repository-behind-cdn.md +++ b/website/blog/en/2020/package-repository-behind-cdn.md @@ -8,27 +8,27 @@ tags: ['article', 'CDN', 'Cloudflare', 'repository', 'deb', 'rpm', 'tgz'] On initial open-source launch, ClickHouse packages were published at an independent repository implemented on Yandex infrastructure. We'd love to use the default repositories of Linux distributions, but, unfortunately, they have their own strict rules on third-party library usage and software compilation options. These rules happen to contradict with how ClickHouse is produced. In 2018 ClickHouse was added to [official Debian repository](https://packages.debian.org/sid/clickhouse-server) as an experiment, but it didn't get much traction. Adaptation to those rules ended up producing more like a demo version of ClickHouse with crippled performance and limited features. !!! info "TL;DR" - If you have configured your system to use for fetching ClickHouse packages, replace it with . + If you have configured your system to use for fetching ClickHouse packages, replace it with . Distributing packages via our own repository was working totally fine until ClickHouse has started getting traction in countries far from Moscow, most notably the USA and China. Downloading large files of packages from remote location was especially painful for Chinese ClickHouse users, likely due to how China is connected to the rest of the world via its famous firewall. But at least it worked (with high latencies and low throughput), while in some smaller countries there was completely no access to this repository and people living there had to host their own mirrors on neutral ground as a workaround. -Earlier this year we made the ClickHouse official website to be served via global CDN by [Cloudflare](https://www.cloudflare.com) on a `clickhouse.tech` domain. To solve the download issues discussed above, we have also configured a new location for ClickHouse packages that are also served by Cloudflare at [repo.clickhouse.tech](https://repo.clickhouse.tech). It used to have some quirks, but now it seems to be working fine while improving throughput and latencies in remote geographical locations by over an order of magnitude. +Earlier this year we made the ClickHouse official website to be served via global CDN by [Cloudflare](https://www.cloudflare.com) on a `clickhouse.tech` domain. To solve the download issues discussed above, we have also configured a new location for ClickHouse packages that are also served by Cloudflare at [repo.clickhouse.com](https://repo.clickhouse.com). It used to have some quirks, but now it seems to be working fine while improving throughput and latencies in remote geographical locations by over an order of magnitude. ## Switching To Repository Behind CDN -This transition has some more benefits besides improving the package fetching, but let's get back to them in a minute. One of the key reasons for this post is that we can't actually influence the repository configuration of ClickHouse users. We have updated all instructions, but for people who have followed these instructions earlier, **action is required** to use the new location behind CDN. Basically, you need to replace `http://repo.yandex.ru/clickhouse/` with `https://repo.clickhouse.tech/` in your package manager configuration. +This transition has some more benefits besides improving the package fetching, but let's get back to them in a minute. One of the key reasons for this post is that we can't actually influence the repository configuration of ClickHouse users. We have updated all instructions, but for people who have followed these instructions earlier, **action is required** to use the new location behind CDN. Basically, you need to replace `http://repo.yandex.ru/clickhouse/` with `https://repo.clickhouse.com/` in your package manager configuration. One-liner for Ubuntu or Debian: ```bash -sudo apt-get install apt-transport-https ca-certificates && sudo perl -pi -e 's|http://repo.yandex.ru/clickhouse/|https://repo.clickhouse.tech/|g' /etc/apt/sources.list.d/clickhouse.list && sudo apt-get update +sudo apt-get install apt-transport-https ca-certificates && sudo perl -pi -e 's|http://repo.yandex.ru/clickhouse/|https://repo.clickhouse.com/|g' /etc/apt/sources.list.d/clickhouse.list && sudo apt-get update ``` One-liner for RedHat or CentOS: ```bash -sudo perl -pi -e 's|http://repo.yandex.ru/clickhouse/|https://repo.clickhouse.tech/|g' /etc/yum.repos.d/clickhouse* +sudo perl -pi -e 's|http://repo.yandex.ru/clickhouse/|https://repo.clickhouse.com/|g' /etc/yum.repos.d/clickhouse* ``` -As you might have noticed, the domain name is not the only thing that has changed: the new URL uses `https://` protocol. Usually, it's considered less important for package repositories compared to normal websites because most package managers check [GPG signatures](https://en.wikipedia.org/wiki/GNU_Privacy_Guard) for what they download anyway. However it still has some benefits: for example, it's not so uncommon for people to download packages via browser, `curl` or `wget`, and install them manually (while for [tgz](https://repo.clickhouse.tech/tgz/) builds it's the only option). Fewer opportunities for sniffing traffic can't hurt either. The downside is that `apt` in some Debian flavors has no HTTPS support by default and needs a couple more packages to be installed (`apt-transport-https` and `ca-certificates`). +As you might have noticed, the domain name is not the only thing that has changed: the new URL uses `https://` protocol. Usually, it's considered less important for package repositories compared to normal websites because most package managers check [GPG signatures](https://en.wikipedia.org/wiki/GNU_Privacy_Guard) for what they download anyway. However it still has some benefits: for example, it's not so uncommon for people to download packages via browser, `curl` or `wget`, and install them manually (while for [tgz](https://repo.clickhouse.com/tgz/) builds it's the only option). Fewer opportunities for sniffing traffic can't hurt either. The downside is that `apt` in some Debian flavors has no HTTPS support by default and needs a couple more packages to be installed (`apt-transport-https` and `ca-certificates`). ## Investigating Repository Usage @@ -61,7 +61,7 @@ There's not so much data collected yet, but here's a live example of how the res While here we confirmed that `rpm` is at least as popular as `deb`: ![iframe](https://datalens.yandex/lfvldsf92i2uh?_embedded=1) -Or you can take a look at all key charts for `repo.clickhouse.tech` together on a handy **[dashboard](https://datalens.yandex/pjzq4rot3t2ql)** with a filtering possibility. +Or you can take a look at all key charts for `repo.clickhouse.com` together on a handy **[dashboard](https://datalens.yandex/pjzq4rot3t2ql)** with a filtering possibility. ## Lessons Learned From 3cd3850edf8b4d4aebf6c43712b9952363110ab1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 28 Sep 2021 11:51:02 +0300 Subject: [PATCH 245/317] Update gtest_coordination.cpp --- src/Coordination/tests/gtest_coordination.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index fa0f549f479..c6a4e723ec5 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1469,7 +1469,7 @@ TEST_P(CoordinationTest, TestCompressedLogsMultipleRewrite) } -TEST_P(CoordinationTest, TestStorageSnapshotDifferenCompressions) +TEST_P(CoordinationTest, TestStorageSnapshotDifferentCompressions) { auto params = GetParam(); From 71fb9e60599afa10d0571a00e1b849c1a37a5e53 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 28 Sep 2021 12:38:27 +0300 Subject: [PATCH 246/317] Bloom filter indexes updated tests --- .../MergeTreeIndexConditionBloomFilter.cpp | 1 - .../2021_map_bloom_filter_index.sql | 46 +++++++++---------- ...ray_full_text_bloom_filter_index.reference | 16 +++---- ...022_array_full_text_bloom_filter_index.sql | 36 +++++++-------- 4 files changed, 49 insertions(+), 50 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index a11eaa9a4be..7219eec8ffb 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -408,7 +408,6 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn( 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; } else { 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 c7e522db631..7a03ff3c473 100644 --- a/tests/queries/0_stateless/2021_map_bloom_filter_index.sql +++ b/tests/queries/0_stateless/2021_map_bloom_filter_index.sql @@ -4,51 +4,51 @@ CREATE TABLE map_test_index_map_keys row_id UInt32, map Map(String, String), INDEX map_bloom_filter_keys mapKeys(map) TYPE bloom_filter GRANULARITY 1 -) Engine=MergeTree() ORDER BY row_id; +) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1; INSERT INTO map_test_index_map_keys VALUES (0, {'K0':'V0'}), (1, {'K1':'V1'}); SELECT 'Map bloom filter mapKeys'; SELECT 'Equals with existing key'; -SELECT * FROM map_test_index_map_keys WHERE map['K0'] = 'V0'; +SELECT * FROM map_test_index_map_keys WHERE map['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_bloom_filter_keys'; SELECT 'Equals with non existing key'; -SELECT * FROM map_test_index_map_keys WHERE map['K2'] = 'V2'; +SELECT * FROM map_test_index_map_keys WHERE map['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_bloom_filter_keys'; SELECT 'Equals with non existing key and default value'; SELECT * FROM map_test_index_map_keys WHERE map['K3'] = ''; SELECT 'Not equals with existing key'; -SELECT * FROM map_test_index_map_keys WHERE map['K0'] != 'V0'; +SELECT * FROM map_test_index_map_keys WHERE map['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_bloom_filter_keys'; SELECT 'Not equals with non existing key'; -SELECT * FROM map_test_index_map_keys WHERE map['K2'] != 'V2'; +SELECT * FROM map_test_index_map_keys WHERE map['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_bloom_filter_keys'; SELECT 'Not equals with non existing key and default value'; SELECT * FROM map_test_index_map_keys WHERE map['K3'] != ''; SELECT 'IN with existing key'; -SELECT * FROM map_test_index_map_keys WHERE map['K0'] IN 'V0'; +SELECT * FROM map_test_index_map_keys WHERE map['K0'] IN 'V0' SETTINGS force_data_skipping_indices='map_bloom_filter_keys'; SELECT 'IN with non existing key'; -SELECT * FROM map_test_index_map_keys WHERE map['K2'] IN 'V2'; +SELECT * FROM map_test_index_map_keys WHERE map['K2'] IN 'V2' SETTINGS force_data_skipping_indices='map_bloom_filter_keys'; SELECT 'IN with non existing key and default value'; SELECT * FROM map_test_index_map_keys WHERE map['K3'] IN ''; SELECT 'NOT IN with existing key'; -SELECT * FROM map_test_index_map_keys WHERE map['K0'] NOT IN 'V0'; +SELECT * FROM map_test_index_map_keys WHERE map['K0'] NOT IN 'V0' SETTINGS force_data_skipping_indices='map_bloom_filter_keys'; SELECT 'NOT IN with non existing key'; -SELECT * FROM map_test_index_map_keys WHERE map['K2'] NOT IN 'V2'; +SELECT * FROM map_test_index_map_keys WHERE map['K2'] NOT IN 'V2' SETTINGS force_data_skipping_indices='map_bloom_filter_keys'; SELECT 'NOT IN with non existing key and default value'; SELECT * FROM map_test_index_map_keys WHERE map['K3'] NOT IN ''; SELECT 'MapContains with existing key'; -SELECT * FROM map_test_index_map_keys WHERE mapContains(map, 'K0'); +SELECT * FROM map_test_index_map_keys WHERE mapContains(map, 'K0') SETTINGS force_data_skipping_indices='map_bloom_filter_keys'; SELECT 'MapContains with non existing key'; -SELECT * FROM map_test_index_map_keys WHERE mapContains(map, 'K2'); +SELECT * FROM map_test_index_map_keys WHERE mapContains(map, 'K2') SETTINGS force_data_skipping_indices='map_bloom_filter_keys'; SELECT 'MapContains with non existing key and default value'; SELECT * FROM map_test_index_map_keys WHERE mapContains(map, ''); SELECT 'Has with existing key'; -SELECT * FROM map_test_index_map_keys WHERE has(map, 'K0'); +SELECT * FROM map_test_index_map_keys WHERE has(map, 'K0') SETTINGS force_data_skipping_indices='map_bloom_filter_keys'; SELECT 'Has with non existing key'; -SELECT * FROM map_test_index_map_keys WHERE has(map, 'K2'); +SELECT * FROM map_test_index_map_keys WHERE has(map, 'K2') SETTINGS force_data_skipping_indices='map_bloom_filter_keys'; SELECT 'Has with non existing key and default value'; -SELECT * FROM map_test_index_map_keys WHERE has(map, ''); +SELECT * FROM map_test_index_map_keys WHERE has(map, '') SETTINGS force_data_skipping_indices='map_bloom_filter_keys'; DROP TABLE map_test_index_map_keys; @@ -58,34 +58,34 @@ CREATE TABLE map_test_index_map_values row_id UInt32, map Map(String, String), INDEX map_bloom_filter_values mapValues(map) TYPE bloom_filter GRANULARITY 1 -) Engine=MergeTree() ORDER BY row_id; +) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1; 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 * FROM map_test_index_map_values WHERE map['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_bloom_filter_values'; SELECT 'Equals with non existing key'; -SELECT * FROM map_test_index_map_values WHERE map['K2'] = 'V2'; +SELECT * FROM map_test_index_map_values WHERE map['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_bloom_filter_values'; 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 * FROM map_test_index_map_values WHERE map['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_bloom_filter_values'; SELECT 'Not equals with non existing key'; -SELECT * FROM map_test_index_map_values WHERE map['K2'] != 'V2'; +SELECT * FROM map_test_index_map_values WHERE map['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_bloom_filter_values'; 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 * FROM map_test_index_map_values WHERE map['K0'] IN 'V0' SETTINGS force_data_skipping_indices='map_bloom_filter_values'; SELECT 'IN with non existing key'; -SELECT * FROM map_test_index_map_values WHERE map['K2'] IN 'V2'; +SELECT * FROM map_test_index_map_values WHERE map['K2'] IN 'V2' SETTINGS force_data_skipping_indices='map_bloom_filter_values'; SELECT 'IN with non existing key and default value'; SELECT * FROM map_test_index_map_values WHERE map['K3'] IN ''; SELECT 'NOT IN with existing key'; -SELECT * FROM map_test_index_map_values WHERE map['K0'] NOT IN 'V0'; +SELECT * FROM map_test_index_map_values WHERE map['K0'] NOT IN 'V0' SETTINGS force_data_skipping_indices='map_bloom_filter_values'; SELECT 'NOT IN with non existing key'; -SELECT * FROM map_test_index_map_values WHERE map['K2'] NOT IN 'V2'; +SELECT * FROM map_test_index_map_values WHERE map['K2'] NOT IN 'V2' SETTINGS force_data_skipping_indices='map_bloom_filter_values'; SELECT 'NOT IN with non existing key and default value'; SELECT * FROM map_test_index_map_values WHERE map['K3'] NOT IN ''; diff --git a/tests/queries/0_stateless/2022_array_full_text_bloom_filter_index.reference b/tests/queries/0_stateless/2022_array_full_text_bloom_filter_index.reference index f61dedd9bd2..3d9bc03bd63 100644 --- a/tests/queries/0_stateless/2022_array_full_text_bloom_filter_index.reference +++ b/tests/queries/0_stateless/2022_array_full_text_bloom_filter_index.reference @@ -1,8 +1,8 @@ -1 ['K1 K1'] ['K1 K1'] -2 ['K2 K2'] ['K2 K2'] -1 ['K1 K1'] ['K1 K1'] -2 ['K2 K2'] ['K2 K2'] -1 ['K1 K1'] ['K1 K1'] -2 ['K2 K2'] ['K2 K2'] -1 ['K1 K1'] ['K1 K1'] -2 ['K2 K2'] ['K2 K2'] +1 ['K1'] ['K1'] +2 ['K2'] ['K2'] +1 ['K1'] ['K1'] +2 ['K2'] ['K2'] +1 ['K1'] ['K1'] +2 ['K2'] ['K2'] +1 ['K1'] ['K1'] +2 ['K2'] ['K2'] diff --git a/tests/queries/0_stateless/2022_array_full_text_bloom_filter_index.sql b/tests/queries/0_stateless/2022_array_full_text_bloom_filter_index.sql index 6a2a00674cb..646a566b877 100644 --- a/tests/queries/0_stateless/2022_array_full_text_bloom_filter_index.sql +++ b/tests/queries/0_stateless/2022_array_full_text_bloom_filter_index.sql @@ -5,38 +5,38 @@ CREATE TABLE bf_tokenbf_array_test ( row_id UInt32, array Array(String), - array_fixed Array(FixedString(5)), + array_fixed Array(FixedString(2)), INDEX array_bf_tokenbf array TYPE tokenbf_v1(256,2,0) GRANULARITY 1, INDEX array_fixed_bf_tokenbf array_fixed TYPE tokenbf_v1(256,2,0) GRANULARITY 1 -) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 2; +) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1; CREATE TABLE bf_ngram_array_test ( row_id UInt32, array Array(String), - array_fixed Array(FixedString(5)), + array_fixed Array(FixedString(2)), INDEX array_ngram array TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1, INDEX array_fixed_ngram array_fixed TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1 -) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 2; +) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1; -INSERT INTO bf_tokenbf_array_test VALUES (1, ['K1 K1'], ['K1 K1']), (2, ['K2 K2'], ['K2 K2']); -INSERT INTO bf_ngram_array_test VALUES (1, ['K1 K1'], ['K1 K1']), (2, ['K2 K2'], ['K2 K2']); +INSERT INTO bf_tokenbf_array_test VALUES (1, ['K1'], ['K1']), (2, ['K2'], ['K2']); +INSERT INTO bf_ngram_array_test VALUES (1, ['K1'], ['K1']), (2, ['K2'], ['K2']); -SELECT * FROM bf_tokenbf_array_test WHERE has(array, 'K1 K1'); -SELECT * FROM bf_tokenbf_array_test WHERE has(array, 'K2 K2'); -SELECT * FROM bf_tokenbf_array_test WHERE has(array, 'K3 K3'); +SELECT * FROM bf_tokenbf_array_test WHERE has(array, 'K1') SETTINGS force_data_skipping_indices='array_bf_tokenbf'; +SELECT * FROM bf_tokenbf_array_test WHERE has(array, 'K2') SETTINGS force_data_skipping_indices='array_bf_tokenbf'; +SELECT * FROM bf_tokenbf_array_test WHERE has(array, 'K3') SETTINGS force_data_skipping_indices='array_bf_tokenbf'; -SELECT * FROM bf_tokenbf_array_test WHERE has(array_fixed, 'K1 K1'); -SELECT * FROM bf_tokenbf_array_test WHERE has(array_fixed, 'K2 K2'); -SELECT * FROM bf_tokenbf_array_test WHERE has(array_fixed, 'K3 K3'); +SELECT * FROM bf_tokenbf_array_test WHERE has(array_fixed, 'K1') SETTINGS force_data_skipping_indices='array_fixed_bf_tokenbf'; +SELECT * FROM bf_tokenbf_array_test WHERE has(array_fixed, 'K2') SETTINGS force_data_skipping_indices='array_fixed_bf_tokenbf'; +SELECT * FROM bf_tokenbf_array_test WHERE has(array_fixed, 'K3') SETTINGS force_data_skipping_indices='array_fixed_bf_tokenbf'; -SELECT * FROM bf_ngram_array_test WHERE has(array, 'K1 K1'); -SELECT * FROM bf_ngram_array_test WHERE has(array, 'K2 K2'); -SELECT * FROM bf_ngram_array_test WHERE has(array, 'K3 K3'); +SELECT * FROM bf_ngram_array_test WHERE has(array, 'K1') SETTINGS force_data_skipping_indices='array_ngram'; +SELECT * FROM bf_ngram_array_test WHERE has(array, 'K2') SETTINGS force_data_skipping_indices='array_ngram'; +SELECT * FROM bf_ngram_array_test WHERE has(array, 'K3') SETTINGS force_data_skipping_indices='array_ngram'; -SELECT * FROM bf_ngram_array_test WHERE has(array_fixed, 'K1 K1'); -SELECT * FROM bf_ngram_array_test WHERE has(array_fixed, 'K2 K2'); -SELECT * FROM bf_ngram_array_test WHERE has(array_fixed, 'K3 K3'); +SELECT * FROM bf_ngram_array_test WHERE has(array_fixed, 'K1') SETTINGS force_data_skipping_indices='array_fixed_ngram'; +SELECT * FROM bf_ngram_array_test WHERE has(array_fixed, 'K2') SETTINGS force_data_skipping_indices='array_fixed_ngram'; +SELECT * FROM bf_ngram_array_test WHERE has(array_fixed, 'K3') SETTINGS force_data_skipping_indices='array_fixed_ngram'; DROP TABLE bf_tokenbf_array_test; DROP TABLE bf_ngram_array_test; From 09cd955aa69453f6a272f21cbd41f7ab2ec1e2ba Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 28 Sep 2021 12:46:34 +0300 Subject: [PATCH 247/317] Remove filter column from HAVING when it is not needed. --- .../2025_having_filter_column.reference | 0 .../0_stateless/2025_having_filter_column.sql | 40 +++++++++++++++++++ 2 files changed, 40 insertions(+) create mode 100644 tests/queries/0_stateless/2025_having_filter_column.reference create mode 100644 tests/queries/0_stateless/2025_having_filter_column.sql diff --git a/tests/queries/0_stateless/2025_having_filter_column.reference b/tests/queries/0_stateless/2025_having_filter_column.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/2025_having_filter_column.sql b/tests/queries/0_stateless/2025_having_filter_column.sql new file mode 100644 index 00000000000..aab419adc16 --- /dev/null +++ b/tests/queries/0_stateless/2025_having_filter_column.sql @@ -0,0 +1,40 @@ +drop table if exists test; + +-- #29010 +CREATE TABLE test +( + d DateTime, + a String, + b UInt64 +) +ENGINE = MergeTree +PARTITION BY toDate(d) +ORDER BY d; + +SELECT * +FROM ( + SELECT + a, + max((d, b)).2 AS value + FROM test + GROUP BY rollup(a) +) +WHERE a <> ''; + +-- the same query, but after syntax optimization +SELECT + a, + value +FROM +( + SELECT + a, + max((d, b)).2 AS value + FROM test + GROUP BY a + WITH ROLLUP + HAVING a != '' +) +WHERE a != ''; + +drop table if exists test; From cfd8df089573f0fda1a25057437f972a8fbff6ca Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 28 Sep 2021 17:46:46 +0800 Subject: [PATCH 248/317] disable fasttest --- tests/queries/0_stateless/02013_json_function_null_column.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02013_json_function_null_column.sql b/tests/queries/0_stateless/02013_json_function_null_column.sql index 43ba5c04dcd..e72c7453c9a 100644 --- a/tests/queries/0_stateless/02013_json_function_null_column.sql +++ b/tests/queries/0_stateless/02013_json_function_null_column.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + SELECT JSONExtract('{"string_value":null}', 'string_value', 'Nullable(String)') as x, toTypeName(x); SELECT JSONExtract('{"string_value":null}', 'string_value', 'String') as x, toTypeName(x); SELECT JSONExtract(toNullable('{"string_value":null}'), 'string_value', 'Nullable(String)') as x, toTypeName(x); From f2402e1a103f7cd526bfb3ef22c927a8be095730 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 28 Sep 2021 12:52:07 +0300 Subject: [PATCH 249/317] Fixed tests --- .../MergeTree/MergeTreeIndexFullText.cpp | 4 +- ...map_full_text_bloom_filter_index.reference | 104 +++++++++- ...02000_map_full_text_bloom_filter_index.sql | 186 ++++++++++++++++-- 3 files changed, 268 insertions(+), 26 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index d76f19d6304..ccc92a853ca 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -16,8 +16,6 @@ #include #include #include -#include -#include #include @@ -550,7 +548,7 @@ bool MergeTreeConditionFullText::traverseASTEquals( else if (function_name == "equals") { out.key_column = key_column_num; - return createFunctionEqualsCondition(out, value_field, params, token_extractor); + return createFunctionEqualsCondition(out, const_value, params, token_extractor); } else if (function_name == "like") { diff --git a/tests/queries/0_stateless/02000_map_full_text_bloom_filter_index.reference b/tests/queries/0_stateless/02000_map_full_text_bloom_filter_index.reference index 93ca7fc2ff9..28ede3d4074 100644 --- a/tests/queries/0_stateless/02000_map_full_text_bloom_filter_index.reference +++ b/tests/queries/0_stateless/02000_map_full_text_bloom_filter_index.reference @@ -1,4 +1,104 @@ +Map full text bloom filter tokenbf mapKeys +Equals with existing key +0 {'K0':'V0'} {'K0':'V0'} +Equals with non existing key +Equals with non existing key and default value +0 {'K0':'V0'} {'K0':'V0'} 1 {'K1':'V1'} {'K1':'V1'} -2 {'K2':'V2'} {'K2':'V2'} +Not equals with existing key 1 {'K1':'V1'} {'K1':'V1'} -2 {'K2':'V2'} {'K2':'V2'} +Not equals with non existing key +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key and default value +Map fixed full text bloom filter tokenbf mapKeys +Equals with existing key +0 {'K0':'V0'} {'K0':'V0'} +Equals with non existing key +Equals with non existing key and default value +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with existing key +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key and default value +Map full text bloom filter tokenbf mapValues +Equals with existing key +0 {'K0':'V0'} {'K0':'V0'} +Equals with non existing key +Equals with non existing key and default value +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with existing key +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key and default value +Map fixed full text bloom filter tokenbf mapKeys +Equals with existing key +0 {'K0':'V0'} {'K0':'V0'} +Equals with non existing key +Equals with non existing key and default value +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with existing key +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key and default value +Map full text bloom filter ngrambf mapKeys +Equals with existing key +0 {'K0':'V0'} {'K0':'V0'} +Equals with non existing key +Equals with non existing key and default value +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with existing key +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key and default value +Map fixed full text bloom filter ngrambf mapKeys +Equals with existing key +0 {'K0':'V0'} {'K0':'V0'} +Equals with non existing key +Equals with non existing key and default value +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with existing key +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key and default value +Map full text bloom filter ngrambf mapValues +Equals with existing key +0 {'K0':'V0'} {'K0':'V0'} +Equals with non existing key +Equals with non existing key and default value +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with existing key +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key and default value +Map fixed full text bloom filter ngrambf mapKeys +Equals with existing key +0 {'K0':'V0'} {'K0':'V0'} +Equals with non existing key +Equals with non existing key and default value +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with existing key +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key +0 {'K0':'V0'} {'K0':'V0'} +1 {'K1':'V1'} {'K1':'V1'} +Not equals with non existing key and default value diff --git a/tests/queries/0_stateless/02000_map_full_text_bloom_filter_index.sql b/tests/queries/0_stateless/02000_map_full_text_bloom_filter_index.sql index ca262411eb8..18e75eac9bc 100644 --- a/tests/queries/0_stateless/02000_map_full_text_bloom_filter_index.sql +++ b/tests/queries/0_stateless/02000_map_full_text_bloom_filter_index.sql @@ -1,30 +1,174 @@ -DROP TABLE IF EXISTS bf_tokenbf_map_test; -DROP TABLE IF EXISTS bf_ngram_map_test; +DROP TABLE IF EXISTS bf_tokenbf_map_keys_test; +DROP TABLE IF EXISTS bf_ngrambf_map_keys_test; -CREATE TABLE bf_tokenbf_map_test -( - row_id UInt32, - map Map(String, String), - INDEX map_tokenbf map TYPE tokenbf_v1(256,2,0) GRANULARITY 1 -) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 2; - -CREATE TABLE bf_ngram_map_test +CREATE TABLE bf_tokenbf_map_keys_test ( row_id UInt32, map Map(String, String), map_fixed Map(FixedString(2), String), - INDEX map_ngram map TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1, - INDEX map_fixed_ngram map_fixed TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1 -) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 2; + INDEX map_keys_tokenbf mapKeys(map) TYPE tokenbf_v1(256,2,0) GRANULARITY 1, + INDEX map_fixed_keys_tokenbf mapKeys(map_fixed) TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1 +) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1; -INSERT INTO bf_tokenbf_map_test VALUES (1, {'K1':'V1'}, {'K1':'V1'}), (2, {'K2':'V2'}, {'K2':'V2'}); -INSERT INTO bf_ngram_map_test VALUES (1, {'K1':'V1'}, {'K1':'V1'}), (2, {'K2':'V2'}, {'K2':'V2'}); +INSERT INTO bf_tokenbf_map_keys_test VALUES (0, {'K0':'V0'}, {'K0':'V0'}), (1, {'K1':'V1'}, {'K1':'V1'}); -SELECT * FROM bf_tokenbf_map_test WHERE map['K1']='V1'; -SELECT * FROM bf_ngram_map_test WHERE map['K2']='V2'; +SELECT 'Map full text bloom filter tokenbf mapKeys'; -SELECT * FROM bf_tokenbf_map_test WHERE map_fixed['K1']='V1'; -SELECT * FROM bf_ngram_map_test WHERE map_fixed['K2']='V2'; +SELECT 'Equals with existing key'; +SELECT * FROM bf_tokenbf_map_keys_test WHERE map['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_keys_tokenbf'; +SELECT 'Equals with non existing key'; +SELECT * FROM bf_tokenbf_map_keys_test WHERE map['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_keys_tokenbf'; +SELECT 'Equals with non existing key and default value'; +SELECT * FROM bf_tokenbf_map_keys_test WHERE map['K3'] = ''; +SELECT 'Not equals with existing key'; +SELECT * FROM bf_tokenbf_map_keys_test WHERE map['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_keys_tokenbf'; +SELECT 'Not equals with non existing key'; +SELECT * FROM bf_tokenbf_map_keys_test WHERE map['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_keys_tokenbf'; +SELECT 'Not equals with non existing key and default value'; +SELECT * FROM bf_tokenbf_map_keys_test WHERE map['K3'] != ''; -DROP TABLE bf_tokenbf_map_test; -DROP TABLE bf_ngram_map_test; +SELECT 'Map fixed full text bloom filter tokenbf mapKeys'; + +SELECT 'Equals with existing key'; +SELECT * FROM bf_tokenbf_map_keys_test WHERE map_fixed['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_fixed_keys_tokenbf'; +SELECT 'Equals with non existing key'; +SELECT * FROM bf_tokenbf_map_keys_test WHERE map_fixed['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_fixed_keys_tokenbf'; +SELECT 'Equals with non existing key and default value'; +SELECT * FROM bf_tokenbf_map_keys_test WHERE map_fixed['K3'] = ''; +SELECT 'Not equals with existing key'; +SELECT * FROM bf_tokenbf_map_keys_test WHERE map_fixed['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_fixed_keys_tokenbf'; +SELECT 'Not equals with non existing key'; +SELECT * FROM bf_tokenbf_map_keys_test WHERE map_fixed['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_fixed_keys_tokenbf'; +SELECT 'Not equals with non existing key and default value'; +SELECT * FROM bf_tokenbf_map_keys_test WHERE map_fixed['K3'] != ''; + +DROP TABLE bf_tokenbf_map_keys_test; + +CREATE TABLE bf_tokenbf_map_values_test +( + row_id UInt32, + map Map(String, String), + map_fixed Map(FixedString(2), String), + INDEX map_values_tokenbf mapValues(map) TYPE tokenbf_v1(256,2,0) GRANULARITY 1, + INDEX map_fixed_values_tokenbf mapValues(map_fixed) TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1 +) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1; + +INSERT INTO bf_tokenbf_map_values_test VALUES (0, {'K0':'V0'}, {'K0':'V0'}), (1, {'K1':'V1'}, {'K1':'V1'}); + +SELECT 'Map full text bloom filter tokenbf mapValues'; + +SELECT 'Equals with existing key'; +SELECT * FROM bf_tokenbf_map_values_test WHERE map['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_values_tokenbf'; +SELECT 'Equals with non existing key'; +SELECT * FROM bf_tokenbf_map_values_test WHERE map['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_values_tokenbf'; +SELECT 'Equals with non existing key and default value'; +SELECT * FROM bf_tokenbf_map_values_test WHERE map['K3'] = ''; +SELECT 'Not equals with existing key'; +SELECT * FROM bf_tokenbf_map_values_test WHERE map['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_values_tokenbf'; +SELECT 'Not equals with non existing key'; +SELECT * FROM bf_tokenbf_map_values_test WHERE map['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_values_tokenbf'; +SELECT 'Not equals with non existing key and default value'; +SELECT * FROM bf_tokenbf_map_values_test WHERE map['K3'] != ''; + +SELECT 'Map fixed full text bloom filter tokenbf mapKeys'; + +SELECT 'Equals with existing key'; +SELECT * FROM bf_tokenbf_map_values_test WHERE map_fixed['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_fixed_values_tokenbf'; +SELECT 'Equals with non existing key'; +SELECT * FROM bf_tokenbf_map_values_test WHERE map_fixed['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_fixed_values_tokenbf'; +SELECT 'Equals with non existing key and default value'; +SELECT * FROM bf_tokenbf_map_values_test WHERE map_fixed['K3'] = ''; +SELECT 'Not equals with existing key'; +SELECT * FROM bf_tokenbf_map_values_test WHERE map_fixed['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_fixed_values_tokenbf'; +SELECT 'Not equals with non existing key'; +SELECT * FROM bf_tokenbf_map_values_test WHERE map_fixed['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_fixed_values_tokenbf'; +SELECT 'Not equals with non existing key and default value'; +SELECT * FROM bf_tokenbf_map_values_test WHERE map_fixed['K3'] != ''; + +DROP TABLE bf_tokenbf_map_values_test; + +CREATE TABLE bf_ngrambf_map_keys_test +( + row_id UInt32, + map Map(String, String), + map_fixed Map(FixedString(2), String), + INDEX map_keys_ngrambf mapKeys(map) TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1, + INDEX map_fixed_keys_ngrambf mapKeys(map_fixed) TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1 +) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1; + +INSERT INTO bf_ngrambf_map_keys_test VALUES (0, {'K0':'V0'}, {'K0':'V0'}), (1, {'K1':'V1'}, {'K1':'V1'}); + +SELECT 'Map full text bloom filter ngrambf mapKeys'; + +SELECT 'Equals with existing key'; +SELECT * FROM bf_ngrambf_map_keys_test WHERE map['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_keys_ngrambf'; +SELECT 'Equals with non existing key'; +SELECT * FROM bf_ngrambf_map_keys_test WHERE map['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_keys_ngrambf'; +SELECT 'Equals with non existing key and default value'; +SELECT * FROM bf_ngrambf_map_keys_test WHERE map['K3'] = ''; +SELECT 'Not equals with existing key'; +SELECT * FROM bf_ngrambf_map_keys_test WHERE map['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_keys_ngrambf'; +SELECT 'Not equals with non existing key'; +SELECT * FROM bf_ngrambf_map_keys_test WHERE map['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_keys_ngrambf'; +SELECT 'Not equals with non existing key and default value'; +SELECT * FROM bf_ngrambf_map_keys_test WHERE map['K3'] != ''; + +SELECT 'Map fixed full text bloom filter ngrambf mapKeys'; + +SELECT 'Equals with existing key'; +SELECT * FROM bf_ngrambf_map_keys_test WHERE map_fixed['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_fixed_keys_ngrambf'; +SELECT 'Equals with non existing key'; +SELECT * FROM bf_ngrambf_map_keys_test WHERE map_fixed['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_fixed_keys_ngrambf'; +SELECT 'Equals with non existing key and default value'; +SELECT * FROM bf_ngrambf_map_keys_test WHERE map_fixed['K3'] = ''; +SELECT 'Not equals with existing key'; +SELECT * FROM bf_ngrambf_map_keys_test WHERE map_fixed['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_fixed_keys_ngrambf'; +SELECT 'Not equals with non existing key'; +SELECT * FROM bf_ngrambf_map_keys_test WHERE map_fixed['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_fixed_keys_ngrambf'; +SELECT 'Not equals with non existing key and default value'; +SELECT * FROM bf_ngrambf_map_keys_test WHERE map_fixed['K3'] != ''; + +DROP TABLE bf_ngrambf_map_keys_test; + +CREATE TABLE bf_ngrambf_map_values_test +( + row_id UInt32, + map Map(String, String), + map_fixed Map(FixedString(2), String), + INDEX map_values_ngrambf mapKeys(map) TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1, + INDEX map_fixed_values_ngrambf mapKeys(map_fixed) TYPE ngrambf_v1(4,256,2,0) GRANULARITY 1 +) Engine=MergeTree() ORDER BY row_id SETTINGS index_granularity = 1; + +INSERT INTO bf_ngrambf_map_values_test VALUES (0, {'K0':'V0'}, {'K0':'V0'}), (1, {'K1':'V1'}, {'K1':'V1'}); + +SELECT 'Map full text bloom filter ngrambf mapValues'; + +SELECT 'Equals with existing key'; +SELECT * FROM bf_ngrambf_map_values_test WHERE map['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_values_ngrambf'; +SELECT 'Equals with non existing key'; +SELECT * FROM bf_ngrambf_map_values_test WHERE map['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_values_ngrambf'; +SELECT 'Equals with non existing key and default value'; +SELECT * FROM bf_ngrambf_map_values_test WHERE map['K3'] = ''; +SELECT 'Not equals with existing key'; +SELECT * FROM bf_ngrambf_map_values_test WHERE map['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_values_ngrambf'; +SELECT 'Not equals with non existing key'; +SELECT * FROM bf_ngrambf_map_values_test WHERE map['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_values_ngrambf'; +SELECT 'Not equals with non existing key and default value'; +SELECT * FROM bf_ngrambf_map_values_test WHERE map['K3'] != ''; + +SELECT 'Map fixed full text bloom filter ngrambf mapKeys'; + +SELECT 'Equals with existing key'; +SELECT * FROM bf_ngrambf_map_values_test WHERE map_fixed['K0'] = 'V0' SETTINGS force_data_skipping_indices='map_fixed_values_ngrambf'; +SELECT 'Equals with non existing key'; +SELECT * FROM bf_ngrambf_map_values_test WHERE map_fixed['K2'] = 'V2' SETTINGS force_data_skipping_indices='map_fixed_values_ngrambf'; +SELECT 'Equals with non existing key and default value'; +SELECT * FROM bf_ngrambf_map_values_test WHERE map_fixed['K3'] = ''; +SELECT 'Not equals with existing key'; +SELECT * FROM bf_ngrambf_map_values_test WHERE map_fixed['K0'] != 'V0' SETTINGS force_data_skipping_indices='map_fixed_values_ngrambf'; +SELECT 'Not equals with non existing key'; +SELECT * FROM bf_ngrambf_map_values_test WHERE map_fixed['K2'] != 'V2' SETTINGS force_data_skipping_indices='map_fixed_values_ngrambf'; +SELECT 'Not equals with non existing key and default value'; +SELECT * FROM bf_ngrambf_map_values_test WHERE map_fixed['K3'] != ''; + +DROP TABLE bf_ngrambf_map_values_test; From 1a4ee1f77f95d8e8e4db737a9f88988ed0d666dc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 28 Sep 2021 13:01:47 +0300 Subject: [PATCH 250/317] Remove filter column from HAVING when it is not needed. --- src/Interpreters/ExpressionAnalyzer.cpp | 38 ++++++++++++------- src/Interpreters/ExpressionAnalyzer.h | 9 ++++- src/Interpreters/InterpreterSelectQuery.cpp | 13 ++++--- src/Interpreters/InterpreterSelectQuery.h | 4 +- src/Processors/QueryPlan/TotalsHavingStep.cpp | 10 ++++- src/Processors/QueryPlan/TotalsHavingStep.h | 2 + .../Transforms/TotalsHavingTransform.cpp | 21 +++++++++- .../Transforms/TotalsHavingTransform.h | 4 +- 8 files changed, 75 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 1dd63695ad4..182553cb91b 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1481,18 +1481,15 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( const Settings & settings = context->getSettingsRef(); const ConstStoragePtr & storage = query_analyzer.storage(); - bool finalized = false; - size_t where_step_num = 0; + ssize_t prewhere_step_num = -1; + ssize_t where_step_num = -1; + ssize_t having_step_num = -1; auto finalize_chain = [&](ExpressionActionsChain & chain) { chain.finalize(); - if (!finalized) - { - finalize(chain, where_step_num, query); - finalized = true; - } + finalize(chain, prewhere_step_num, where_step_num, having_step_num, query); chain.clear(); }; @@ -1523,6 +1520,8 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere)) { + /// Prewhere is always the first one. + prewhere_step_num = 0; prewhere_info = std::make_shared(actions, query.prewhere()->getColumnName()); if (allowEarlyConstantFolding(*prewhere_info->prewhere_actions, settings)) @@ -1591,6 +1590,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (query_analyzer.appendHaving(chain, only_types || !second_stage)) { + having_step_num = chain.steps.size() - 1; before_having = chain.getLastActions(); chain.addStep(); } @@ -1691,13 +1691,16 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( checkActions(); } -void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, size_t where_step_num, const ASTSelectQuery & query) +void ExpressionAnalysisResult::finalize( + const ExpressionActionsChain & chain, + ssize_t & prewhere_step_num, + ssize_t & where_step_num, + ssize_t & having_step_num, + const ASTSelectQuery & query) { - size_t next_step_i = 0; - - if (hasPrewhere()) + if (prewhere_step_num >= 0) { - const ExpressionActionsChain::Step & step = *chain.steps.at(next_step_i++); + const ExpressionActionsChain::Step & step = *chain.steps.at(prewhere_step_num); prewhere_info->prewhere_actions->projectInput(false); NameSet columns_to_remove; @@ -1710,12 +1713,21 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si } columns_to_remove_after_prewhere = std::move(columns_to_remove); + prewhere_step_num = -1; } - if (hasWhere()) + if (where_step_num >= 0) { where_column_name = query.where()->getColumnName(); remove_where_filter = chain.steps.at(where_step_num)->required_output.find(where_column_name)->second; + where_step_num = -1; + } + + if (having_step_num >= 0) + { + having_column_name = query.having()->getColumnName(); + remove_having_filter = chain.steps.at(having_step_num)->required_output.find(having_column_name)->second; + having_step_num = -1; } } diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 5e3a7af8e9e..c785b085a57 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -229,6 +229,8 @@ struct ExpressionAnalysisResult ActionsDAGPtr before_where; ActionsDAGPtr before_aggregation; ActionsDAGPtr before_having; + String having_column_name; + bool remove_having_filter = false; ActionsDAGPtr before_window; ActionsDAGPtr before_order_by; ActionsDAGPtr before_limit_by; @@ -274,7 +276,12 @@ struct ExpressionAnalysisResult void removeExtraColumns() const; void checkActions() const; - void finalize(const ExpressionActionsChain & chain, size_t where_step_num, const ASTSelectQuery & query); + void finalize( + const ExpressionActionsChain & chain, + ssize_t & prewhere_step_num, + ssize_t & where_step_num, + ssize_t & having_step_num, + const ASTSelectQuery & query); }; /// SelectQuery specific ExpressionAnalyzer part. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5fe9948f857..e5adb03e08e 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1248,7 +1248,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu { bool final = !query.group_by_with_rollup && !query.group_by_with_cube; executeTotalsAndHaving( - query_plan, expressions.hasHaving(), expressions.before_having, aggregate_overflow_row, final); + query_plan, expressions.hasHaving(), expressions.before_having, expressions.remove_having_filter, aggregate_overflow_row, final); } if (query.group_by_with_rollup) @@ -1262,11 +1262,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu throw Exception( "WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING", ErrorCodes::NOT_IMPLEMENTED); - executeHaving(query_plan, expressions.before_having); + executeHaving(query_plan, expressions.before_having, expressions.remove_having_filter); } } else if (expressions.hasHaving()) - executeHaving(query_plan, expressions.before_having); + executeHaving(query_plan, expressions.before_having, expressions.remove_having_filter); } else if (query.group_by_with_totals || query.group_by_with_rollup || query.group_by_with_cube) throw Exception("WITH TOTALS, ROLLUP or CUBE are not supported without aggregation", ErrorCodes::NOT_IMPLEMENTED); @@ -2133,10 +2133,10 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool } -void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression) +void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter) { auto having_step - = std::make_unique(query_plan.getCurrentDataStream(), expression, getSelectQuery().having()->getColumnName(), false); + = std::make_unique(query_plan.getCurrentDataStream(), expression, getSelectQuery().having()->getColumnName(), remove_filter); having_step->setStepDescription("HAVING"); query_plan.addStep(std::move(having_step)); @@ -2144,7 +2144,7 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions void InterpreterSelectQuery::executeTotalsAndHaving( - QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool overflow_row, bool final) + QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool remove_filter, bool overflow_row, bool final) { const Settings & settings = context->getSettingsRef(); @@ -2153,6 +2153,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving( overflow_row, expression, has_having ? getSelectQuery().having()->getColumnName() : "", + remove_filter, settings.totals_mode, settings.totals_auto_threshold, final); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index aec3b0b8bd3..99c95a8d624 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -131,8 +131,8 @@ private: void executeAggregation( QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); void executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final); - void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool overflow_row, bool final); - void executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression); + void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool remove_filter, bool overflow_row, bool final); + void executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter); static void executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description); /// FIXME should go through ActionsDAG to behave as a proper function void executeWindow(QueryPlan & query_plan); diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index db82538d5a0..4cac12639a8 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -30,6 +30,7 @@ TotalsHavingStep::TotalsHavingStep( bool overflow_row_, const ActionsDAGPtr & actions_dag_, const std::string & filter_column_, + bool remove_filter_, TotalsMode totals_mode_, double auto_include_threshold_, bool final_) @@ -38,11 +39,14 @@ TotalsHavingStep::TotalsHavingStep( TotalsHavingTransform::transformHeader( input_stream_.header, actions_dag_.get(), + filter_column_, + remove_filter_, final_), getTraits(!filter_column_.empty())) , overflow_row(overflow_row_) , actions_dag(actions_dag_) , filter_column_name(filter_column_) + , remove_filter(remove_filter_) , totals_mode(totals_mode_) , auto_include_threshold(auto_include_threshold_) , final(final_) @@ -58,6 +62,7 @@ void TotalsHavingStep::transformPipeline(QueryPipeline & pipeline, const BuildQu overflow_row, expression_actions, filter_column_name, + remove_filter, totals_mode, auto_include_threshold, final); @@ -85,7 +90,10 @@ static String totalsModeToString(TotalsMode totals_mode, double auto_include_thr void TotalsHavingStep::describeActions(FormatSettings & settings) const { String prefix(settings.offset, ' '); - settings.out << prefix << "Filter column: " << filter_column_name << '\n'; + settings.out << prefix << "Filter column: " << filter_column_name; + if (remove_filter) + settings.out << " (removed)"; + settings.out << '\n'; settings.out << prefix << "Mode: " << totalsModeToString(totals_mode, auto_include_threshold) << '\n'; if (actions_dag) diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index bc053c96970..1ad98a70a01 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -18,6 +18,7 @@ public: bool overflow_row_, const ActionsDAGPtr & actions_dag_, const std::string & filter_column_, + bool remove_filter_, TotalsMode totals_mode_, double auto_include_threshold_, bool final_); @@ -35,6 +36,7 @@ private: bool overflow_row; ActionsDAGPtr actions_dag; String filter_column_name; + bool remove_filter; TotalsMode totals_mode; double auto_include_threshold; bool final; diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index 9724d332f15..8497b4b0069 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -28,13 +28,22 @@ void finalizeChunk(Chunk & chunk) chunk.setColumns(std::move(columns), num_rows); } -Block TotalsHavingTransform::transformHeader(Block block, const ActionsDAG * expression, bool final) +Block TotalsHavingTransform::transformHeader( + Block block, + const ActionsDAG * expression, + const std::string & filter_column_name, + bool remove_filter, + bool final) { if (final) finalizeBlock(block); if (expression) + { block = expression->updateHeader(std::move(block)); + if (remove_filter) + block.erase(filter_column_name); + } return block; } @@ -44,13 +53,15 @@ TotalsHavingTransform::TotalsHavingTransform( bool overflow_row_, const ExpressionActionsPtr & expression_, const std::string & filter_column_, + bool remove_filter_, TotalsMode totals_mode_, double auto_include_threshold_, bool final_) - : ISimpleTransform(header, transformHeader(header, expression_ ? &expression_->getActionsDAG() : nullptr, final_), true) + : ISimpleTransform(header, transformHeader(header, expression_ ? &expression_->getActionsDAG() : nullptr, filter_column_, remove_filter_, final_), true) , overflow_row(overflow_row_) , expression(expression_) , filter_column_name(filter_column_) + , remove_filter(remove_filter_) , totals_mode(totals_mode_) , auto_include_threshold(auto_include_threshold_) , final(final_) @@ -67,6 +78,8 @@ TotalsHavingTransform::TotalsHavingTransform( auto totals_header = finalized_header; size_t num_rows = totals_header.rows(); expression->execute(totals_header, num_rows); + if (remove_filter) + totals_header.erase(filter_column_name); outputs.emplace_back(totals_header, this); } else @@ -167,6 +180,8 @@ void TotalsHavingTransform::transform(Chunk & chunk) } expression->execute(finalized_block, num_rows); + if (remove_filter) + finalized_block.erase(filter_column_name); auto columns = finalized_block.getColumns(); ColumnPtr filter_column_ptr = columns[filter_column_pos]; @@ -270,6 +285,8 @@ void TotalsHavingTransform::prepareTotals() size_t num_rows = totals.getNumRows(); auto block = finalized_header.cloneWithColumns(totals.detachColumns()); expression->execute(block, num_rows); + if (remove_filter) + block.erase(filter_column_name); /// Note: after expression totals may have several rows if `arrayJoin` was used in expression. totals = Chunk(block.getColumns(), num_rows); } diff --git a/src/Processors/Transforms/TotalsHavingTransform.h b/src/Processors/Transforms/TotalsHavingTransform.h index d42543d311a..03635054c65 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.h +++ b/src/Processors/Transforms/TotalsHavingTransform.h @@ -28,6 +28,7 @@ public: bool overflow_row_, const ExpressionActionsPtr & expression_, const std::string & filter_column_, + bool remove_filter_, TotalsMode totals_mode_, double auto_include_threshold_, bool final_); @@ -39,7 +40,7 @@ public: Status prepare() override; void work() override; - static Block transformHeader(Block block, const ActionsDAG * expression, bool final); + static Block transformHeader(Block block, const ActionsDAG * expression, const std::string & filter_column_name, bool remove_filter, bool final); protected: void transform(Chunk & chunk) override; @@ -55,6 +56,7 @@ private: bool overflow_row; ExpressionActionsPtr expression; String filter_column_name; + bool remove_filter; TotalsMode totals_mode; double auto_include_threshold; bool final; From 462088f5001257d0cf392787c4fc014149aac85f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 28 Sep 2021 13:14:05 +0300 Subject: [PATCH 251/317] fix table name escaping --- src/Databases/DatabaseLazy.cpp | 3 +- src/Databases/DatabaseOrdinary.cpp | 2 +- .../01015_database_bad_tables.reference | 2 + .../0_stateless/01015_database_bad_tables.sh | 39 +++++++++++++++++++ .../0_stateless/01015_database_bad_tables.sql | 24 ------------ 5 files changed, 44 insertions(+), 26 deletions(-) create mode 100755 tests/queries/0_stateless/01015_database_bad_tables.sh delete mode 100644 tests/queries/0_stateless/01015_database_bad_tables.sql diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 384c5ff47dd..e6f69f797cf 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -40,7 +41,7 @@ void DatabaseLazy::loadStoredObjects( { iterateMetadataFiles(local_context, [this](const String & file_name) { - const std::string table_name = file_name.substr(0, file_name.size() - 4); + const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4)); fs::path detached_permanently_flag = fs::path(getMetadataPath()) / (file_name + detached_suffix); if (fs::exists(detached_permanently_flag)) diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 1bdb273c9fb..a122c5fc175 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -176,7 +176,7 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables /// if (create_query->uuid != UUIDHelpers::Nil) /// DatabaseCatalog::instance().addUUIDMapping(create_query->uuid); - const std::string table_name = file_name.substr(0, file_name.size() - 4); + const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4)); LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name)); return; } diff --git a/tests/queries/0_stateless/01015_database_bad_tables.reference b/tests/queries/0_stateless/01015_database_bad_tables.reference index b69cd02f83d..917f987fda7 100644 --- a/tests/queries/0_stateless/01015_database_bad_tables.reference +++ b/tests/queries/0_stateless/01015_database_bad_tables.reference @@ -1,3 +1,5 @@ 1 1 1 1 1 1 +1 1 +1 1 diff --git a/tests/queries/0_stateless/01015_database_bad_tables.sh b/tests/queries/0_stateless/01015_database_bad_tables.sh new file mode 100755 index 00000000000..d1b63cef476 --- /dev/null +++ b/tests/queries/0_stateless/01015_database_bad_tables.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e + +db="db_$CLICKHOUSE_DATABASE" +$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS $db;" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE $db;" +$CLICKHOUSE_CLIENT -q "CREATE TABLE $db.\`таблица_со_странным_названием\` (a UInt64, b UInt64) ENGINE = Log;" +$CLICKHOUSE_CLIENT -q "INSERT INTO $db.\`таблица_со_странным_названием\` VALUES (1, 1);" +$CLICKHOUSE_CLIENT -q "SELECT * FROM $db.\`таблица_со_странным_названием\`;" +$CLICKHOUSE_CLIENT -q "DETACH DATABASE $db;" +$CLICKHOUSE_CLIENT -q "ATTACH DATABASE $db;" +$CLICKHOUSE_CLIENT -q "SELECT * FROM $db.\`таблица_со_странным_названием\`;" +$CLICKHOUSE_CLIENT -q "DROP TABLE $db.\`таблица_со_странным_названием\`;" +$CLICKHOUSE_CLIENT -q "DROP DATABASE $db;" + +lazy_db="lazy_$CLICKHOUSE_DATABASE" +$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS $lazy_db;" +$CLICKHOUSE_CLIENT -q "CREATE DATABASE $lazy_db ENGINE = Lazy(1);" +$CLICKHOUSE_CLIENT -q "CREATE TABLE $lazy_db.\`таблица_со_странным_названием\` (a UInt64, b UInt64) ENGINE = Log;" +$CLICKHOUSE_CLIENT -q "INSERT INTO $lazy_db.\`таблица_со_странным_названием\` VALUES (1, 1);" +$CLICKHOUSE_CLIENT -q "SELECT * FROM $lazy_db.\`таблица_со_странным_названием\`;" +$CLICKHOUSE_CLIENT -q "DETACH DATABASE $lazy_db;" +$CLICKHOUSE_CLIENT -q "ATTACH DATABASE $lazy_db;" +$CLICKHOUSE_CLIENT -q "SELECT * FROM $lazy_db.\`таблица_со_странным_названием\`;" +$CLICKHOUSE_CLIENT -q "DROP TABLE $lazy_db.\`таблица_со_странным_названием\`;" +$CLICKHOUSE_CLIENT -q "DROP DATABASE $lazy_db;" + +$CLICKHOUSE_CLIENT -q "CREATE DATABASE $lazy_db ENGINE = Lazy(10);" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS $lazy_db.test;" +$CLICKHOUSE_CLIENT -q "CREATE TABLE IF NOT EXISTS $lazy_db.test (a UInt64, b UInt64) ENGINE = Log;" +$CLICKHOUSE_CLIENT -q "CREATE TABLE IF NOT EXISTS $lazy_db.test (a UInt64, b UInt64) ENGINE = Log;" +$CLICKHOUSE_CLIENT -q "INSERT INTO $lazy_db.test VALUES (1, 1);" +$CLICKHOUSE_CLIENT -q "SELECT * FROM $lazy_db.test;" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS $lazy_db.test;" +$CLICKHOUSE_CLIENT -q "DROP DATABASE $lazy_db;" diff --git a/tests/queries/0_stateless/01015_database_bad_tables.sql b/tests/queries/0_stateless/01015_database_bad_tables.sql deleted file mode 100644 index 5931b3b6fa5..00000000000 --- a/tests/queries/0_stateless/01015_database_bad_tables.sql +++ /dev/null @@ -1,24 +0,0 @@ --- Tags: no-parallel - -DROP DATABASE IF EXISTS testlazy; - -CREATE TABLE `таблица_со_странным_названием` (a UInt64, b UInt64) ENGINE = Log; -INSERT INTO `таблица_со_странным_названием` VALUES (1, 1); -SELECT * FROM `таблица_со_странным_названием`; -DROP TABLE `таблица_со_странным_названием`; - -CREATE DATABASE testlazy ENGINE = Lazy(1); -CREATE TABLE testlazy.`таблица_со_странным_названием` (a UInt64, b UInt64) ENGINE = Log; -INSERT INTO testlazy.`таблица_со_странным_названием` VALUES (1, 1); -SELECT * FROM testlazy.`таблица_со_странным_названием`; -DROP TABLE testlazy.`таблица_со_странным_названием`; -DROP DATABASE testlazy; - -CREATE DATABASE testlazy ENGINE = Lazy(10); -DROP TABLE IF EXISTS testlazy.test; -CREATE TABLE IF NOT EXISTS testlazy.test (a UInt64, b UInt64) ENGINE = Log; -CREATE TABLE IF NOT EXISTS testlazy.test (a UInt64, b UInt64) ENGINE = Log; -INSERT INTO testlazy.test VALUES (1, 1); -SELECT * FROM testlazy.test; -DROP TABLE IF EXISTS testlazy.test; -DROP DATABASE testlazy; From 3766d47f311ccd353e7b27af9bfbcf19a4491f9f Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 25 Jun 2021 15:03:10 +0300 Subject: [PATCH 252/317] ORs in JOINs --- src/Columns/ColumnNullable.h | 1 + src/Core/Names.h | 1 + src/Interpreters/CollectJoinOnKeysVisitor.cpp | 26 + src/Interpreters/CollectJoinOnKeysVisitor.h | 4 +- src/Interpreters/DictionaryReader.cpp | 2 +- src/Interpreters/ExpressionActions.cpp | 12 +- src/Interpreters/HashJoin.cpp | 1325 +++++++++++------ src/Interpreters/HashJoin.h | 76 +- src/Interpreters/JoinSwitcher.cpp | 10 +- .../LogicalExpressionsOptimizer.cpp | 1 + src/Interpreters/MergeJoin.cpp | 10 +- src/Interpreters/RequiredSourceColumnsData.h | 1 + .../RequiredSourceColumnsVisitor.cpp | 8 +- src/Interpreters/TableJoin.cpp | 212 ++- src/Interpreters/TableJoin.h | 41 +- src/Interpreters/TreeRewriter.cpp | 196 ++- src/Interpreters/joinDispatch.h | 31 + src/Interpreters/join_common.cpp | 35 +- src/Interpreters/join_common.h | 7 +- src/Parsers/ASTTablesInSelectQuery.h | 2 + src/Parsers/IAST_fwd.h | 1 + .../Optimizations/filterPushDown.cpp | 4 +- src/Storages/StorageJoin.cpp | 4 +- 23 files changed, 1446 insertions(+), 564 deletions(-) diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 9da7b0dac1c..700ad24d27d 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -13,6 +13,7 @@ namespace DB using NullMap = ColumnUInt8::Container; using ConstNullMapPtr = const NullMap *; +using ConstNullMapPtrVector = std::vector; /// Class that specifies nullable columns. A nullable column represents /// a column, which may have any type, provided with the possibility of diff --git a/src/Core/Names.h b/src/Core/Names.h index 3281daa560e..869530b1a16 100644 --- a/src/Core/Names.h +++ b/src/Core/Names.h @@ -11,6 +11,7 @@ namespace DB { using Names = std::vector; +using NamesVector = std::vector; using NameSet = std::unordered_set; using NameOrderedSet = std::set; using NameToNameMap = std::unordered_map; diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/src/Interpreters/CollectJoinOnKeysVisitor.cpp index 9715af01a0a..cf8e2642255 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -32,6 +32,23 @@ bool isRightIdentifier(JoinIdentifierPos pos) } +void CollectJoinOnKeysMatcher::Data::setDisjuncts(const ASTFunction & func) +{ + const auto * expression_list = func.children.front()->as(); + std::vector v; + for (const auto & child : expression_list->children) + { + v.push_back(child.get()); + } + + analyzed_join.setDisjuncts(std::move(v)); +} + +void CollectJoinOnKeysMatcher::Data::addDisjunct(const ASTFunction & func) +{ + analyzed_join.addDisjunct(static_cast(&func)); +} + void CollectJoinOnKeysMatcher::Data::addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos) { ASTPtr left = left_ast->clone(); @@ -87,6 +104,15 @@ void CollectJoinOnKeysMatcher::visit(const ASTIdentifier & ident, const ASTPtr & void CollectJoinOnKeysMatcher::visit(const ASTFunction & func, const ASTPtr & ast, Data & data) { + if (func.name == "or") + { + // throw Exception("JOIN ON does not support OR. Unexpected '" + queryToString(ast) + "'", ErrorCodes::NOT_IMPLEMENTED); + data.setDisjuncts(func); + return; + } + + data.addDisjunct(func); + if (func.name == "and") return; /// go into children diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index 0647f58f79b..58946b9fdcc 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -51,6 +51,8 @@ public: void addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos); void addAsofJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos, const ASOF::Inequality & asof_inequality); + void setDisjuncts(const ASTFunction & or_func); + void addDisjunct(const ASTFunction & func); void asofToJoinKeys(); }; @@ -73,7 +75,7 @@ public: static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { if (auto * func = node->as()) - return func->name == "and"; + return func->name == "and" || func->name == "or"; return true; } diff --git a/src/Interpreters/DictionaryReader.cpp b/src/Interpreters/DictionaryReader.cpp index fc795b9cdfb..3c66b0019ed 100644 --- a/src/Interpreters/DictionaryReader.cpp +++ b/src/Interpreters/DictionaryReader.cpp @@ -128,7 +128,7 @@ void DictionaryReader::readKeys(const IColumn & keys, Block & out_block, ColumnV found.swap(typeid_cast &>(*mutable_has).getData()); has_column.column = nullptr; - /// set mapping form source keys to resulting rows in output block + /// set mapping from source keys to resulting rows in output block positions.clear(); positions.resize(size, 0); size_t pos = 0; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 91ba4085c3e..a56f7c66bf1 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1056,11 +1056,15 @@ void ExpressionActionsChain::JoinStep::finalize(const NameSet & required_output_ /// That's an input columns we need. NameSet required_names = required_output_; - for (const auto & name : analyzed_join->keyNamesLeft()) - required_names.emplace(name); + if (!analyzed_join->keyNamesLeft().empty()) // why do we need this condition ? + for (const auto & name_part : analyzed_join->keyNamesLeft()) + for (const auto & name : name_part) + required_names.emplace(name); - if (ASTPtr extra_condition_column = analyzed_join->joinConditionColumn(JoinTableSide::Left)) - required_names.emplace(extra_condition_column->getColumnName()); + const size_t disjuncts = analyzed_join->keyNamesLeft().size(); + for (size_t d = 0; d < disjuncts; ++d) + if (ASTPtr extra_condition_column = analyzed_join->joinConditionColumn(JoinTableSide::Left, d)) + required_names.emplace(extra_condition_column->getColumnName()); for (const auto & column : required_columns) { diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 9d1480c2030..f49b2a3d826 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -75,34 +76,43 @@ namespace JoinStuff } } - template <> - void JoinUsedFlags::setUsed(size_t i [[maybe_unused]]) {} - - template <> - bool JoinUsedFlags::getUsed(size_t i [[maybe_unused]]) { return true; } - - template <> - bool JoinUsedFlags::setUsedOnce(size_t i [[maybe_unused]]) { return true; } - - template <> - void JoinUsedFlags::setUsed(size_t i) + template + void JoinUsedFlags::setUsed(const FindResult & f) { - /// Could be set simultaneously from different threads. - flags[i].store(true, std::memory_order_relaxed); + if constexpr (use_flags) + { + /// Could be set simultaneously from different threads. + if constexpr (!multiple_disjuncts) + { + flags[f.getOffset()].store(true, std::memory_order_relaxed); + } + } } - template <> - bool JoinUsedFlags::getUsed(size_t i) { return flags[i].load(); } - - template <> - bool JoinUsedFlags::setUsedOnce(size_t i) + template + bool JoinUsedFlags::getUsed(const FindResult & f) { - /// fast check to prevent heavy CAS with seq_cst order - if (flags[i].load(std::memory_order_relaxed)) - return false; + if constexpr (use_flags) + { + return flags[f.getOffset()].load(); + } + return true; + } - bool expected = false; - return flags[i].compare_exchange_strong(expected, true); + template + bool JoinUsedFlags::setUsedOnce(const FindResult & f) + { + if constexpr (use_flags) + { + size_t off = f.getOffset(); + /// fast check to prevent heavy CAS with seq_cst order + if (flags[off].load(std::memory_order_relaxed)) + return false; + + bool expected = false; + return flags[off].compare_exchange_strong(expected, true); + } + return true; } } @@ -173,12 +183,20 @@ static ColumnWithTypeAndName correctNullability(ColumnWithTypeAndName && column, return std::move(column); } +static std::string formatKeysDebug(const NamesVector & key_names) +{ + std::vector res; + for (const auto & keys : key_names) + res.emplace_back(fmt::format("{}", fmt::join(keys, ", "))); + return fmt::format("{}", fmt::join(res, " | ")); +} HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_sample_block_, bool any_take_last_row_) : table_join(table_join_) , kind(table_join->kind()) , strictness(table_join->strictness()) , key_names_right(table_join->keyNamesRight()) + , key_names_left(table_join->keyNamesLeft()) , nullable_right_side(table_join->forceNullableRight()) , nullable_left_side(table_join->forceNullableLeft()) , any_take_last_row(any_take_last_row_) @@ -187,65 +205,111 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s , right_sample_block(right_sample_block_) , log(&Poco::Logger::get("HashJoin")) { - LOG_DEBUG(log, "Right sample block: {}", right_sample_block.dumpStructure()); + bool multiple_disjuncts = key_names_right.size() > 1; - JoinCommon::splitAdditionalColumns(key_names_right, right_sample_block, right_table_keys, sample_block_with_columns_to_add); - - required_right_keys = table_join->getRequiredRightKeys(right_table_keys, required_right_keys_sources); - - LOG_DEBUG(log, "Right keys: [{}] (required: [{}]), left keys: [{}]", - fmt::join(key_names_right, ", "), - fmt::join(required_right_keys.getNames(), ", "), - fmt::join(table_join->keyNamesLeft(), ", ")); - - LOG_DEBUG(log, "Columns to add: [{}]", sample_block_with_columns_to_add.dumpStructure()); - - std::tie(condition_mask_column_name_left, condition_mask_column_name_right) = table_join->joinConditionColumnNames(); - - JoinCommon::removeLowCardinalityInplace(right_table_keys); - initRightBlockStructure(data->sample_block); - - ColumnRawPtrs key_columns = JoinCommon::extractKeysForJoin(right_table_keys, key_names_right); - - JoinCommon::createMissedColumns(sample_block_with_columns_to_add); - if (nullable_right_side) - JoinCommon::convertColumnsToNullable(sample_block_with_columns_to_add); - - if (table_join->getDictionaryReader()) + if (multiple_disjuncts) { - LOG_DEBUG(log, "Performing join over dict"); - data->type = Type::DICT; - std::get(data->maps).create(Type::DICT); - chooseMethod(key_columns, key_sizes); /// init key_sizes - } - else if (strictness == ASTTableJoin::Strictness::Asof) - { - /// @note ASOF JOIN is not INNER. It's better avoid use of 'INNER ASOF' combination in messages. - /// In fact INNER means 'LEFT SEMI ASOF' while LEFT means 'LEFT OUTER ASOF'. - if (!isLeft(kind) && !isInner(kind)) - throw Exception("Wrong ASOF JOIN type. Only ASOF and LEFT ASOF joins are supported", ErrorCodes::NOT_IMPLEMENTED); - - if (key_columns.size() <= 1) - throw Exception("ASOF join needs at least one equi-join column", ErrorCodes::SYNTAX_ERROR); - - if (right_table_keys.getByName(key_names_right.back()).type->isNullable()) - throw Exception("ASOF join over right table Nullable column is not implemented", ErrorCodes::NOT_IMPLEMENTED); - - size_t asof_size; - asof_type = AsofRowRefs::getTypeSize(*key_columns.back(), asof_size); - key_columns.pop_back(); - - /// this is going to set up the appropriate hash table for the direct lookup part of the join - /// However, this does not depend on the size of the asof join key (as that goes into the BST) - /// Therefore, add it back in such that it can be extracted appropriately from the full stored - /// key_columns and key_sizes - init(chooseMethod(key_columns, key_sizes)); - key_sizes.push_back(asof_size); + // required_right_keys_sources concept does not work well if multiple disjuncts + sample_block_with_columns_to_add = right_table_keys = materializeBlock(right_sample_block); } else { - /// Choose data structure to use for JOIN. - init(chooseMethod(key_columns, key_sizes)); + JoinCommon::splitAdditionalColumns(key_names_right, right_sample_block, right_table_keys, sample_block_with_columns_to_add); + required_right_keys = table_join->getRequiredRightKeys(right_table_keys, required_right_keys_sources); + } + + LOG_DEBUG(log, "Right keys: [{}] (required: [{}]), left keys: [{}]", + formatKeysDebug(key_names_right), + fmt::join(required_right_keys.getNames(), ", "), + formatKeysDebug(key_names_left)); + + LOG_DEBUG(log, "Columns to add: [{}]", sample_block_with_columns_to_add.dumpStructure()); + + JoinCommon::removeLowCardinalityInplace(right_table_keys); + key_sizes.resize(key_names_right.size()); + + Type join_method = Type::EMPTY; + + initRightBlockStructure(data->sample_block); + + + const size_t disjuncts_num = key_names_right.size(); + JoinCommon::createMissedColumns(sample_block_with_columns_to_add); + + if (table_join->getDictionaryReader()) + { + data->maps.resize(disjuncts_num); + } + condition_mask_column_name_left.resize(disjuncts_num); + condition_mask_column_name_right.resize(disjuncts_num); + + if (nullable_right_side) + { + JoinCommon::convertColumnsToNullable(sample_block_with_columns_to_add); + } + + for (size_t d = 0; d < disjuncts_num; ++d) + { + std::tie(condition_mask_column_name_left[d], condition_mask_column_name_right[d]) = table_join->joinConditionColumnNames(d); + ColumnRawPtrs key_columns = JoinCommon::extractKeysForJoin(right_table_keys, key_names_right[d]); + + if (table_join->dictionary_reader) + { + LOG_DEBUG(log, "Performing join over dict"); + join_method = Type::DICT; + + std::get(data->maps[d]).create(Type::DICT); + chooseMethod(key_columns, key_sizes[d]); /// init key_sizes + continue; // break ? + } + else if (strictness == ASTTableJoin::Strictness::Asof) + { + /// @note ASOF JOIN is not INNER. It's better avoid use of 'INNER ASOF' combination in messages. + /// In fact INNER means 'LEFT SEMI ASOF' while LEFT means 'LEFT OUTER ASOF'. + if (!isLeft(kind) && !isInner(kind)) + throw Exception("Wrong ASOF JOIN type. Only ASOF and LEFT ASOF joins are supported", ErrorCodes::NOT_IMPLEMENTED); + + if (key_columns.size() <= 1) + throw Exception("ASOF join needs at least one equi-join column", ErrorCodes::SYNTAX_ERROR); + + if (right_table_keys.getByName(key_names_right[0].back()).type->isNullable()) + throw Exception("ASOF join over right table Nullable column is not implemented", ErrorCodes::NOT_IMPLEMENTED); + + size_t asof_size; + asof_type = AsofRowRefs::getTypeSize(*key_columns.back(), asof_size); + key_columns.pop_back(); + + /// this is going to set up the appropriate hash table for the direct lookup part of the join + /// However, this does not depend on the size of the asof join key (as that goes into the BST) + /// Therefore, add it back in such that it can be extracted appropriately from the full stored + /// key_columns and key_sizes + key_sizes[d].push_back(asof_size); + } + else + { + /// Choose data structure to use for JOIN. + } + + auto current_join_method = chooseMethod(key_columns, key_sizes[d]); + if (join_method == Type::EMPTY) + { + join_method = current_join_method; + } + else if (join_method != current_join_method) + { + join_method = Type::hashed; + } + } + + data->type = join_method; + if (join_method != Type::DICT) + { + data->maps.resize(key_names_right.size()); + + for (size_t d = 0; d < disjuncts_num; ++d) + { + data_map_init(data->maps[d]); + } } } @@ -339,7 +403,7 @@ public: JoinCommon::convertColumnToNullable(column); } - FindResult findKey(void *, size_t row, const Arena &) + FindResult findKey(const TableJoin & /* void * */, size_t row, const Arena &) { result.block = &read_result; result.row_num = positions[row]; @@ -404,15 +468,12 @@ struct KeyGetterForType using Type = typename KeyGetterForTypeImpl::Type; }; - -void HashJoin::init(Type type_) +void HashJoin::data_map_init(MapsVariant & map) { - data->type = type_; - if (kind == ASTTableJoin::Kind::Cross) return; - joinDispatchInit(kind, strictness, data->maps); - joinDispatch(kind, strictness, data->maps, [&](auto, auto, auto & map) { map.create(data->type); }); + joinDispatchInit(kind, strictness, map); + joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.create(data->type); }); } bool HashJoin::overDictionary() const @@ -437,11 +498,14 @@ size_t HashJoin::getTotalRowCount() const if (data->type == Type::CROSS) { for (const auto & block : data->blocks) - res += block.rows(); + res += block.block.rows(); } else if (data->type != Type::DICT) { - joinDispatch(kind, strictness, data->maps, [&](auto, auto, auto & map) { res += map.getTotalRowCount(data->type); }); + for (const auto & map : data->maps) + { + joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalRowCount(data->type); }); + } } return res; @@ -454,11 +518,14 @@ size_t HashJoin::getTotalByteCount() const if (data->type == Type::CROSS) { for (const auto & block : data->blocks) - res += block.bytes(); + res += block.block.bytes(); } else if (data->type != Type::DICT) { - joinDispatch(kind, strictness, data->maps, [&](auto, auto, auto & map) { res += map.getTotalByteCountImpl(data->type); }); + for (const auto & map : data->maps) + { + joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalByteCountImpl(data->type); }); + } res += data->pool.size(); } @@ -533,9 +600,13 @@ namespace if constexpr (is_asof_join) Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); else if constexpr (mapped_one) + { Inserter::insertOne(join, map, key_getter, stored_block, i, pool); + } else + { Inserter::insertAll(join, map, key_getter, stored_block, i, pool); + } } return map.getBufferSizeInCells(); } @@ -581,7 +652,7 @@ namespace void HashJoin::initRightBlockStructure(Block & saved_block_sample) { /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). - bool save_key_columns = !table_join->forceHashJoin() || isRightOrFull(kind); + bool save_key_columns = !table_join->forceHashJoin() || isRightOrFull(kind) || key_names_right.size() > 1; if (save_key_columns) { saved_block_sample = right_table_keys.cloneEmpty(); @@ -594,21 +665,29 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) /// Save non key columns for (auto & column : sample_block_with_columns_to_add) - saved_block_sample.insert(column); + { + if (!saved_block_sample.findByName(column.name)) + { + saved_block_sample.insert(column); + } + } if (nullable_right_side) + { JoinCommon::convertColumnsToNullable(saved_block_sample, (isFull(kind) ? right_table_keys.columns() : 0)); + } + } -Block HashJoin::structureRightBlock(const Block & block) const +HashJoin::BlockWithFlags HashJoin::structureRightBlock(const Block & block) const { - Block structured_block; + BlockWithFlags structured_block; for (const auto & sample_column : savedBlockSample().getColumnsWithTypeAndName()) { ColumnWithTypeAndName column = block.getByName(sample_column.name); if (sample_column.column->isNullable()) JoinCommon::convertColumnToNullable(column); - structured_block.insert(column); + structured_block.block.insert(column); } return structured_block; @@ -630,47 +709,88 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) Block block = materializeBlock(source_block); size_t rows = block.rows(); - ColumnRawPtrs key_columns = JoinCommon::materializeColumnsInplace(block, key_names_right); + size_t total_rows = 0; + size_t total_bytes = 0; - /// We will insert to the map only keys, where all components are not NULL. - ConstNullMapPtr null_map{}; - ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); + // Collect all keys in all_key_names_right + // and lists of indexes in this vector for all disjuncts + Names all_key_names_right = key_names_right.front(); + const size_t disjuncts_num = key_names_right.size(); + std::vector> key_names_right_indexes(disjuncts_num); + key_names_right_indexes[0].resize(all_key_names_right.size()); + std::iota(std::begin(key_names_right_indexes[0]), std::end(key_names_right_indexes[0]), 0); - /// If RIGHT or FULL save blocks with nulls for NotJoinedBlocks - UInt8 save_nullmap = 0; - if (isRightOrFull(kind) && null_map) + for (size_t d = 1; d < disjuncts_num; ++d) { - /// Save rows with NULL keys - for (size_t i = 0; !save_nullmap && i < null_map->size(); ++i) - save_nullmap |= (*null_map)[i]; - } - - auto join_mask_col = JoinCommon::getColumnAsMask(block, condition_mask_column_name_right); - - /// Save blocks that do not hold conditions in ON section - ColumnUInt8::MutablePtr not_joined_map = nullptr; - if (isRightOrFull(kind) && join_mask_col) - { - const auto & join_mask = assert_cast(*join_mask_col).getData(); - /// Save rows that do not hold conditions - not_joined_map = ColumnUInt8::create(block.rows(), 0); - for (size_t i = 0, sz = join_mask.size(); i < sz; ++i) + for (size_t i = 0; i < key_names_right[d].size(); ++i) { - /// Condition hold, do not save row - if (join_mask[i]) - continue; - - /// NULL key will be saved anyway because, do not save twice - if (save_nullmap && (*null_map)[i]) - continue; - - not_joined_map->getData()[i] = 1; + auto it = std::find(std::cbegin(all_key_names_right), std::cend(all_key_names_right), key_names_right[d][i]); + if (it == std::cend(all_key_names_right)) + { + key_names_right_indexes[d].push_back(all_key_names_right.size()); + all_key_names_right.push_back(key_names_right[d][i]); + } + else + { + key_names_right_indexes[d].push_back(std::distance(std::cbegin(all_key_names_right), it)); + } } } - Block structured_block = structureRightBlock(block); - size_t total_rows = 0; - size_t total_bytes = 0; + ColumnRawPtrs all_key_columns = JoinCommon::materializeColumnsInplace(block, all_key_names_right); + + BlockWithFlags structured_block = structureRightBlock(block); + bool multiple_disjuncts = disjuncts_num > 1; + // if (nullable_right_side && multiple_disjuncts) + // { + // JoinCommon::convertColumnsToNullable(structured_block.block); + // } + std::vector join_mask_col_vector(disjuncts_num); + // std::vector join_mask_vector(disjuncts_num); + bool use_join_mask_col = false; + for (size_t d = 0; d < disjuncts_num; ++d) + { + join_mask_col_vector[d] = JoinCommon::getColumnAsMask(block, condition_mask_column_name_right[d]); + // join_mask_vector[d] = assert_cast(*(join_mask_col_vector[d])).getData(); + if (join_mask_col_vector[d]) + use_join_mask_col = true; + } + + + /// Save blocks that do not hold conditions in ON section + ColumnUInt8::MutablePtr not_joined_map = nullptr; + if (isRightOrFull(kind) && use_join_mask_col) + { + /// Save rows that do not hold conditions + not_joined_map = ColumnUInt8::create(block.rows(), 0); + const size_t sz = assert_cast(*(join_mask_col_vector[0])).getData().size(); + for (size_t i = 0; i < sz; ++i) + { + bool add_to_not_joined_map = true; + + /// Condition hold, do not save row + for (size_t d = 0; d < disjuncts_num; ++d) + { + const auto & join_mask = assert_cast(*(join_mask_col_vector[d])).getData(); + if (join_mask[i]) + { + add_to_not_joined_map = false; + break; + } + } + + + // !!! + // /// NULL key will be saved anyway because, do not save twice + // if (save_nullmap && (*null_map)[i]) + // continue; + if (add_to_not_joined_map) + not_joined_map->getData()[i] = 1; + } + } + + std::vector null_map_vector; + Columns null_map_holder_vector; { if (storage_join_lock.mutex()) @@ -678,44 +798,77 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) ErrorCodes::LOGICAL_ERROR); data->blocks.emplace_back(std::move(structured_block)); - Block * stored_block = &data->blocks.back(); + BlockWithFlags & stored_block_with_flags = data->blocks.back(); + Block * stored_block = &stored_block_with_flags.block; + stored_block_with_flags.flags = std::vector(stored_block->rows()); if (rows) data->empty = false; - if (kind != ASTTableJoin::Kind::Cross) + bool save_a_nullmap = false; + + for (size_t d = 0; d < disjuncts_num; ++d) { - joinDispatch(kind, strictness, data->maps, [&](auto kind_, auto strictness_, auto & map) + ColumnRawPtrs key_columns(key_names_right_indexes[d].size()); + std::transform(std::cbegin(key_names_right_indexes[d]), std::cend(key_names_right_indexes[d]), std::begin(key_columns), [&](size_t ind){return all_key_columns[ind];}); + + /// We will insert to the map only keys, where all components are not NULL. + + null_map_vector.emplace_back(); + null_map_holder_vector.push_back(extractNestedColumnsAndNullMap(key_columns, null_map_vector.back())); + + + /// If RIGHT or FULL save blocks with nulls for NonJoinedBlockInputStream + UInt8 save_nullmap = 0; + if (isRightOrFull(kind) && null_map_vector.back()) { - size_t size = insertFromBlockImpl( - *this, data->type, map, rows, key_columns, key_sizes, stored_block, null_map, - join_mask_col ? &assert_cast(*join_mask_col).getData() : nullptr, - data->pool); - /// Number of buckets + 1 value from zero storage - used_flags.reinit(size + 1); - }); + for (size_t i = 0; !save_nullmap && i < null_map_vector.back()->size(); ++i) + save_nullmap |= (*null_map_vector.back())[i]; + } + save_a_nullmap |= save_nullmap; + + { + if (kind != ASTTableJoin::Kind::Cross) + { + joinDispatch(kind, strictness, data->maps[d], [&](auto kind_, auto strictness_, auto & map) + { + size_t size = insertFromBlockImpl( + *this, data->type, map, rows, key_columns, key_sizes[d], stored_block, null_map_vector.back(), + join_mask_col_vector[d] ? &assert_cast(*join_mask_col_vector[d]).getData() : nullptr, + data->pool); + /// Number of buckets + 1 value from zero storage + + if (!d) + { + used_flags.reinit(size + 1); + } + }); + } + + if (not_joined_map) + data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map)); + + if (!check_limits) + return true; + + /// TODO: Do not calculate them every time + total_rows = getTotalRowCount(); + total_bytes = getTotalByteCount(); + } } - if (save_nullmap) - data->blocks_nullmaps.emplace_back(stored_block, null_map_holder); - if (not_joined_map) - data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map)); - - if (!check_limits) - return true; - - /// TODO: Do not calculate them every time - total_rows = getTotalRowCount(); - total_bytes = getTotalByteCount(); + if (save_a_nullmap && !multiple_disjuncts) + { + data->blocks_nullmaps.emplace_back(stored_block, null_map_holder_vector[0]); + } } return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); } - -namespace -{ +using ColumnRawPtrsVector = std::vector; +using SizesVector = std::vector; class AddedColumns { @@ -737,9 +890,9 @@ public: const Block & block, const Block & saved_block_sample, const HashJoin & join, - const ColumnRawPtrs & key_columns_, - const Sizes & key_sizes_, - const UInt8ColumnDataPtr & join_mask_column_, + const ColumnRawPtrsVector & key_columns_, + const SizesVector & key_sizes_, + const std::vector & join_mask_column_, bool is_asof_join, bool is_join_get_) : key_columns(key_columns_) @@ -772,7 +925,7 @@ public: { const ColumnWithTypeAndName & right_asof_column = join.rightAsofKeyColumn(); addColumn(right_asof_column, right_asof_column.name); - left_asof_key = key_columns.back(); + left_asof_key = key_columns.front().back(); } for (auto & tn : type_name) @@ -832,13 +985,24 @@ public: ASOF::Inequality asofInequality() const { return asof_inequality; } const IColumn & leftAsofKey() const { return *left_asof_key; } - bool isRowFiltered(size_t i) { return join_mask_column && !(*join_mask_column)[i]; } + bool isRowFiltered(size_t i, size_t d) + { + if (join_mask_column[d]) + { - const ColumnRawPtrs & key_columns; - const Sizes & key_sizes; + UInt8ColumnDataPtr jmc = &assert_cast(*(join_mask_column[d])).getData(); + return !(*jmc)[i]; + } + + return false; + } + + const ColumnRawPtrsVector key_columns; + const SizesVector key_sizes; size_t rows_to_add; std::unique_ptr offsets_to_replicate; bool need_filter = false; + IColumn::Filter row_filter; private: std::vector type_name; @@ -849,7 +1013,7 @@ private: std::optional asof_type; ASOF::Inequality asof_inequality; const IColumn * left_asof_key = nullptr; - UInt8ColumnDataPtr join_mask_column; + std::vector join_mask_column; bool is_join_get; void addColumn(const ColumnWithTypeAndName & src_column, const std::string & qualified_name) @@ -859,17 +1023,145 @@ private: type_name.emplace_back(src_column.type, src_column.name, qualified_name); } }; +using AddedColumnsV = std::vector>; -template -void addFoundRowAll(const typename Map::mapped_type & mapped, AddedColumns & added, IColumn::Offset & current_offset) +namespace +{ +template +struct JoinFeatures +{ + static constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any; + static constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All; + static constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof; + static constexpr bool is_semi_join = STRICTNESS == ASTTableJoin::Strictness::Semi; + static constexpr bool is_anti_join = STRICTNESS == ASTTableJoin::Strictness::Anti; + + static constexpr bool left = KIND == ASTTableJoin::Kind::Left; + static constexpr bool right = KIND == ASTTableJoin::Kind::Right; + static constexpr bool inner = KIND == ASTTableJoin::Kind::Inner; + static constexpr bool full = KIND == ASTTableJoin::Kind::Full; + + static constexpr bool need_replication = is_all_join || (is_any_join && right) || (is_semi_join && right); + static constexpr bool need_filter = !need_replication && (inner || right || (is_semi_join && left) || (is_anti_join && left)); + static constexpr bool add_missing = (left || full) && !is_semi_join; + + static constexpr bool need_flags = MapGetter::flagged; + +}; + +template +class KnownRowsHolder; + +// TODO: helper to clean, instead of recreating + +template<> +class KnownRowsHolder +{ +public: + using Type = std::pair; + +private: + static const size_t MAX_LINEAR = 16; + using LinearHolder = std::array; + using LogHolder = std::set; + using LogHolderPtr = std::unique_ptr; + + LinearHolder linh; + LogHolderPtr logh_ptr; + + size_t items; + +public: + KnownRowsHolder() + : items(0) + { + } + + + template + void add(InputIt from, InputIt to) + { + size_t new_items = std::distance(from, to); + if (items + new_items <= MAX_LINEAR) + { + std::copy(from, to, &linh[items]); + } + else + { + if (items <= MAX_LINEAR) + { + logh_ptr = std::make_unique(); + logh_ptr->insert(std::cbegin(linh), std::cbegin(linh) + items); + } + logh_ptr->insert(from, to); + } + items += new_items; + } + + template + bool isKnown(const Needle & needle) + { + return items <= MAX_LINEAR + ? std::find(std::cbegin(linh), std::cbegin(linh) + items, needle) != std::cbegin(linh) + items + : logh_ptr->find(needle) != logh_ptr->end(); + } +}; + +template<> +class KnownRowsHolder +{ +public: + template + void add(InputIt, InputIt) + { + } + + template + static bool isKnown(const Needle &) + { + return false; + } +}; + +template +void addFoundRowAll(const typename Map::mapped_type & mapped, AddedColumns & added, IColumn::Offset & current_offset, KnownRowsHolder & known_rows [[maybe_unused]]) { if constexpr (add_missing) added.applyLazyDefaults(); - for (auto it = mapped.begin(); it.ok(); ++it) + if constexpr (multiple_disjuncts) { - added.appendFromBlock(*it->block, it->row_num); - ++current_offset; + std::unique_ptr::Type>> new_known_rows_ptr; + + for (auto it = mapped.begin(); it.ok(); ++it) + { + if (!known_rows.isKnown(std::make_pair(it->block, it->row_num))) + { + added.appendFromBlock(*it->block, it->row_num); + ++current_offset; + if (!new_known_rows_ptr) + { + new_known_rows_ptr = std::make_unique::Type>>(); + } + new_known_rows_ptr->push_back(std::make_pair(it->block, it->row_num)); + const HashJoin::BlockWithFlags * block_with_flags = reinterpret_cast(it->block); + + block_with_flags->flags[it->row_num].store(true, std::memory_order_relaxed); + } + } + + if (new_known_rows_ptr) + { + known_rows.add(std::cbegin(*new_known_rows_ptr), std::cend(*new_known_rows_ptr)); + } + } + else + { + for (auto it = mapped.begin(); it.ok(); ++it) + { + added.appendFromBlock(*it->block, it->row_num); + ++current_offset; + } } }; @@ -891,30 +1183,17 @@ void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unuse filter[pos] = 1; } - /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). -template +template NO_INLINE IColumn::Filter joinRightColumns( - KeyGetter && key_getter, - const Map & map, + std::vector && key_getter_vector, + const std::vector & mapv, AddedColumns & added_columns, - const ConstNullMapPtr & null_map [[maybe_unused]], + const std::vector & null_map [[maybe_unused]], JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]]) { - constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any; - constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All; - constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof; - constexpr bool is_semi_join = STRICTNESS == ASTTableJoin::Strictness::Semi; - constexpr bool is_anti_join = STRICTNESS == ASTTableJoin::Strictness::Anti; - constexpr bool left = KIND == ASTTableJoin::Kind::Left; - constexpr bool right = KIND == ASTTableJoin::Kind::Right; - constexpr bool full = KIND == ASTTableJoin::Kind::Full; - - constexpr bool need_flags = MapGetter::flagged; - - constexpr bool add_missing = (left || full) && !is_semi_join; - constexpr bool need_replication = is_all_join || (is_any_join && right) || (is_semi_join && right); + JoinFeatures jf; size_t rows = added_columns.rows_to_add; IColumn::Filter filter; @@ -923,141 +1202,195 @@ NO_INLINE IColumn::Filter joinRightColumns( Arena pool; - if constexpr (need_replication) + if constexpr (jf.need_replication) added_columns.offsets_to_replicate = std::make_unique(rows); + size_t disjunct_num = added_columns.key_columns.size(); + + // std::vector key_getter_vector; + + // for (size_t d = 0; d < disjunct_num; ++d) + // { + // auto key_getter = createKeyGetter(added_columns.key_columns[d], added_columns.key_sizes[d]); + // key_getter_vector.push_back(std::move(key_getter)); + // } + IColumn::Offset current_offset = 0; for (size_t i = 0; i < rows; ++i) { + bool right_row_found = false; + bool null_element_found = false; + + KnownRowsHolder known_rows; + size_t d = 0; + do + { + if constexpr (has_null_map) + { + if (null_map[d] && (*null_map[d])[i]) + { + null_element_found = true; + continue; + } + } + + bool row_acceptable = !added_columns.isRowFiltered(i, d); + using FindResult = typename KeyGetter::FindResult; + auto find_result = row_acceptable ? key_getter_vector[d].findKey(*(mapv[d]), i, pool) : FindResult(); + + if (find_result.isFound()) + { + right_row_found = true; + auto & mapped = find_result.getMapped(); + if constexpr (jf.is_asof_join) + { + TypeIndex asof_type = added_columns.asofType(); + ASOF::Inequality asof_inequality = added_columns.asofInequality(); + const IColumn & left_asof_key = added_columns.leftAsofKey(); + + if (const RowRef * found = mapped.findAsof(asof_type, asof_inequality, left_asof_key, i)) + { + setUsed(filter, i); + used_flags.template setUsed(find_result); + added_columns.appendFromBlock(*found->block, found->row_num); + } + else + addNotFoundRow(added_columns, current_offset); + } + else if constexpr (jf.is_all_join) + { + setUsed(filter, i); + used_flags.template setUsed(find_result); + addFoundRowAll(mapped, added_columns, current_offset, known_rows); + } + else if constexpr ((jf.is_any_join || jf.is_semi_join) && jf.right) + { + /// Use first appeared left key + it needs left columns replication + bool used_once = used_flags.template setUsedOnce(find_result); + + if (used_once) + { + setUsed(filter, i); + addFoundRowAll(mapped, added_columns, current_offset, known_rows); + } + } + else if constexpr (jf.is_any_join && KIND == ASTTableJoin::Kind::Inner) + { + bool used_once = used_flags.template setUsedOnce(find_result); + + /// Use first appeared left key only + if (used_once) + { + setUsed(filter, i); + added_columns.appendFromBlock(*mapped.block, mapped.row_num); + } + + break; + } + else if constexpr (jf.is_any_join && jf.full) + { + /// TODO + } + else if constexpr (jf.is_anti_join) + { + if constexpr (jf.right && jf.need_flags) + used_flags.template setUsed(find_result); + } + else /// ANY LEFT, SEMI LEFT, old ANY (RightAny) + { + setUsed(filter, i); + used_flags.template setUsed(find_result); + added_columns.appendFromBlock(*mapped.block, mapped.row_num); + if constexpr (multiple_disjuncts) + { + const HashJoin::BlockWithFlags * block_with_flags = reinterpret_cast(mapped.block); + block_with_flags->flags[mapped.row_num].store(true, std::memory_order_relaxed); + } + + + if (jf.is_any_join) + { + break; + } + } + } + } while (multiple_disjuncts && ++d < disjunct_num); + if constexpr (has_null_map) { - if ((*null_map)[i]) + if (!right_row_found && null_element_found) { - addNotFoundRow(added_columns, current_offset); + addNotFoundRow(added_columns, current_offset); - if constexpr (need_replication) + if constexpr (jf.need_replication) + { (*added_columns.offsets_to_replicate)[i] = current_offset; + } continue; } } - bool row_acceptable = !added_columns.isRowFiltered(i); - using FindResult = typename KeyGetter::FindResult; - auto find_result = row_acceptable ? key_getter.findKey(map, i, pool) : FindResult(); - - if (find_result.isFound()) + if (!right_row_found) { - auto & mapped = find_result.getMapped(); - - if constexpr (is_asof_join) - { - TypeIndex asof_type = added_columns.asofType(); - ASOF::Inequality asof_inequality = added_columns.asofInequality(); - const IColumn & left_asof_key = added_columns.leftAsofKey(); - - if (const RowRef * found = mapped.findAsof(asof_type, asof_inequality, left_asof_key, i)) - { - setUsed(filter, i); - used_flags.template setUsed(find_result.getOffset()); - added_columns.appendFromBlock(*found->block, found->row_num); - } - else - addNotFoundRow(added_columns, current_offset); - } - else if constexpr (is_all_join) - { + if constexpr (jf.is_anti_join && jf.left) setUsed(filter, i); - used_flags.template setUsed(find_result.getOffset()); - addFoundRowAll(mapped, added_columns, current_offset); - } - else if constexpr ((is_any_join || is_semi_join) && right) - { - /// Use first appeared left key + it needs left columns replication - bool used_once = used_flags.template setUsedOnce(find_result.getOffset()); - - if (used_once) - { - setUsed(filter, i); - addFoundRowAll(mapped, added_columns, current_offset); - } - } - else if constexpr (is_any_join && KIND == ASTTableJoin::Kind::Inner) - { - bool used_once = used_flags.template setUsedOnce(find_result.getOffset()); - - /// Use first appeared left key only - if (used_once) - { - setUsed(filter, i); - added_columns.appendFromBlock(*mapped.block, mapped.row_num); - } - } - else if constexpr (is_any_join && full) - { - /// TODO - } - else if constexpr (is_anti_join) - { - if constexpr (right && need_flags) - used_flags.template setUsed(find_result.getOffset()); - } - else /// ANY LEFT, SEMI LEFT, old ANY (RightAny) - { - setUsed(filter, i); - used_flags.template setUsed(find_result.getOffset()); - added_columns.appendFromBlock(*mapped.block, mapped.row_num); - } - } - else - { - if constexpr (is_anti_join && left) - setUsed(filter, i); - addNotFoundRow(added_columns, current_offset); + addNotFoundRow(added_columns, current_offset); } - if constexpr (need_replication) + if constexpr (jf.need_replication) + { (*added_columns.offsets_to_replicate)[i] = current_offset; + } } added_columns.applyLazyDefaults(); return filter; } +template +IColumn::Filter joinRightColumnsSwitchMultipleDisjuncts( + std::vector && key_getter_vector, + const std::vector & mapv, + AddedColumns & added_columns, + const std::vector & null_map [[maybe_unused]], + JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]]) +{ + return mapv.size() > 1 + ? joinRightColumns(std::forward>(key_getter_vector), mapv, added_columns, null_map, used_flags) + : joinRightColumns(std::forward>(key_getter_vector), mapv, added_columns, null_map, used_flags); +} + template IColumn::Filter joinRightColumnsSwitchNullability( - KeyGetter && key_getter, - const Map & map, - AddedColumns & added_columns, - const ConstNullMapPtr & null_map, - JoinStuff::JoinUsedFlags & used_flags) + std::vector && key_getter_vector, + const std::vector/***/ & mapv, + AddedColumns & added_columns, + const std::vector & null_map, + JoinStuff::JoinUsedFlags & used_flags) { if (added_columns.need_filter) { - if (null_map) - return joinRightColumns( - std::forward(key_getter), map, added_columns, null_map, used_flags); + if (!null_map.empty()) + return joinRightColumnsSwitchMultipleDisjuncts(std::forward>(key_getter_vector), mapv, added_columns, null_map, used_flags); else - return joinRightColumns( - std::forward(key_getter), map, added_columns, nullptr, used_flags); + return joinRightColumnsSwitchMultipleDisjuncts(std::forward>(key_getter_vector), mapv, added_columns, null_map, used_flags); } else { - if (null_map) - return joinRightColumns( - std::forward(key_getter), map, added_columns, null_map, used_flags); + if (!null_map.empty()) + return joinRightColumnsSwitchMultipleDisjuncts(std::forward>(key_getter_vector), mapv, added_columns, null_map, used_flags); else - return joinRightColumns( - std::forward(key_getter), map, added_columns, nullptr, used_flags); + return joinRightColumnsSwitchMultipleDisjuncts(std::forward>(key_getter_vector), mapv, added_columns, null_map, used_flags); } } template IColumn::Filter switchJoinRightColumns( - const Maps & maps_, + const std::vector & mapv, AddedColumns & added_columns, HashJoin::Type type, - const ConstNullMapPtr & null_map, + const std::vector & null_map, JoinStuff::JoinUsedFlags & used_flags) { constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof; @@ -1065,12 +1398,20 @@ IColumn::Filter switchJoinRightColumns( { #define M(TYPE) \ case HashJoin::Type::TYPE: \ - { \ - using KeyGetter = typename KeyGetterForType>::Type; \ - auto key_getter = createKeyGetter(added_columns.key_columns, added_columns.key_sizes); \ + { \ + using AMapTypeVal = const typename std::remove_reference_t::element_type; \ + using KeyGetter = typename KeyGetterForType::Type; \ + std::vector a_map_type_vector(mapv.size()); \ + std::vector key_getter_vector; \ + size_t disjunct_num = added_columns.key_columns.size(); \ + for (size_t d = 0; d < disjunct_num; ++d) \ + { \ + a_map_type_vector[d] = mapv[d]->TYPE.get(); \ + key_getter_vector.push_back(std::move(createKeyGetter(added_columns.key_columns[d], added_columns.key_sizes[d]))); \ + } \ return joinRightColumnsSwitchNullability( \ - std::move(key_getter), *maps_.TYPE, added_columns, null_map, used_flags); \ - } + std::move(key_getter_vector), a_map_type_vector, added_columns, null_map, used_flags); \ + } APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -1089,10 +1430,21 @@ IColumn::Filter dictionaryJoinRightColumns(const TableJoin & table_join, AddedCo { assert(added_columns.key_columns.size() == 1); + // JoinStuff::JoinUsedFlags flags; + // KeyGetterForDict key_getter(table_join, added_columns.key_columns); + // return joinRightColumnsSwitchNullability( + // std::move(key_getter), nullptr, added_columns, null_map, flags); + std::vector maps_vector; + maps_vector.push_back(&table_join); + + std::vector null_maps_vector; + null_maps_vector.push_back(null_map); + JoinStuff::JoinUsedFlags flags; - KeyGetterForDict key_getter(table_join, added_columns.key_columns); - return joinRightColumnsSwitchNullability( - std::move(key_getter), nullptr, added_columns, null_map, flags); + std::vector key_getter_vector; + key_getter_vector.push_back(KeyGetterForDict(table_join, added_columns.key_columns[0])); + // KeyGetterForDict key_getter(table_join, added_columns.key_columns); + return joinRightColumnsSwitchNullability(std::move(key_getter_vector), maps_vector, added_columns, null_maps_vector, flags); } throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", STRICTNESS, KIND); @@ -1100,44 +1452,43 @@ IColumn::Filter dictionaryJoinRightColumns(const TableJoin & table_join, AddedCo } /// nameless - template -void HashJoin::joinBlockImpl( +std::unique_ptr HashJoin::makeAddedColumns( Block & block, - const Names & key_names_left, + const NamesVector & key_names_left_vector, const Block & block_with_columns_to_add, - const Maps & maps_, + const std::vector & maps_, bool is_join_get) const { - constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any; - constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All; - constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof; - constexpr bool is_semi_join = STRICTNESS == ASTTableJoin::Strictness::Semi; - constexpr bool is_anti_join = STRICTNESS == ASTTableJoin::Strictness::Anti; + constexpr JoinFeatures jf; - constexpr bool left = KIND == ASTTableJoin::Kind::Left; - constexpr bool right = KIND == ASTTableJoin::Kind::Right; - constexpr bool inner = KIND == ASTTableJoin::Kind::Inner; - constexpr bool full = KIND == ASTTableJoin::Kind::Full; + ColumnRawPtrsVector left_key_columns_vector; + std::vector null_map_vector; + std::vector null_map_holder_vector; + std::vector materialized_keys_vector; + std::vector join_mask_column_vector; /// Only rows where mask == true can be joined - constexpr bool need_replication = is_all_join || (is_any_join && right) || (is_semi_join && right); - constexpr bool need_filter = !need_replication && (inner || right || (is_semi_join && left) || (is_anti_join && left)); + size_t disjunct = 0; - /// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them. - Columns materialized_keys = JoinCommon::materializeColumns(block, key_names_left); - ColumnRawPtrs left_key_columns = JoinCommon::getRawPointers(materialized_keys); + for (const auto & key_names_left_part : key_names_left_vector) + { + /// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them. + materialized_keys_vector.emplace_back(JoinCommon::materializeColumns(block, key_names_left_part)); + ColumnRawPtrs left_key_columns = JoinCommon::getRawPointers(materialized_keys_vector.back()); + left_key_columns_vector.push_back(std::move(left_key_columns)); - /// Keys with NULL value in any column won't join to anything. - ConstNullMapPtr null_map{}; - ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(left_key_columns, null_map); + /// Keys with NULL value in any column won't join to anything. + null_map_vector.emplace_back(); + null_map_holder_vector.push_back(extractNestedColumnsAndNullMap(left_key_columns_vector.back(), null_map_vector.back())); - size_t existing_columns = block.columns(); + join_mask_column_vector.push_back(JoinCommon::getColumnAsMask(block, condition_mask_column_name_left[disjunct++])); + } /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. * Because if they are constants, then in the "not joined" rows, they may have different values * - default values, which can differ from the values of these constants. */ - if constexpr (right || full) + if constexpr (jf.right || jf.full) { materializeBlockInplace(block); @@ -1150,58 +1501,68 @@ void HashJoin::joinBlockImpl( * but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped. * For ASOF, the last column is used as the ASOF column */ - - /// Only rows where mask == true can be joined - ColumnPtr join_mask_column = JoinCommon::getColumnAsMask(block, condition_mask_column_name_left); - - AddedColumns added_columns( + auto added_columns = std::make_unique( block_with_columns_to_add, - block, - savedBlockSample(), + block, savedBlockSample(), *this, - left_key_columns, + left_key_columns_vector, key_sizes, - join_mask_column ? &assert_cast(*join_mask_column).getData() : nullptr, - is_asof_join, + join_mask_column_vector, + jf.is_asof_join, is_join_get); bool has_required_right_keys = (required_right_keys.columns() != 0); - added_columns.need_filter = need_filter || has_required_right_keys; + added_columns->need_filter = jf.need_filter || has_required_right_keys; - IColumn::Filter row_filter = overDictionary() ? - dictionaryJoinRightColumns(*table_join, added_columns, null_map) : - switchJoinRightColumns(maps_, added_columns, data->type, null_map, used_flags); + added_columns->row_filter = overDictionary() ? + dictionaryJoinRightColumns(*table_join, *added_columns, null_map_vector[0]): + switchJoinRightColumns(maps_, *added_columns, data->type, null_map_vector, used_flags); - for (size_t i = 0; i < added_columns.size(); ++i) - block.insert(added_columns.moveColumn(i)); + for (size_t i = 0; i < added_columns->size(); ++i) + block.insert(added_columns->moveColumn(i)); + return added_columns; +} + +template +void HashJoin::joinBlockImpl( + Block & block, + std::unique_ptr added_columns, + size_t existing_columns) const +{ + JoinFeatures jf; + bool has_required_right_keys = (required_right_keys.columns() != 0); std::vector right_keys_to_replicate [[maybe_unused]]; - if constexpr (need_filter) + if constexpr (jf.need_filter) { /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. for (size_t i = 0; i < existing_columns; ++i) - block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(row_filter, -1); + block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(added_columns->row_filter, -1); - /// Add join key columns from right block if needed. + /// Add join key columns from right block if needed + /// using value from left table because of equality for (size_t i = 0; i < required_right_keys.columns(); ++i) { const auto & right_key = required_right_keys.getByPosition(i); - const auto & left_name = required_right_keys_sources[i]; + // renamed ??? + if (!block.findByName(right_key.name)) + { + const auto & left_name = required_right_keys_sources[i]; - /// asof column is already in block. - if (is_asof_join && right_key.name == key_names_right.back()) - continue; + /// asof column is already in block. + if (jf.is_asof_join && right_key.name == key_names_right[0].back()) + continue; - const auto & col = block.getByName(left_name); - bool is_nullable = nullable_right_side || right_key.type->isNullable(); - - auto right_col_name = getTableJoin().renamedRightColumnName(right_key.name); - ColumnWithTypeAndName right_col(col.column, col.type, right_col_name); - if (right_col.type->lowCardinality() != right_key.type->lowCardinality()) - JoinCommon::changeLowCardinalityInplace(right_col); - right_col = correctNullability(std::move(right_col), is_nullable); - block.insert(right_col); + const auto & col = block.getByName(left_name); + bool is_nullable = nullable_right_side || right_key.type->isNullable(); + auto right_col_name = getTableJoin().renamedRightColumnName(right_key.name); + ColumnWithTypeAndName right_col(col.column, col.type, right_col_name); + if (right_col.type->lowCardinality() != right_key.type->lowCardinality()) + JoinCommon::changeLowCardinalityInplace(right_col); + right_col = correctNullability(std::move(right_col), is_nullable); + block.insert(right_col); + } } } else if (has_required_right_keys) @@ -1209,39 +1570,40 @@ void HashJoin::joinBlockImpl( /// Some trash to represent IColumn::Filter as ColumnUInt8 needed for ColumnNullable::applyNullMap() auto null_map_filter_ptr = ColumnUInt8::create(); ColumnUInt8 & null_map_filter = assert_cast(*null_map_filter_ptr); - null_map_filter.getData().swap(row_filter); + null_map_filter.getData().swap(added_columns->row_filter); const IColumn::Filter & filter = null_map_filter.getData(); - /// Add join key columns from right block if needed. for (size_t i = 0; i < required_right_keys.columns(); ++i) { const auto & right_key = required_right_keys.getByPosition(i); - const auto & left_name = required_right_keys_sources[i]; - - /// asof column is already in block. - if (is_asof_join && right_key.name == key_names_right.back()) - continue; - - const auto & col = block.getByName(left_name); - bool is_nullable = nullable_right_side || right_key.type->isNullable(); - - ColumnPtr thin_column = filterWithBlanks(col.column, filter); - auto right_col_name = getTableJoin().renamedRightColumnName(right_key.name); - ColumnWithTypeAndName right_col(thin_column, col.type, right_col_name); - if (right_col.type->lowCardinality() != right_key.type->lowCardinality()) - JoinCommon::changeLowCardinalityInplace(right_col); - right_col = correctNullability(std::move(right_col), is_nullable, null_map_filter); - block.insert(right_col); + if (!block.findByName(right_col_name /*right_key.name*/)) + { + const auto & left_name = required_right_keys_sources[i]; + /// asof column is already in block. + if (jf.is_asof_join && right_key.name == key_names_right[0].back()) + continue; - if constexpr (need_replication) - right_keys_to_replicate.push_back(block.getPositionByName(right_key.name)); + const auto & col = block.getByName(left_name); + bool is_nullable = nullable_right_side || right_key.type->isNullable(); + + ColumnPtr thin_column = filterWithBlanks(col.column, filter); + + ColumnWithTypeAndName right_col(thin_column, col.type, right_col_name); + if (right_col.type->lowCardinality() != right_key.type->lowCardinality()) + JoinCommon::changeLowCardinalityInplace(right_col); + right_col = correctNullability(std::move(right_col), is_nullable, null_map_filter); + block.insert(right_col); + + if constexpr (jf.need_replication) + right_keys_to_replicate.push_back(block.getPositionByName(right_key.name)); + } } } - if constexpr (need_replication) + if constexpr (jf.need_replication) { - std::unique_ptr & offsets_to_replicate = added_columns.offsets_to_replicate; + std::unique_ptr & offsets_to_replicate = added_columns->offsets_to_replicate; /// If ALL ... JOIN - we replicate all the columns except the new ones. for (size_t i = 0; i < existing_columns; ++i) @@ -1249,7 +1611,10 @@ void HashJoin::joinBlockImpl( /// Replicate additional right keys for (size_t pos : right_keys_to_replicate) + { block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); + } + } } @@ -1258,6 +1623,7 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) size_t max_joined_block_rows = table_join->maxJoinedBlockRows(); size_t start_left_row = 0; size_t start_right_block = 0; + if (not_processed) { auto & continuation = static_cast(*not_processed); @@ -1295,8 +1661,10 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) for (size_t left_row = start_left_row; left_row < rows_left; ++left_row) { size_t block_number = 0; - for (const Block & block_right : data->blocks) + for (const auto & block_wrapper : data->blocks) { + const Block & block_right = block_wrapper.block; + ++block_number; if (block_number < start_right_block) continue; @@ -1376,14 +1744,25 @@ ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block for (size_t i = 0; i < block.columns(); ++i) { auto key = block.getByPosition(i); - key.name = key_names_right[i]; + key.name = key_names_right.front()[i]; keys.insert(std::move(key)); } static_assert(!MapGetter::flagged, "joinGet are not protected from hash table changes between block processing"); + + + size_t existing_columns = block.columns(); + + std::vector maps_vector; + maps_vector.push_back(&std::get(data->maps[0])); + + auto added_columns = makeAddedColumns( + keys, key_names_right, block_with_columns_to_add, maps_vector, /* is_join_get */ true); + + joinBlockImpl( - keys, key_names_right, block_with_columns_to_add, std::get(data->maps), /* is_join_get */ true); + keys, std::move(added_columns), existing_columns); return keys.getByPosition(keys.columns() - 1); } @@ -1394,50 +1773,107 @@ void HashJoin::checkTypesOfKeys(const Block & block) const void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) { - const Names & key_names_left = table_join->keyNamesLeft(); - JoinCommon::checkTypesOfKeys(block, key_names_left, condition_mask_column_name_left, - right_sample_block, key_names_right, condition_mask_column_name_right); + for (size_t i = 0; i < key_names_left.size(); ++i) + { + JoinCommon::checkTypesOfKeys(block, key_names_left[i], condition_mask_column_name_left[i], + right_sample_block, key_names_right[i], condition_mask_column_name_right[i]); + } + + if (kind == ASTTableJoin::Kind::Cross) + { + joinBlockImplCross(block, not_processed); + return; + } + else if (kind == ASTTableJoin::Kind::Right || kind == ASTTableJoin::Kind::Full) + { + materializeBlockInplace(block); + + if (nullable_left_side) + JoinCommon::convertColumnsToNullable(block); + } + + AddedColumnsV added_columns_v; + size_t existing_columns = block.columns(); if (overDictionary()) { using Kind = ASTTableJoin::Kind; using Strictness = ASTTableJoin::Strictness; - auto & map = std::get(data->maps); + auto & map = std::get(data->maps[0]); + std::vector*> maps_vector; + maps_vector.push_back(&map); + if (kind == Kind::Left) { switch (strictness) { - case Strictness::Any: - case Strictness::All: - joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add, map); + case Strictness::Any: + case Strictness::All: + { + auto added_columns = makeAddedColumns( + block, key_names_left, sample_block_with_columns_to_add, maps_vector); + joinBlockImpl(block, std::move(added_columns), existing_columns); break; - case Strictness::Semi: - joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add, map); + } + + case Strictness::Semi: + { + auto added_columns = makeAddedColumns( + block, key_names_left, sample_block_with_columns_to_add, maps_vector); + joinBlockImpl(block, std::move(added_columns), existing_columns); break; - case Strictness::Anti: - joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add, map); + } + case Strictness::Anti: + { + auto added_columns = makeAddedColumns( + block, key_names_left, sample_block_with_columns_to_add, maps_vector); + joinBlockImpl(block, std::move(added_columns), existing_columns); break; - default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: dictionary + {} {}", strictness, kind); + } + default: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: dictionary + {} {}", strictness, kind); } } else if (kind == Kind::Inner && strictness == Strictness::All) - joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add, map); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: dictionary + {} {}", strictness, kind); - } - else if (joinDispatch(kind, strictness, data->maps, [&](auto kind_, auto strictness_, auto & map) { - joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add, map); - })) - { - /// Joined + auto added_columns = makeAddedColumns( + block, key_names_left/*[0]*/, sample_block_with_columns_to_add, maps_vector); + joinBlockImpl(block, std::move(added_columns), existing_columns); + } + + else + throw throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind); } - else if (kind == ASTTableJoin::Kind::Cross) - joinBlockImplCross(block, not_processed); else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind); + { + + // MapsVariantPtrVector maps_vector; + std::vectormaps[0])>* > maps_vector; + + for (size_t i = 0; i < key_names_left.size(); ++i) + { + // JoinCommon::checkTypesOfKeys(block, key_names_left[i], condition_mask_column_name_left[i], + // right_table_keys, key_names_right[i], condition_mask_column_name_right[i]); + maps_vector.push_back(&data->maps[i]); + } + std::unique_ptr added_columns; + + joinDispatch(kind, strictness, maps_vector, [&](auto kind_, auto strictness_, auto & maps_vector_) + { + added_columns = makeAddedColumns(block, key_names_left, sample_block_with_columns_to_add, maps_vector_); + }); + + if (joinDispatch(kind, strictness, data->maps[0], [&](auto kind_, auto strictness_, auto &) + { + joinBlockImpl(block, std::move(added_columns), existing_columns); + })) + { + /// Joined + } + else + throw throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind); + } } template @@ -1480,6 +1916,7 @@ struct AdderNonJoined /// Stream from not joined earlier rows of the right table. +template class NotJoinedHash final : public NotJoinedBlocks::RightColumnsFiller { public: @@ -1491,17 +1928,25 @@ public: size_t fillColumns(MutableColumns & columns_right) override { - size_t rows_added = 0; + // if (multiple_disjuncts && parent.nullable_right_side) + // { + // JoinCommon::convertColumnsToNullable(columns_right); + // } + size_t rows_added = 0; auto fill_callback = [&](auto, auto strictness, auto & map) { rows_added = fillColumnsFromMap(map, columns_right); }; - if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps, fill_callback)) + if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps.front(), fill_callback)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness '{}' (must be on of: ANY, ALL, ASOF)", parent.strictness); - fillNullsFromBlocks(columns_right, rows_added); + if constexpr (!multiple_disjuncts) + { + fillNullsFromBlocks(columns_right, rows_added); + } + return rows_added; } @@ -1509,8 +1954,10 @@ private: const HashJoin & parent; UInt64 max_block_size; + std::any position; std::optional nulls_position; + std::optional used_position; template size_t fillColumnsFromMap(const Maps & maps, MutableColumns & columns_keys_and_right) @@ -1533,31 +1980,61 @@ private: template size_t fillColumns(const Map & map, MutableColumns & columns_keys_and_right) { - using Mapped = typename Map::mapped_type; - using Iterator = typename Map::const_iterator; - size_t rows_added = 0; - if (!position.has_value()) - position = std::make_any(map.begin()); - - Iterator & it = std::any_cast(position); - auto end = map.end(); - - for (; it != end; ++it) + if constexpr (multiple_disjuncts) { - const Mapped & mapped = it->getMapped(); + if (!used_position.has_value()) + used_position = parent.data->blocks.begin(); - size_t off = map.offsetInternal(it.getPtr()); - if (parent.isUsed(off)) - continue; + auto end = parent.data->blocks.end(); - AdderNonJoined::add(mapped, rows_added, columns_keys_and_right); - - if (rows_added >= max_block_size) + for (auto & it = *used_position; it != end && rows_added < max_block_size; ++it) { - ++it; - break; + const HashJoin::BlockWithFlags & block_with_flags = *it; + + for (size_t row = 0; row < block_with_flags.flags.size(); ++row) + { + if (!block_with_flags.flags[row]) + { + for (size_t colnum = 0; colnum < columns_keys_and_right.size(); ++colnum) + { + auto clmn = block_with_flags.block.getByPosition(colnum).column; + columns_keys_and_right[colnum]->insertFrom(*clmn, row); + } + + ++rows_added; + } + } + } + } + else + { + using Mapped = typename Map::mapped_type; + using Iterator = typename Map::const_iterator; + + + if (!position.has_value()) + position = std::make_any(map.begin()); + + Iterator & it = std::any_cast(position); + auto end = map.end(); + + for (; it != end; ++it) + { + const Mapped & mapped = it->getMapped(); + + size_t off = map.offsetInternal(it.getPtr()); + if (parent.isUsed(off)) + continue; + AdderNonJoined::add(mapped, rows_added, columns_keys_and_right); + + + if (rows_added >= max_block_size) + { + ++it; + break; + } } } @@ -1589,7 +2066,6 @@ private: } }; - std::shared_ptr HashJoin::getNonJoinedBlocks(const Block & result_sample_block, UInt64 max_block_size) const { if (table_join->strictness() == ASTTableJoin::Strictness::Asof || @@ -1598,20 +2074,35 @@ std::shared_ptr HashJoin::getNonJoinedBlocks(const Block & resu { return {}; } + bool multiple_disjuncts = key_names_right.size() > 1; - size_t left_columns_count = result_sample_block.columns() - required_right_keys.columns() - sample_block_with_columns_to_add.columns(); - auto non_joined = std::make_unique(*this, max_block_size); - return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); + if (multiple_disjuncts) + { + /// ... calculate `left_columns_count` ... + // throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "TODO"); + size_t left_columns_count = result_sample_block.columns() - required_right_keys.columns() - sample_block_with_columns_to_add.columns(); + auto non_joined = std::make_unique>(*this, max_block_size); + return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); + + } + else + { + size_t left_columns_count = result_sample_block.columns() - required_right_keys.columns() - sample_block_with_columns_to_add.columns(); + auto non_joined = std::make_unique>(*this, max_block_size); + return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); + } } void HashJoin::reuseJoinedData(const HashJoin & join) { data = join.data; from_storage_join = true; - joinDispatch(kind, strictness, data->maps, [this](auto kind_, auto strictness_, auto & map) + for (auto & map : data->maps) { - used_flags.reinit(map.getBufferSizeInCells(data->type) + 1); - }); + joinDispatch(kind, strictness, map, [this](auto kind_, auto strictness_, auto & map_) + { + used_flags.reinit(map_.getBufferSizeInCells(data->type) + 1); + }); + } } - } diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index a36393278ff..3d523a6346f 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -48,18 +48,22 @@ public: bool getUsedSafe(size_t i) const; - template - void setUsed(size_t i); + template + void setUsed(const T & f); - template - bool getUsed(size_t i); + template + bool getUsed(const T & f); - template - bool setUsedOnce(size_t i); + template + bool setUsedOnce(const T & f); }; } +using SizesVector = std::vector; + +class AddedColumns; + /** Data structure for implementation of JOIN. * It is just a hash table: keys -> rows of joined ("right") table. * Additionally, CROSS JOIN is supported: instead of hash table, it use just set of blocks without keys. @@ -183,7 +187,7 @@ public: const ColumnWithTypeAndName & rightAsofKeyColumn() const { /// It should be nullable if nullable_right_side is true - return savedBlockSample().getByName(key_names_right.back()); + return savedBlockSample().getByName(key_names_right.front().back()); } /// Different types of keys for maps. @@ -306,21 +310,33 @@ public: using MapsVariant = std::variant; using BlockNullmapList = std::deque>; + using BlockUsedmapList = std::deque>; + + struct BlockWithFlags : public ExtraBlock + { + mutable std::vector flags; + }; + + using BlocksWithFlagsList = std::list; + struct RightTableData { Type type = Type::EMPTY; bool empty = true; - MapsVariant maps; + std::vector maps; Block sample_block; /// Block as it would appear in the BlockList - BlocksList blocks; /// Blocks of "right" table. + // BlocksList blocks; /// Blocks of "right" table. + BlocksWithFlagsList blocks; BlockNullmapList blocks_nullmaps; /// Nullmaps for blocks of "right" table (if needed) /// Additional data - strings for string keys and continuation elements of single-linked lists of references to rows. Arena pool; }; + using RightTableDataPtr = std::shared_ptr; + /// We keep correspondence between used_flags and hash table internal buffer. /// Hash table cannot be modified during HashJoin lifetime and must be protected with lock. void setLock(std::shared_mutex & rwlock) @@ -338,7 +354,8 @@ public: bool isUsed(size_t off) const { return used_flags.getUsedSafe(off); } private: - friend class NotJoinedHash; + template friend class NotJoinedHash; + friend class JoinSource; std::shared_ptr table_join; @@ -349,7 +366,8 @@ private: bool from_storage_join = false; /// Names of key columns in right-side table (in the order they appear in ON/USING clause). @note It could contain duplicates. - const Names & key_names_right; + const NamesVector key_names_right; + const NamesVector key_names_left; bool nullable_right_side; /// In case of LEFT and FULL joins, if use_nulls, convert right-side columns to Nullable. bool nullable_left_side; /// In case of RIGHT and FULL joins, if use_nulls, convert left-side columns to Nullable. @@ -358,14 +376,14 @@ private: ASOF::Inequality asof_inequality; /// Right table data. StorageJoin shares it between many Join objects. - std::shared_ptr data; /// Flags that indicate that particular row already used in join. /// Flag is stored for every record in hash map. /// Number of this flags equals to hashtable buffer size (plus one for zero value). /// Changes in hash table broke correspondence, /// so we must guarantee constantness of hash table during HashJoin lifetime (using method setLock) mutable JoinStuff::JoinUsedFlags used_flags; - Sizes key_sizes; + RightTableDataPtr data; + SizesVector key_sizes; /// Block with columns from the right-side table. Block right_sample_block; @@ -379,8 +397,8 @@ private: std::vector required_right_keys_sources; /// Additional conditions for rows to join from JOIN ON section - String condition_mask_column_name_left; - String condition_mask_column_name_right; + std::vector condition_mask_column_name_left; + std::vector condition_mask_column_name_right; Poco::Logger * log; @@ -390,28 +408,50 @@ private: /// If set HashJoin instance is not available for modification (addJoinedBlock) std::shared_lock storage_join_lock; - void init(Type type_); + // void init(Type type_); + void init(Type type_, RightTableDataPtr); + void data_map_init(MapsVariant &); const Block & savedBlockSample() const { return data->sample_block; } /// Modify (structure) right block to save it in block list - Block structureRightBlock(const Block & stored_block) const; + BlockWithFlags structureRightBlock(const Block & stored_block) const; void initRightBlockStructure(Block & saved_block_sample); - template + template void joinBlockImpl( + Block & block, + std::unique_ptr, + size_t existing_columns) const; + + template + std::unique_ptr makeAddedColumns( + Block & block, + const NamesVector & key_names_left, + const Block & block_with_columns_to_add, + const std::vector & maps, + bool is_join_get = false) const; + + template + std::unique_ptr makeAddedColumnsV( Block & block, const Names & key_names_left, const Block & block_with_columns_to_add, const Maps & maps, + const Sizes & key_sizes_, + HashJoin::Type, bool is_join_get = false) const; void joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const; + // template + // ColumnWithTypeAndName joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_, HashJoin::Type) const; + static Type chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes); bool empty() const; bool overDictionary() const; }; + } diff --git a/src/Interpreters/JoinSwitcher.cpp b/src/Interpreters/JoinSwitcher.cpp index 480d105ebb6..599755e43ea 100644 --- a/src/Interpreters/JoinSwitcher.cpp +++ b/src/Interpreters/JoinSwitcher.cpp @@ -51,7 +51,7 @@ bool JoinSwitcher::addJoinedBlock(const Block & block, bool) void JoinSwitcher::switchJoin() { std::shared_ptr joined_data = static_cast(*join).getJoinedData(); - BlocksList right_blocks = std::move(joined_data->blocks); + HashJoin::BlocksWithFlagsList right_blocks = std::move(joined_data->blocks); /// Destroy old join & create new one. Early destroy for memory saving. join = std::make_shared(table_join, right_sample_block); @@ -62,20 +62,20 @@ void JoinSwitcher::switchJoin() if (!right_blocks.empty()) { positions.reserve(right_sample_block.columns()); - const Block & tmp_block = *right_blocks.begin(); + const HashJoin::BlockWithFlags & tmp_block = *right_blocks.begin(); for (const auto & sample_column : right_sample_block) { - positions.emplace_back(tmp_block.getPositionByName(sample_column.name)); + positions.emplace_back(tmp_block.block.getPositionByName(sample_column.name)); is_nullable.emplace_back(sample_column.type->isNullable()); } } - for (Block & saved_block : right_blocks) + for (HashJoin::BlockWithFlags & saved_block : right_blocks) { Block restored_block; for (size_t i = 0; i < positions.size(); ++i) { - auto & column = saved_block.getByPosition(positions[i]); + auto & column = saved_block.block.getByPosition(positions[i]); restored_block.insert(correctNullability(std::move(column), is_nullable[i])); } join->addJoinedBlock(restored_block); diff --git a/src/Interpreters/LogicalExpressionsOptimizer.cpp b/src/Interpreters/LogicalExpressionsOptimizer.cpp index 936ed0149d2..ad3a1b8424a 100644 --- a/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -1,3 +1,4 @@ +#include #include #include diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index ebcc8739fc4..651a5093508 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -507,7 +507,7 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right ErrorCodes::PARAMETER_OUT_OF_BOUND); } - std::tie(mask_column_name_left, mask_column_name_right) = table_join->joinConditionColumnNames(); + std::tie(mask_column_name_left, mask_column_name_right) = table_join->joinConditionColumnNames(0); /// Add auxiliary joining keys to join only rows where conditions from JOIN ON sections holds /// Input boolean column converted to nullable and only rows with non NULLS value will be joined @@ -519,11 +519,11 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right key_names_right.push_back(deriveTempName(mask_column_name_right)); } - key_names_left.insert(key_names_left.end(), table_join->keyNamesLeft().begin(), table_join->keyNamesLeft().end()); - key_names_right.insert(key_names_right.end(), table_join->keyNamesRight().begin(), table_join->keyNamesRight().end()); + key_names_left.insert(key_names_left.end(), table_join->keyNamesLeft().front().begin(), table_join->keyNamesLeft().front().end()); + key_names_right.insert(key_names_right.end(), table_join->keyNamesRight().front().begin(), table_join->keyNamesRight().front().end()); addConditionJoinColumn(right_sample_block, JoinTableSide::Right); - JoinCommon::splitAdditionalColumns(key_names_right, right_sample_block, right_table_keys, right_columns_to_add); + JoinCommon::splitAdditionalColumns(NamesVector{key_names_right}, right_sample_block, right_table_keys, right_columns_to_add); for (const auto & right_key : key_names_right) { @@ -654,7 +654,7 @@ bool MergeJoin::saveRightBlock(Block && block) Block MergeJoin::modifyRightBlock(const Block & src_block) const { Block block = materializeBlock(src_block); - JoinCommon::removeLowCardinalityInplace(block, table_join->keyNamesRight()); + JoinCommon::removeLowCardinalityInplace(block, table_join->keyNamesRight().front()); return block; } diff --git a/src/Interpreters/RequiredSourceColumnsData.h b/src/Interpreters/RequiredSourceColumnsData.h index d425989393e..6b2e514ff94 100644 --- a/src/Interpreters/RequiredSourceColumnsData.h +++ b/src/Interpreters/RequiredSourceColumnsData.h @@ -36,6 +36,7 @@ struct RequiredSourceColumnsData bool has_table_join = false; bool has_array_join = false; + bool converted_to_dnf = false; bool addColumnAliasIfAny(const IAST & ast); void addColumnIdentifier(const ASTIdentifier & node); diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index 21ec94a6917..85bd8d1b70e 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -178,8 +178,14 @@ void RequiredSourceColumnsMatcher::visit(const ASTFunction & node, const ASTPtr void RequiredSourceColumnsMatcher::visit(const ASTTablesInSelectQueryElement & node, const ASTPtr &, Data & data) { for (const auto & child : node.children) - if (child->as()) + { + const auto * join_node = child->as(); + if (join_node) + { data.has_table_join = true; + data.converted_to_dnf = join_node->converted_to_dnf; + } + } } /// ASTIdentifiers here are tables. Do not visit them as generic ones. diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index b0c89adee45..0ec581ad29f 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -62,6 +62,10 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) , partial_merge_join_left_table_buffer_bytes(settings.partial_merge_join_left_table_buffer_bytes) , max_files_to_merge(settings.join_on_disk_max_files_to_merge) , temporary_files_codec(settings.temporary_files_codec) + , key_names_left(1) + , key_names_right(1) + , on_filter_condition_asts_left(1) + , on_filter_condition_asts_right(1) , tmp_volume(tmp_volume_) { } @@ -72,31 +76,60 @@ void TableJoin::resetCollected() key_names_right.clear(); key_asts_left.clear(); key_asts_right.clear(); + on_filter_condition_asts_left.clear(); + on_filter_condition_asts_right.clear(); columns_from_joined_table.clear(); columns_added_by_join.clear(); original_names.clear(); renames.clear(); left_type_map.clear(); right_type_map.clear(); + key_names_left.resize(1); + key_names_right.resize(1); + on_filter_condition_asts_left.resize(1); + on_filter_condition_asts_right.resize(1); } void TableJoin::addUsingKey(const ASTPtr & ast) { - key_names_left.push_back(ast->getColumnName()); - key_names_right.push_back(ast->getAliasOrColumnName()); + key_names_left.front().push_back(ast->getColumnName()); + key_names_right.front().push_back(ast->getAliasOrColumnName()); key_asts_left.push_back(ast); key_asts_right.push_back(ast); - auto & right_key = key_names_right.back(); + auto & right_key = key_names_right.front().back(); if (renames.count(right_key)) right_key = renames[right_key]; } + +void TableJoin::addDisjunct(const IAST* addr) +{ + if (std::find(disjuncts.begin(), disjuncts.end(), addr) != disjuncts.end()) + { + assert(key_names_left.size() == disjunct_num + 1); + + if (!key_names_left[disjunct_num].empty()) + { + disjunct_num++; + key_names_left.resize(disjunct_num+1); + key_names_right.resize(disjunct_num+1); + on_filter_condition_asts_left.resize(disjunct_num+1); + on_filter_condition_asts_right.resize(disjunct_num+1); + } + } +} + +void TableJoin::setDisjuncts(std::vector&& disjuncts_) +{ + disjuncts = disjuncts_; +} + void TableJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast) { - key_names_left.push_back(left_table_ast->getColumnName()); - key_names_right.push_back(right_table_ast->getAliasOrColumnName()); + key_names_left[disjunct_num].push_back(left_table_ast->getColumnName()); + key_names_right[disjunct_num].push_back(right_table_ast->getAliasOrColumnName()); key_asts_left.push_back(left_table_ast); key_asts_right.push_back(right_table_ast); @@ -109,9 +142,9 @@ size_t TableJoin::rightKeyInclusion(const String & name) const return 0; size_t count = 0; - for (const auto & key_name : key_names_right) - if (name == key_name) - ++count; + for (const auto & key_names : key_names_right) + count += std::count(key_names.begin(), key_names.end(), name); + return count; } @@ -159,8 +192,10 @@ ASTPtr TableJoin::leftKeysList() const { ASTPtr keys_list = std::make_shared(); keys_list->children = key_asts_left; - if (ASTPtr extra_cond = joinConditionColumn(JoinTableSide::Left)) - keys_list->children.push_back(extra_cond); + const size_t disjuncts_num = key_names_left.size(); + for (size_t d = 0; d < disjuncts_num; ++d) + if (ASTPtr extra_cond = joinConditionColumn(JoinTableSide::Left, d)) + keys_list->children.push_back(extra_cond); return keys_list; } @@ -169,14 +204,18 @@ ASTPtr TableJoin::rightKeysList() const ASTPtr keys_list = std::make_shared(); if (hasOn()) keys_list->children = key_asts_right; - if (ASTPtr extra_cond = joinConditionColumn(JoinTableSide::Right)) - keys_list->children.push_back(extra_cond); + const size_t disjuncts_num = key_names_left.size(); + for (size_t d = 0; d < disjuncts_num; ++d) + if (ASTPtr extra_cond = joinConditionColumn(JoinTableSide::Right, d)) + keys_list->children.push_back(extra_cond); return keys_list; } Names TableJoin::requiredJoinedNames() const { - NameSet required_columns_set(key_names_right.begin(), key_names_right.end()); + NameSet required_columns_set; + for (const auto& key_names_right_part : key_names_right) + required_columns_set.insert(key_names_right_part.begin(), key_names_right_part.end()); for (const auto & joined_column : columns_added_by_join) required_columns_set.insert(joined_column.name); @@ -186,12 +225,15 @@ Names TableJoin::requiredJoinedNames() const NameSet TableJoin::requiredRightKeys() const { NameSet required; - for (const auto & name : key_names_right) + for (const auto & key_names_right_part : key_names_right) { - auto rename = renamedRightColumnName(name); - for (const auto & column : columns_added_by_join) - if (rename == column.name) - required.insert(name); + for (const auto & name : key_names_right_part) + { + auto rename = renamedRightColumnName(name); + for (const auto & column : columns_added_by_join) + if (rename == column.name) + required.insert(name); + } } return required; } @@ -201,28 +243,36 @@ NamesWithAliases TableJoin::getRequiredColumns(const Block & sample, const Names NameSet required_columns(action_required_columns.begin(), action_required_columns.end()); for (auto & column : requiredJoinedNames()) + { if (!sample.has(column)) required_columns.insert(column); + } return getNamesWithAliases(required_columns); } Block TableJoin::getRequiredRightKeys(const Block & right_table_keys, std::vector & keys_sources) const { - const Names & left_keys = keyNamesLeft(); - const Names & right_keys = keyNamesRight(); + const auto & left_keys = keyNamesLeft(); + const auto & right_keys = keyNamesRight(); NameSet required_keys = requiredRightKeys(); Block required_right_keys; - for (size_t i = 0; i < right_keys.size(); ++i) + for (size_t p = 0; p < right_keys.size(); ++p) { - const String & right_key_name = right_keys[i]; + const auto & right_keys_part = right_keys[p]; - if (required_keys.count(right_key_name) && !required_right_keys.has(right_key_name)) + for (size_t i = 0; i < right_keys_part.size(); ++i) { - const auto & right_key = right_table_keys.getByName(right_key_name); - required_right_keys.insert(right_key); - keys_sources.push_back(left_keys[i]); + const String & right_key_name = right_keys_part[i]; + + if (required_keys.count(right_key_name) && !required_right_keys.has(right_key_name)) + { + const auto & right_key = right_table_keys.getByName(right_key_name); + required_right_keys.insert(right_key); + const auto & left_keys_part = left_keys[p]; + keys_sources.push_back(left_keys_part[i]); + } } } @@ -315,7 +365,10 @@ bool TableJoin::allowMergeJoin() const bool all_join = is_all && (isInner(kind()) || isLeft(kind()) || isRight(kind()) || isFull(kind())); bool special_left = isLeft(kind()) && (is_any || is_semi); - return all_join || special_left; + + bool no_ors = (key_names_right.size() == 1); + + return (all_join || special_left) && no_ors; } bool TableJoin::needStreamWithNonJoinedRows() const @@ -351,7 +404,7 @@ bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context) if (!allowed_inner && !allowed_left) return false; - const Names & right_keys = keyNamesRight(); + const Names & right_keys = keyNamesRight().front(); if (right_keys.size() != 1) return false; @@ -414,43 +467,39 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig for (const auto & col : right) right_types[renamedRightColumnName(col.name)] = col.type; - for (size_t i = 0; i < key_names_left.size(); ++i) + for (size_t d = 0; d < key_names_left.size(); ++d) { - auto ltype = left_types.find(key_names_left[i]); - auto rtype = right_types.find(key_names_right[i]); - if (ltype == left_types.end() || rtype == right_types.end()) + for (size_t i = 0; i < key_names_left[d].size(); ++i) { - /// Name mismatch, give up - left_type_map.clear(); - right_type_map.clear(); - return false; - } + auto ltype = left_types.find(key_names_left[d][i]); + auto rtype = right_types.find(key_names_right[d][i]); + if (ltype == left_types.end() || rtype == right_types.end()) + { + /// Name mismatch, give up + left_type_map.clear(); + right_type_map.clear(); + return false; + } - if (JoinCommon::typesEqualUpToNullability(ltype->second, rtype->second)) - continue; + if (JoinCommon::typesEqualUpToNullability(ltype->second, rtype->second)) + continue; - DataTypePtr common_type; - try - { - /// TODO(vdimir): use getMostSubtype if possible - common_type = DB::getLeastSupertype({ltype->second, rtype->second}); + DataTypePtr supertype; + try + { + /// TODO(vdimir): use getMostSubtype if possible + common_type = DB::getLeastSupertype({ltype->second, rtype->second}); + } + catch (DB::Exception & ex) + { + throw DB::Exception(ErrorCodes::TYPE_MISMATCH, + "Can't infer common type for joined columns: {}: {} at left, {}: {} at right. {}", + key_names_left[d][i], ltype->second->getName(), + key_names_right[d][i], rtype->second->getName(), + ex.message()); + } + left_type_map[key_names_left[d][i]] = right_type_map[key_names_right[d][i]] = supertype; } - catch (DB::Exception & ex) - { - throw DB::Exception(ErrorCodes::TYPE_MISMATCH, - "Can't infer common type for joined columns: {}: {} at left, {}: {} at right. {}", - key_names_left[i], ltype->second->getName(), - key_names_right[i], rtype->second->getName(), - ex.message()); - } - - if (!allow_right && !common_type->equals(*rtype->second)) - { - throw DB::Exception(ErrorCodes::TYPE_MISMATCH, - "Can't change type for right table: {}: {} -> {}.", - key_names_right[i], rtype->second->getName(), common_type->getName()); - } - left_type_map[key_names_left[i]] = right_type_map[key_names_right[i]] = common_type; } if (!left_type_map.empty() || !right_type_map.empty()) @@ -466,7 +515,7 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig } ActionsDAGPtr TableJoin::applyKeyConvertToTable( - const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, Names & names_to_rename) const + const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, NamesVector & names_vector_to_rename) const { bool has_some_to_do = false; @@ -488,12 +537,16 @@ ActionsDAGPtr TableJoin::applyKeyConvertToTable( auto dag = ActionsDAG::makeConvertingActions( cols_src, cols_dst, ActionsDAG::MatchColumnsMode::Name, true, !hasUsing(), &key_column_rename); - for (auto & name : names_to_rename) + for (auto & disjunct_names : names_vector_to_rename) { - const auto it = key_column_rename.find(name); - if (it != key_column_rename.end()) - name = it->second; + for (auto & name : disjunct_names) + { + const auto it = key_column_rename.find(name); + if (it != key_column_rename.end()) + name = it->second; + } } + return dag; } @@ -520,12 +573,10 @@ String TableJoin::renamedRightColumnName(const String & name) const void TableJoin::addJoinCondition(const ASTPtr & ast, bool is_left) { - LOG_TRACE(&Poco::Logger::get("TableJoin"), "Add join condition for {} table: {}", (is_left ? "left" : "right"), queryToString(ast)); - if (is_left) - on_filter_condition_asts_left.push_back(ast); + on_filter_condition_asts_left[disjunct_num].push_back(ast); else - on_filter_condition_asts_right.push_back(ast); + on_filter_condition_asts_right[disjunct_num].push_back(ast); } std::unordered_map TableJoin::leftToRightKeyRemap() const @@ -547,35 +598,34 @@ std::unordered_map TableJoin::leftToRightKeyRemap() const } /// Returns all conditions related to one table joined with 'and' function -static ASTPtr buildJoinConditionColumn(const ASTs & on_filter_condition_asts) +static ASTPtr buildJoinConditionColumn(const ASTsVector & on_filter_condition_asts, size_t disjunct) { - if (on_filter_condition_asts.empty()) + if (on_filter_condition_asts[disjunct].empty()) return nullptr; - - if (on_filter_condition_asts.size() == 1) - return on_filter_condition_asts[0]; + if (on_filter_condition_asts[disjunct].size() == 1) + return on_filter_condition_asts[disjunct][0]; auto function = std::make_shared(); function->name = "and"; function->arguments = std::make_shared(); function->children.push_back(function->arguments); - function->arguments->children = on_filter_condition_asts; + function->arguments->children = on_filter_condition_asts[disjunct]; return function; } -ASTPtr TableJoin::joinConditionColumn(JoinTableSide side) const +ASTPtr TableJoin::joinConditionColumn(JoinTableSide side, size_t disjunct) const { if (side == JoinTableSide::Left) - return buildJoinConditionColumn(on_filter_condition_asts_left); - return buildJoinConditionColumn(on_filter_condition_asts_right); + return buildJoinConditionColumn(on_filter_condition_asts_left, disjunct); + return buildJoinConditionColumn(on_filter_condition_asts_right, disjunct); } -std::pair TableJoin::joinConditionColumnNames() const +std::pair TableJoin::joinConditionColumnNames(size_t disjunct) const { std::pair res; - if (auto cond_ast = joinConditionColumn(JoinTableSide::Left)) + if (auto cond_ast = joinConditionColumn(JoinTableSide::Left, disjunct)) res.first = cond_ast->getColumnName(); - if (auto cond_ast = joinConditionColumn(JoinTableSide::Right)) + if (auto cond_ast = joinConditionColumn(JoinTableSide::Right, disjunct)) res.second = cond_ast->getColumnName(); return res; } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 462dd540a00..333c81b1b86 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -73,10 +73,13 @@ private: const size_t max_files_to_merge = 0; const String temporary_files_codec = "LZ4"; - Names key_names_left; - Names key_names_right; /// Duplicating names are qualified. - ASTs on_filter_condition_asts_left; - ASTs on_filter_condition_asts_right; + NamesVector key_names_left; + NamesVector key_names_right; /// Duplicating names are qualified. + ASTsVector on_filter_condition_asts_left; + ASTsVector on_filter_condition_asts_right; +private: + size_t disjunct_num = 0; + std::vector disjuncts; ASTs key_asts_left; ASTs key_asts_right; @@ -112,7 +115,7 @@ private: /// Create converting actions and change key column names if required ActionsDAGPtr applyKeyConvertToTable( - const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, Names & names_to_rename) const; + const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, NamesVector & names_vector_to_rename) const; /// Calculates common supertypes for corresponding join key columns. template @@ -121,17 +124,27 @@ private: NamesAndTypesList correctedColumnsAddedByJoin() const; public: - TableJoin() = default; - TableJoin(const Settings &, VolumePtr tmp_volume); + TableJoin() + : key_names_left(1) + , key_names_right(1) + , on_filter_condition_asts_left(1) + , on_filter_condition_asts_right(1) + { + } + + TableJoin(const Settings & settings, VolumePtr tmp_volume_); /// for StorageJoin TableJoin(SizeLimits limits, bool use_nulls, ASTTableJoin::Kind kind, ASTTableJoin::Strictness strictness, - const Names & key_names_right_) + const NamesVector & key_names_right_) : size_limits(limits) , default_max_bytes(0) , join_use_nulls(use_nulls) , join_algorithm(JoinAlgorithm::HASH) + , key_names_left(1) , key_names_right(key_names_right_) + , on_filter_condition_asts_left(1) + , on_filter_condition_asts_right(1) { table_join.kind = kind; table_join.strictness = strictness; @@ -164,6 +177,8 @@ public: void resetCollected(); void addUsingKey(const ASTPtr & ast); + void addDisjunct(const IAST*); + void setDisjuncts(std::vector&&); void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast); /* Conditions for left/right table from JOIN ON section. @@ -180,8 +195,8 @@ public: * doesn't supported yet, it can be added later. */ void addJoinCondition(const ASTPtr & ast, bool is_left); - ASTPtr joinConditionColumn(JoinTableSide side) const; - std::pair joinConditionColumnNames() const; + ASTPtr joinConditionColumn(JoinTableSide side, size_t disjunct_num) const; + std::pair joinConditionColumnNames(size_t disjunct_num) const; bool hasUsing() const { return table_join.using_expression_list != nullptr; } bool hasOn() const { return table_join.on_expression != nullptr; } @@ -211,8 +226,8 @@ public: ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only - const Names & keyNamesLeft() const { return key_names_left; } - const Names & keyNamesRight() const { return key_names_right; } + const NamesVector & keyNamesLeft() const { return key_names_left; } + const NamesVector & keyNamesRight() const { return key_names_right; } const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; } Names columnsAddedByJoin() const @@ -224,7 +239,7 @@ public: } /// StorageJoin overrides key names (cause of different names qualification) - void setRightKeys(const Names & keys) { key_names_right = keys; } + void setRightKeys(const Names & keys) { key_names_right.clear(); key_names_right.push_back(keys); } Block getRequiredRightKeys(const Block & right_table_keys, std::vector & keys_sources) const; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 8ac7b48ddcb..696a2dd9ed2 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -506,6 +506,186 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul out_table_join = table_join; } +class DNF +{ + bool node_added = false; + + void normTree(ASTPtr node) + { + auto *function = node->as(); + + // LOG_TRACE(&Poco::Logger::get("TreeRewrite"), "top of normTree: {}", node->dumpTree()); + if (function && function->children.size() == 1) + { + for (bool touched = true; touched;) + { + touched = false; + + ASTs new_children; + const auto * expression_list = function->children[0]->as(); + for (const auto & child : expression_list->children) + { + auto *f = child->as(); + if (f && function->children.size() == 1 && ((function->name == "or" && f->name == "or") || (function->name == "and" && f->name == "and"))) + { + std::copy(child->children[0]->children.begin(), + child->children[0]->children.end(), + std::back_inserter(new_children)); + touched = true; + } + else + { + new_children.push_back(child); + } + } + + function->arguments->children = std::move(new_children); + } + + for (auto & child : function->arguments->children) + { + normTree(child); + } + } + // LOG_TRACE(&Poco::Logger::get("TreeRewrite"), "bottom of normTree: {}", node->dumpTree()); + } + + ASTPtr distribute(ASTPtr node) + { + const auto * function = node->as(); + + if (function && function->children.size() == 1) + { + if (function->name == "and") + { + const auto * expression_list = function->children[0]->as(); + if (!expression_list) + { + return node; + } + + auto or_child = std::find_if(expression_list->children.begin(), expression_list->children.end(), [](ASTPtr arg) + { + const auto * f = arg->as(); + return f && f->name == "or" && f->children.size() == 1; + }); + if (or_child == expression_list->children.end()) + { + return node; + } + + ASTs rest_children; + + for (const auto & arg : expression_list->children) + { + // LOG_DEBUG(&Poco::Logger::get("toDNF"), "IDs {} vs. {}", arg->getTreeHash(), (*or_child)->getTreeHash()); + + if (arg->getTreeHash() != (*or_child)->getTreeHash()) + { + rest_children.push_back(arg); + } + } + if (rest_children.empty()) + { + return node; + } + + const auto * or_child_function = (*or_child)->as(); + if (!or_child_function) + { + return node; + } + + auto rest = rest_children.size() > 1 ? + makeASTFunction("and", rest_children): + rest_children[0]; + + const auto * or_child_expression_list = or_child_function->children[0]->as(); + assert(or_child_expression_list); + + if (or_child_expression_list) + { + + ASTs lst; + for (const auto & arg : or_child_expression_list->children) + { + ASTs arg_rest_lst; + arg_rest_lst.push_back(arg); + arg_rest_lst.push_back(rest); + + auto and_node = makeASTFunction("and", arg_rest_lst); + lst.push_back(distribute(and_node)); + } + if (lst.empty()) + { + return node; + } + + auto ret = lst.size()>1 ? + makeASTFunction("or", lst) : + lst[0]; + + node_added = true; + + return ret; + } + } + else if (function->name == "or") + { + const auto * expression_list = function->children[0]->as(); + if (!expression_list) + { + return node; + } + + ASTs lst; + for (const auto & arg : expression_list->children) + { + lst.push_back(distribute(arg)); + } + + auto ret = lst.size() > 1 + ? makeASTFunction("or", lst) + : lst[0]; + return ret; + } + } + + return node; + } + + +public: + + void process(const ASTSelectQuery & select_query, const TablesWithColumns & tables) + { + const ASTTablesInSelectQueryElement * node = select_query.join(); + if (!node || tables.size() < 2) + { + return; + } + + auto & table_join = node->table_join->as(); + if (!table_join.on_expression || table_join.strictness == ASTTableJoin::Strictness::Asof) + { + return; + } + + normTree(table_join.on_expression); + + auto distributed_expression = distribute(table_join.on_expression); + + normTree(distributed_expression); + LOG_TRACE(&Poco::Logger::get("TreeRewrite"), "bottom of toDNF: {}, node_added {}", + distributed_expression->dumpTree(), node_added); + + table_join.on_expression = distributed_expression; + + table_join.converted_to_dnf = node_added; + } +}; + + /// Find the columns that are obtained by JOIN. void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_join, const TablesWithColumns & tables, const Aliases & aliases) @@ -524,10 +704,13 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_ CollectJoinOnKeysVisitor::Data data{analyzed_join, tables[0], tables[1], aliases, is_asof}; CollectJoinOnKeysVisitor(data).visit(table_join.on_expression); - if (analyzed_join.keyNamesLeft().empty()) + for (const auto & key_names_left_part : analyzed_join.keyNamesLeft()) { - throw Exception("Cannot get JOIN keys from JOIN ON section: " + queryToString(table_join.on_expression), - ErrorCodes::INVALID_JOIN_ON_EXPRESSION); + if (key_names_left_part.empty()) + { + throw Exception("Cannot get JOIN keys from JOIN ON section: " + queryToString(table_join.on_expression), + ErrorCodes::INVALID_JOIN_ON_EXPRESSION); + } } if (is_asof) @@ -693,7 +876,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (required.count(name)) { /// Optimisation: do not add columns needed only in JOIN ON section. - if (columns_context.nameInclusion(name) > analyzed_join->rightKeyInclusion(name)) + if (columns_context.converted_to_dnf || columns_context.nameInclusion(name) > analyzed_join->rightKeyInclusion(name)) analyzed_join->addJoinedColumn(joined_column); required.erase(name); @@ -864,6 +1047,10 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select } required_source_columns.swap(source_columns); + for (const auto & column : required_source_columns) + { + source_column_names.insert(column.name); + } } NameSet TreeRewriterResult::getArrayJoinSourceNameSet() const @@ -927,6 +1114,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( { for (const auto & [name, _] : table_join->columns_from_joined_table) all_source_columns_set.insert(name); + DNF().process(*select_query, tables_with_columns); } normalize(query, result.aliases, all_source_columns_set, select_options.ignore_alias, settings, /* allow_self_aliases = */ true); diff --git a/src/Interpreters/joinDispatch.h b/src/Interpreters/joinDispatch.h index cac3fcd1e9a..6245aaefe88 100644 --- a/src/Interpreters/joinDispatch.h +++ b/src/Interpreters/joinDispatch.h @@ -100,4 +100,35 @@ inline bool joinDispatch(ASTTableJoin::Kind kind, ASTTableJoin::Strictness stric }); } +/// Call function on specified join map +template +inline bool joinDispatch(ASTTableJoin::Kind kind, ASTTableJoin::Strictness strictness, std::vector & mapsv, Func && func) +{ + return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij) + { + // NOTE: Avoid using nested static loop as GCC and CLANG have bugs in different ways + // See https://stackoverflow.com/questions/44386415/gcc-and-clang-disagree-about-c17-constexpr-lambda-captures + constexpr auto i = ij / STRICTNESSES.size(); + constexpr auto j = ij % STRICTNESSES.size(); + if (kind == KINDS[i] && strictness == STRICTNESSES[j]) + { + using MapType = typename MapGetter::Map; + std::vector v; + for (const auto & el : mapsv) + { + v.push_back(&std::get(*el)); + } + + func( + std::integral_constant(), + std::integral_constant(), + v + /*std::get::Map>(maps)*/); + return true; + } + return false; + }); +} + + } diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index 9a4cdd97c13..d1e9cb09040 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -15,7 +15,6 @@ #include #include -#include namespace DB { @@ -194,6 +193,22 @@ void convertColumnsToNullable(Block & block, size_t starting_pos) convertColumnToNullable(block.getByPosition(i)); } +void convertColumnsToNullable(MutableColumns & mutable_columns, size_t starting_pos) +{ + for (size_t i = starting_pos; i < mutable_columns.size(); ++i) + { + ColumnPtr column = std::move(mutable_columns[i]); + column = makeNullable(column); + mutable_columns[i] = IColumn::mutate(std::move(column)); + + // changeNullability(mutable_columns[i]); + // mutable_columns[i]->type = convertTypeToNullable(mutable_columns[i]->type); + + // convertColumnToNullable(*mutable_columns[i]); + } + +} + /// @warning It assumes that every NULL has default value in nested column (or it does not matter) void removeColumnNullability(ColumnWithTypeAndName & column) { @@ -502,18 +517,21 @@ ColumnPtr getColumnAsMask(const Block & block, const String & column_name) } -void splitAdditionalColumns(const Names & key_names, const Block & sample_block, Block & block_keys, Block & block_others) +void splitAdditionalColumns(const NamesVector & key_names, const Block & sample_block, Block & block_keys, Block & block_others) { block_others = materializeBlock(sample_block); - for (const String & column_name : key_names) + for (const auto & key_names_part : key_names) { - /// Extract right keys with correct keys order. There could be the same key names. - if (!block_keys.has(column_name)) + for (const String & column_name : key_names_part) { - auto & col = block_others.getByName(column_name); - block_keys.insert(col); - block_others.erase(column_name); + /// Extract right keys with correct keys order. There could be the same key names. + if (!block_keys.has(column_name)) + { + auto & col = block_others.getByName(column_name); + block_keys.insert(col); + block_others.erase(column_name); + } } } } @@ -528,7 +546,6 @@ NotJoinedBlocks::NotJoinedBlocks(std::unique_ptr filler_, , saved_block_sample(filler->getEmptyBlock()) , result_sample_block(materializeBlock(result_sample_block_)) { - for (size_t left_pos = 0; left_pos < left_columns_count; ++left_pos) { /// We need right 'x' for 'RIGHT JOIN ... USING(x)' diff --git a/src/Interpreters/join_common.h b/src/Interpreters/join_common.h index ec2e1d3bd50..4b39cee2c37 100644 --- a/src/Interpreters/join_common.h +++ b/src/Interpreters/join_common.h @@ -14,6 +14,7 @@ class TableJoin; class IColumn; using ColumnRawPtrs = std::vector; using UInt8ColumnDataPtr = const ColumnUInt8::Container *; +using UInt8ColumnDataPtrVector = std::vector; namespace JoinCommon { @@ -21,6 +22,7 @@ bool canBecomeNullable(const DataTypePtr & type); DataTypePtr convertTypeToNullable(const DataTypePtr & type); void convertColumnToNullable(ColumnWithTypeAndName & column); void convertColumnsToNullable(Block & block, size_t starting_pos = 0); +void convertColumnsToNullable(MutableColumns & mutable_columns, size_t starting_pos = 0); void removeColumnNullability(ColumnWithTypeAndName & column); void changeColumnRepresentation(const ColumnPtr & src_column, ColumnPtr & dst_column); ColumnPtr emptyNotNullableClone(const ColumnPtr & column); @@ -57,7 +59,7 @@ bool typesEqualUpToNullability(DataTypePtr left_type, DataTypePtr right_type); ColumnPtr getColumnAsMask(const Block & block, const String & column_name); /// Split key and other columns by keys name list -void splitAdditionalColumns(const Names & key_names, const Block & sample_block, Block & block_keys, Block & block_others); +void splitAdditionalColumns(const NamesVector & key_names, const Block & sample_block, Block & block_keys, Block & block_others); void changeLowCardinalityInplace(ColumnWithTypeAndName & column); @@ -100,6 +102,9 @@ private: /// Right block saved in Join Block saved_block_sample; + NamesVector key_names_left; + NamesVector key_names_right; + /// Output of join Block result_sample_block; diff --git a/src/Parsers/ASTTablesInSelectQuery.h b/src/Parsers/ASTTablesInSelectQuery.h index 9d31ca6f653..74666e141b1 100644 --- a/src/Parsers/ASTTablesInSelectQuery.h +++ b/src/Parsers/ASTTablesInSelectQuery.h @@ -99,6 +99,8 @@ struct ASTTableJoin : public IAST Strictness strictness = Strictness::Unspecified; Kind kind = Kind::Inner; + bool converted_to_dnf = false; + /// Condition. One of fields is non-nullptr. ASTPtr using_expression_list; ASTPtr on_expression; diff --git a/src/Parsers/IAST_fwd.h b/src/Parsers/IAST_fwd.h index 18ba79d6618..437bd50b2e7 100644 --- a/src/Parsers/IAST_fwd.h +++ b/src/Parsers/IAST_fwd.h @@ -9,5 +9,6 @@ namespace DB class IAST; using ASTPtr = std::shared_ptr; using ASTs = std::vector; +using ASTsVector = std::vector; } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index e1fac21d5c1..10cfad82dd6 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -197,12 +197,12 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes /// Push down is for left table only. We need to update JoinStep for push down into right. /// Only inner and left join are supported. Other types may generate default values for left table keys. /// So, if we push down a condition like `key != 0`, not all rows may be filtered. - if (table_join.kind() == ASTTableJoin::Kind::Inner || table_join.kind() == ASTTableJoin::Kind::Left) + if (table_join.keyNamesLeft().size() == 1 && (table_join.kind() == ASTTableJoin::Kind::Inner || table_join.kind() == ASTTableJoin::Kind::Left)) { const auto & left_header = join->getInputStreams().front().header; const auto & res_header = join->getOutputStream().header; Names allowed_keys; - for (const auto & name : table_join.keyNamesLeft()) + for (const auto & name : table_join.keyNamesLeft().front()) { /// Skip key if it is renamed. /// I don't know if it is possible. Just in case. diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 5c5b12c7475..39e81e6cf75 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -62,7 +62,7 @@ StorageJoin::StorageJoin( if (!metadata_snapshot->getColumns().hasPhysical(key)) throw Exception{"Key column (" + key + ") does not exist in table declaration.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE}; - table_join = std::make_shared(limits, use_nulls, kind, strictness, key_names); + table_join = std::make_shared(limits, use_nulls, kind, strictness, NamesVector{key_names}); join = std::make_shared(table_join, metadata_snapshot->getSampleBlock().sortColumns(), overwrite); restore(); } @@ -410,7 +410,7 @@ protected: return {}; Chunk chunk; - if (!joinDispatch(join->kind, join->strictness, join->data->maps, + if (!joinDispatch(join->kind, join->strictness, join->data->maps.front(), [&](auto kind, auto strictness, auto & map) { chunk = createChunk(map); })) throw Exception("Logical error: unknown JOIN strictness", ErrorCodes::LOGICAL_ERROR); return chunk; From ba8d91245c29e7406c37c44e5c933ed377fc8644 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 25 Jun 2021 15:04:23 +0300 Subject: [PATCH 253/317] ORs in JOINs tests --- .../0_stateless/00863_comma_join_in.sql | 2 +- .../00875_join_right_nulls.reference | 27 ++++ .../0_stateless/00875_join_right_nulls.sql | 22 +++- .../01429_join_on_error_messages.sql | 1 - .../0_stateless/01660_join_or_all.reference | 68 +++++++++++ .../queries/0_stateless/01660_join_or_all.sql | 68 +++++++++++ .../0_stateless/01660_join_or_any.reference | 41 +++++++ .../queries/0_stateless/01660_join_or_any.sql | 52 ++++++++ .../01660_join_or_subqueries.reference | 10 ++ .../0_stateless/01660_join_or_subqueries.sql | 27 ++++ .../0_stateless/01661_join_complex.reference | 8 ++ .../0_stateless/01661_join_complex.sql | 28 +++++ .../0_stateless/01662_join_mixed.reference | 3 + .../queries/0_stateless/01662_join_mixed.sql | 3 + .../01669_join_or_duplicates.reference | 115 ++++++++++++++++++ .../0_stateless/01669_join_or_duplicates.sql | 23 ++++ .../01747_join_view_filter_dictionary.sql | 4 +- 17 files changed, 496 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/01660_join_or_all.reference create mode 100644 tests/queries/0_stateless/01660_join_or_all.sql create mode 100644 tests/queries/0_stateless/01660_join_or_any.reference create mode 100644 tests/queries/0_stateless/01660_join_or_any.sql create mode 100644 tests/queries/0_stateless/01660_join_or_subqueries.reference create mode 100644 tests/queries/0_stateless/01660_join_or_subqueries.sql create mode 100644 tests/queries/0_stateless/01661_join_complex.reference create mode 100644 tests/queries/0_stateless/01661_join_complex.sql create mode 100644 tests/queries/0_stateless/01662_join_mixed.reference create mode 100644 tests/queries/0_stateless/01662_join_mixed.sql create mode 100644 tests/queries/0_stateless/01669_join_or_duplicates.reference create mode 100644 tests/queries/0_stateless/01669_join_or_duplicates.sql diff --git a/tests/queries/0_stateless/00863_comma_join_in.sql b/tests/queries/0_stateless/00863_comma_join_in.sql index ebccd351c8a..f0bf5e4d470 100644 --- a/tests/queries/0_stateless/00863_comma_join_in.sql +++ b/tests/queries/0_stateless/00863_comma_join_in.sql @@ -10,7 +10,7 @@ insert into test1_00863 (id, code) select number, toString(number) FROM numbers( insert into test3_00863 (id, code) select number, toString(number) FROM numbers(100000); insert into test2_00863 (id, code, test1_id, test3_id) select number, toString(number), number, number FROM numbers(100000); -SET max_memory_usage = 50000000; +SET max_memory_usage = 51000000; select test2_00863.id from test1_00863, test2_00863, test3_00863 diff --git a/tests/queries/0_stateless/00875_join_right_nulls.reference b/tests/queries/0_stateless/00875_join_right_nulls.reference index 2751528db29..54b6a6abc11 100644 --- a/tests/queries/0_stateless/00875_join_right_nulls.reference +++ b/tests/queries/0_stateless/00875_join_right_nulls.reference @@ -17,6 +17,33 @@ n rj t id id n fj t 1 1 n fj t id id n fj t \N \N +on with or +n rj n 1 1 +n rj n id id +n rj n \N \N +n a rj n 1 1 +n a rj n id id +n a rj n \N \N +n fj n 1 1 +n fj n id id +n fj n \N \N +n fj n \N \N +t rj n 1 1 +t rj n id id +t rj n \N \N +t fj n 1 1 +t fj n id id +t fj n \N \N +n rj t 1 1 +n rj t id id +n a rj t 1 1 +n a rj t id id +n fj t 1 1 +n fj t id id +n fj t \N \N +n fj t 1 1 +n fj t id id +n fj t \N \N using n rj n 1 1 n rj n id id diff --git a/tests/queries/0_stateless/00875_join_right_nulls.sql b/tests/queries/0_stateless/00875_join_right_nulls.sql index 288a3043569..7d9f7fdf3c9 100644 --- a/tests/queries/0_stateless/00875_join_right_nulls.sql +++ b/tests/queries/0_stateless/00875_join_right_nulls.sql @@ -1,11 +1,15 @@ DROP TABLE IF EXISTS t; DROP TABLE IF EXISTS nt; +DROP TABLE IF EXISTS ntxy; -CREATE TABLE t (x String) ENGINE = Memory; -CREATE TABLE nt (x Nullable(String)) ENGINE = Memory; +CREATE TABLE t (x String) ENGINE = Log(); +CREATE TABLE nt (x Nullable(String)) ENGINE = Log(); +CREATE TABLE ntxy (x Nullable(String), y Nullable(String)) ENGINE = Log(); INSERT INTO t (x) VALUES ('id'), ('1'); INSERT INTO nt (x) VALUES ('id'), (NULL), ('1'); +INSERT INTO ntxy (x, y) VALUES ('id', 'id'), (NULL, NULL), ('1', '1'); + SET join_use_nulls = 1; @@ -20,6 +24,19 @@ SELECT 't fj n', t1.x, t2.x FROM t AS t1 FULL JOIN nt AS t2 ON t1.x = t2.x ORDER SELECT 'n rj t', t1.x, t2.x FROM nt AS t1 RIGHT JOIN t AS t2 ON t1.x = t2.x ORDER BY t1.x; SELECT 'n fj t', t1.x, t2.x FROM nt AS t1 FULL JOIN t AS t2 ON t1.x = t2.x ORDER BY t1.x; +SELECT 'on with or'; +SELECT 'n rj n', t1.x, t2.x FROM nt AS t1 RIGHT JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; +SELECT 'n a rj n', t1.x, t2.x FROM nt AS t1 ANY RIGHT JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; +SELECT 'n fj n', t1.x, t2.x FROM nt AS t1 FULL JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; + +SELECT 't rj n', t1.x, t2.x FROM t AS t1 RIGHT JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; +SELECT 't fj n', t1.x, t2.x FROM t AS t1 FULL JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; + +SELECT 'n rj t', t1.x, t2.x FROM ntxy AS t1 RIGHT JOIN t AS t2 ON t1.x = t2.x OR t1.y = t2.x ORDER BY t1.x; +SELECT 'n a rj t', t1.x, t2.x FROM ntxy AS t1 ANY RIGHT JOIN t AS t2 ON t1.x = t2.x OR t1.y = t2.x ORDER BY t1.x; +SELECT 'n fj t', t1.x, t2.x FROM ntxy AS t1 FULL JOIN t AS t2 ON t1.x = t2.x OR t2.x = t1.y ORDER BY t1.x; +SELECT 'n fj t', t1.x, t2.x FROM ntxy AS t1 FULL JOIN t AS t2 ON t2.x = t1.y OR t1.x = t2.x ORDER BY t1.x; + SELECT 'using'; SELECT 'n rj n', t1.x, t2.x FROM nt AS t1 RIGHT JOIN nt AS t2 USING(x) ORDER BY t1.x; @@ -46,3 +63,4 @@ SELECT sum(isNull(t1.x)), count(t1.x) FROM nt AS t1 FULL JOIN nt AS t2 USING(x); DROP TABLE t; DROP TABLE nt; +DROP TABLE ntxy; diff --git a/tests/queries/0_stateless/01429_join_on_error_messages.sql b/tests/queries/0_stateless/01429_join_on_error_messages.sql index 6e792e90d42..901e0e16e43 100644 --- a/tests/queries/0_stateless/01429_join_on_error_messages.sql +++ b/tests/queries/0_stateless/01429_join_on_error_messages.sql @@ -4,7 +4,6 @@ SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON (A.a = arrayJoin([1])); -- { SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON equals(a); -- { serverError 62 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON less(a); -- { serverError 62 } -SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b OR a = b; -- { serverError 403 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a > b; -- { serverError 403 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a < b; -- { serverError 403 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a >= b; -- { serverError 403 } diff --git a/tests/queries/0_stateless/01660_join_or_all.reference b/tests/queries/0_stateless/01660_join_or_all.reference new file mode 100644 index 00000000000..9c70d462540 --- /dev/null +++ b/tests/queries/0_stateless/01660_join_or_all.reference @@ -0,0 +1,68 @@ +join on OR chain (all left) +2 3 +6 4 +6 4 +== +2 3 +5 4 +100 4 +== +2 3 2 3 +6 4 5 4 +6 4 100 4 +== +1 +== +1 3 +== +1 2 2 3 +== +2 4 +join on OR chain (all right) +2 3 +6 4 +6 4 +== +2 3 +5 4 +100 4 +== +2 3 2 3 +6 4 5 4 +6 4 100 4 +== +1 +0 +== +1 3 +0 4 +== +1 2 2 3 +0 0 6 4 +== +2 4 +6 5 +join on OR chain (full) +2 3 +6 4 +6 4 +== +2 3 +5 4 +100 4 +== +2 3 2 3 +6 4 5 4 +6 4 100 4 +== +1 +0 +== +1 3 +0 4 +== +1 2 2 3 +0 0 6 4 +== +2 4 +6 5 diff --git a/tests/queries/0_stateless/01660_join_or_all.sql b/tests/queries/0_stateless/01660_join_or_all.sql new file mode 100644 index 00000000000..be05ea840ca --- /dev/null +++ b/tests/queries/0_stateless/01660_join_or_all.sql @@ -0,0 +1,68 @@ +SET joined_subquery_requires_alias = 0; +SET max_threads = 1; + +drop table if exists tab1; +drop table if exists tab2; +drop table if exists tab3; + +create table tab1 (a1 Int32, b1 Int32) engine = MergeTree order by a1; +create table tab2 (a2 Int32, b2 Int32) engine = MergeTree order by a2; +create table tab3 (a3 Int32, b3 Int32) engine = MergeTree order by a3; + +insert into tab1 values (1, 2); +insert into tab2 values (2, 3); +insert into tab2 values (6, 4); +insert into tab3 values (2, 3); +insert into tab3 values (5, 4); +insert into tab3 values (100, 4); + +select 'join on OR chain (all left)'; +select a2, b2 from tab2 all left join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a3, b3 from tab2 all left join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a2, b2, a3, b3 from tab2 all left join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a1 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a1, b2 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a1, b1, a2, b2 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a2, b2 + 1 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; + +select 'join on OR chain (all right)'; +select a2, b2 from tab2 all right join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a3, b3 from tab2 all right join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a2, b2, a3, b3 from tab2 all right join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a1 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a1, b2 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a1, b1, a2, b2 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a2, b2 + 1 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; + + +select 'join on OR chain (full)'; +select a2, b2 from tab2 full join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a3, b3 from tab2 full join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a2, b2, a3, b3 from tab2 full join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a1 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a1, b2 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a1, b1, a2, b2 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a2, b2 + 1 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; + + +drop table tab1; +drop table tab2; +drop table tab3; diff --git a/tests/queries/0_stateless/01660_join_or_any.reference b/tests/queries/0_stateless/01660_join_or_any.reference new file mode 100644 index 00000000000..c049a75762f --- /dev/null +++ b/tests/queries/0_stateless/01660_join_or_any.reference @@ -0,0 +1,41 @@ +join on OR chain (any left) +2 3 +6 4 +== +2 3 +5 4 +== +2 3 2 3 +6 4 5 4 +== +1 +== +1 3 +== +1 2 2 3 +== +2 4 +join on OR chain (any right) +2 3 +6 4 +6 4 +== +2 3 +5 4 +100 4 +== +2 3 2 3 +6 4 5 4 +6 4 100 4 +== +1 +0 +== +1 3 +0 4 +== +1 2 2 3 +0 0 6 4 +== +2 4 +6 5 diff --git a/tests/queries/0_stateless/01660_join_or_any.sql b/tests/queries/0_stateless/01660_join_or_any.sql new file mode 100644 index 00000000000..3c5a39bd056 --- /dev/null +++ b/tests/queries/0_stateless/01660_join_or_any.sql @@ -0,0 +1,52 @@ +SET joined_subquery_requires_alias = 0; +SET max_threads = 1; + +drop table if exists tab1; +drop table if exists tab2; +drop table if exists tab3; + +create table tab1 (a1 Int32, b1 Int32) engine = MergeTree order by a1; +create table tab2 (a2 Int32, b2 Int32) engine = MergeTree order by a2; +create table tab3 (a3 Int32, b3 Int32) engine = MergeTree order by a3; + +insert into tab1 values (1, 2); +insert into tab2 values (2, 3); +insert into tab2 values (6, 4); +insert into tab3 values (2, 3); +insert into tab3 values (5, 4); +insert into tab3 values (100, 4); + +select 'join on OR chain (any left)'; +select a2, b2 from tab2 any left join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a3, b3 from tab2 any left join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a2, b2, a3, b3 from tab2 any left join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a1 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a1, b2 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a1, b1, a2, b2 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a2, b2 + 1 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; + + +select 'join on OR chain (any right)'; +select a2, b2 from tab2 any right join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a3, b3 from tab2 any right join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a2, b2, a3, b3 from tab2 any right join tab3 on a2 = a3 or b2 = b3; +select '=='; +select a1 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a1, b2 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a1, b1, a2, b2 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select '=='; +select a2, b2 + 1 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; + +drop table tab1; +drop table tab2; +drop table tab3; diff --git a/tests/queries/0_stateless/01660_join_or_subqueries.reference b/tests/queries/0_stateless/01660_join_or_subqueries.reference new file mode 100644 index 00000000000..ae5e6686d40 --- /dev/null +++ b/tests/queries/0_stateless/01660_join_or_subqueries.reference @@ -0,0 +1,10 @@ +subqueries with OR +1 +== +1 +== +1 2 +subquery column alias with OR +1 2 2 3 +== +1 2 2 3 diff --git a/tests/queries/0_stateless/01660_join_or_subqueries.sql b/tests/queries/0_stateless/01660_join_or_subqueries.sql new file mode 100644 index 00000000000..bdfddf60c8e --- /dev/null +++ b/tests/queries/0_stateless/01660_join_or_subqueries.sql @@ -0,0 +1,27 @@ +SET joined_subquery_requires_alias = 0; +SET max_threads = 1; + +drop table if exists tab1; +drop table if exists tab2; + +create table tab1 (a1 Int32, b1 Int32) engine = MergeTree order by a1; +create table tab2 (a2 Int32, b2 Int32) engine = MergeTree order by a2; + +insert into tab1 values (1, 2); +insert into tab2 values (2, 3); +insert into tab2 values (6, 4); + +select 'subqueries with OR'; +select a1 from tab1 any left join (select * from tab2) on b1 = a2 or b2 = a1; +select '=='; +select a1 from tab1 any left join (select a2, b2 from tab2) on b1 = a2 or b2 = a1; +select '=='; +select a1, b1 from tab1 any left join (select * from tab2) on b1 = a2 or b2 = a1; + +select 'subquery column alias with OR'; +select a1, b1, a2, b2 from tab1 any left join (select *, a2 as z from tab2) on b1 + 1 = z + 1 or b1 = z * 2; +select '=='; +select a1, b1, a2, b2 from tab1 any left join (select *, a2 + 1 as z from tab2) on b1 + 1 = z or b1 = z * 2; + +drop table tab1; +drop table tab2; diff --git a/tests/queries/0_stateless/01661_join_complex.reference b/tests/queries/0_stateless/01661_join_complex.reference new file mode 100644 index 00000000000..8ce5b860a06 --- /dev/null +++ b/tests/queries/0_stateless/01661_join_complex.reference @@ -0,0 +1,8 @@ +a b c d e f a b c d e f +a b c d e f a b c d e f +a b c d e f a b c d e f +a b c d e f a b c d e f +a b c d e f a b c d e f +join on OR/AND chain +2 3 2 3 +6 4 0 0 diff --git a/tests/queries/0_stateless/01661_join_complex.sql b/tests/queries/0_stateless/01661_join_complex.sql new file mode 100644 index 00000000000..309f8fe41e7 --- /dev/null +++ b/tests/queries/0_stateless/01661_join_complex.sql @@ -0,0 +1,28 @@ +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.b = t2.b and t1.c = t2.b and t1.d = t2.b or t1.e = t2.e; +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b or t1.e = t2.e and t1.a=t2.a and t2.f=t1.f; +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 left join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 right join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or t1.e = t2.e; +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) and (t1.e = t2.e or t1.f=t2.f); + + +SET joined_subquery_requires_alias = 0; +SET max_threads = 1; + +drop table if exists tab2; +drop table if exists tab3; + +create table tab2 (a2 Int32, b2 Int32) engine = MergeTree order by a2; +create table tab3 (a3 Int32, b3 Int32) engine = MergeTree order by a3; + +insert into tab2 values (2, 3); +insert into tab2 values (6, 4); +insert into tab3 values (2, 3); +insert into tab3 values (5, 4); +insert into tab3 values (100, 4); + +select 'join on OR/AND chain'; +select a2, b2, a3, b3 from tab2 any left join tab3 on (a2=a3 or b2=b3) and a2 +1 = b3 + 0; + +drop table tab2; +drop table tab3; diff --git a/tests/queries/0_stateless/01662_join_mixed.reference b/tests/queries/0_stateless/01662_join_mixed.reference new file mode 100644 index 00000000000..0df32f791ae --- /dev/null +++ b/tests/queries/0_stateless/01662_join_mixed.reference @@ -0,0 +1,3 @@ +a b 42 a b 42 +=== +a b 42 a b 42 diff --git a/tests/queries/0_stateless/01662_join_mixed.sql b/tests/queries/0_stateless/01662_join_mixed.sql new file mode 100644 index 00000000000..da548a9a679 --- /dev/null +++ b/tests/queries/0_stateless/01662_join_mixed.sql @@ -0,0 +1,3 @@ +select * from (select 'a' as a, 'b' as b, 42 as forty_two) as t1 inner join (select 'a' as a, 'b' as b, 42 as forty_two) as t2 on t1.b = t2.a or t1.forty_two = t2.forty_two; +select '==='; +select * from (select 'a' as a, 'b' as b, 42 as forty_two) as t1 inner join (select 'a' as a, 'b' as b, 42 as forty_two) as t2 on t1.b = t2.b or t1.forty_two = t2.forty_two; diff --git a/tests/queries/0_stateless/01669_join_or_duplicates.reference b/tests/queries/0_stateless/01669_join_or_duplicates.reference new file mode 100644 index 00000000000..3837112c486 --- /dev/null +++ b/tests/queries/0_stateless/01669_join_or_duplicates.reference @@ -0,0 +1,115 @@ +1 left 1 2 1 2 +5 left 1 2 1 2 +5 left 1 2 1 2 +5 left 1 2 1 2 +5 left 1 2 1 2 +5 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +15 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +16 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 left 1 2 1 2 +17 any left 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 any right 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +17 full 1 2 1 2 +555 +a 0 a 0 +a 0 a 1 +a 1 a 1 +a 1 a 0 +a 0 a 0 +a 0 a 1 +a 1 a 0 +a 1 a 1 diff --git a/tests/queries/0_stateless/01669_join_or_duplicates.sql b/tests/queries/0_stateless/01669_join_or_duplicates.sql new file mode 100644 index 00000000000..96e592dd497 --- /dev/null +++ b/tests/queries/0_stateless/01669_join_or_duplicates.sql @@ -0,0 +1,23 @@ +select '1 left', * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(1)) t2 on x = xx or y = yy; + +select '5 left', * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(5)) t2 on x = xx or y = yy; + +select '15 left', * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(15)) t2 on x = xx or y = yy; + +select '16 left', * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(16)) t2 on x = xx or y = yy; + +select '17 left', * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(17)) t2 on x = xx or y = yy; + +select '17 any left', * from (select 1 as x, 2 as y) t1 any left join (select 1 as xx, 2 as yy from numbers(17)) t2 on x = xx or y = yy; + +select '17 right', * from (select 1 as x, 2 as y) t1 right join (select 1 as xx, 2 as yy from numbers(17)) t2 on x = xx or y = yy; + +select '17 any right', * from (select 1 as x, 2 as y) t1 any right join (select 1 as xx, 2 as yy from numbers(17)) t2 on x = xx or y = yy; + +select '17 full', * from (select 1 as x, 2 as y) t1 full join (select 1 as xx, 2 as yy from numbers(17)) t2 on x = xx or y = yy; + +select count(1) from (select * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(555)) t2 on x = xx or y = yy); + +select * from (select 'a' as a, number as c from numbers(2)) as t1 join (select 'a' as a, number as c from numbers(2)) as t2 on t1.c=t2.c or t1.a = t2.a; + +select * from (select 'a' as a, number as c from numbers(2)) as t1 join (select 'a' as a, number as c from numbers(2)) as t2 on t1.a = t2.a or t1.c=t2.c; diff --git a/tests/queries/0_stateless/01747_join_view_filter_dictionary.sql b/tests/queries/0_stateless/01747_join_view_filter_dictionary.sql index 738c12f00ac..11ecaf1ca2e 100644 --- a/tests/queries/0_stateless/01747_join_view_filter_dictionary.sql +++ b/tests/queries/0_stateless/01747_join_view_filter_dictionary.sql @@ -25,8 +25,8 @@ create table dictst01747(some_name String, field1 String, field2 UInt8) Engine = as select 'name', 'test', 33; CREATE DICTIONARY default.dict01747 (some_name String, field1 String, field2 UInt8) -PRIMARY KEY some_name SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 -TABLE dictst01747 DB currentDatabase() USER 'default')) +PRIMARY KEY some_name SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 +TABLE dictst01747 DB currentDatabase() USER 'default')) LIFETIME(MIN 0 MAX 0) LAYOUT(COMPLEX_KEY_HASHED()); From c0ad89016a0ab9f16f1536e39fe60ea43df48c33 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 25 Jun 2021 15:07:07 +0300 Subject: [PATCH 254/317] attempt to set initial memory usage in comma_join_in test --- tests/queries/0_stateless/00863_comma_join_in.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00863_comma_join_in.sql b/tests/queries/0_stateless/00863_comma_join_in.sql index f0bf5e4d470..ebccd351c8a 100644 --- a/tests/queries/0_stateless/00863_comma_join_in.sql +++ b/tests/queries/0_stateless/00863_comma_join_in.sql @@ -10,7 +10,7 @@ insert into test1_00863 (id, code) select number, toString(number) FROM numbers( insert into test3_00863 (id, code) select number, toString(number) FROM numbers(100000); insert into test2_00863 (id, code, test1_id, test3_id) select number, toString(number), number, number FROM numbers(100000); -SET max_memory_usage = 51000000; +SET max_memory_usage = 50000000; select test2_00863.id from test1_00863, test2_00863, test3_00863 From db50015eed22665395b23b9dc78840fa44e696a8 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 23 Jul 2021 22:55:36 +0300 Subject: [PATCH 255/317] review changes 1 - ASTPtr, some comments --- src/Interpreters/CollectJoinOnKeysVisitor.cpp | 19 +++++----- src/Interpreters/CollectJoinOnKeysVisitor.h | 4 +-- src/Interpreters/HashJoin.cpp | 36 ++++++++++--------- src/Interpreters/TableJoin.cpp | 13 ++++--- src/Interpreters/TableJoin.h | 6 ++-- src/Interpreters/TreeRewriter.cpp | 33 +++++++++-------- 6 files changed, 58 insertions(+), 53 deletions(-) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/src/Interpreters/CollectJoinOnKeysVisitor.cpp index cf8e2642255..dc0422ea38f 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -32,21 +32,22 @@ bool isRightIdentifier(JoinIdentifierPos pos) } -void CollectJoinOnKeysMatcher::Data::setDisjuncts(const ASTFunction & func) +void CollectJoinOnKeysMatcher::Data::setDisjuncts(const ASTPtr & ast) { - const auto * expression_list = func.children.front()->as(); - std::vector v; - for (const auto & child : expression_list->children) + auto * func = ast->as(); + const auto * func_args = func->arguments->as(); + std::vector v; + for (const auto & child : func_args->children) { - v.push_back(child.get()); + v.push_back(child); } analyzed_join.setDisjuncts(std::move(v)); } -void CollectJoinOnKeysMatcher::Data::addDisjunct(const ASTFunction & func) +void CollectJoinOnKeysMatcher::Data::addDisjunct(const ASTPtr & ast) { - analyzed_join.addDisjunct(static_cast(&func)); + analyzed_join.addDisjunct(std::move(ast)); } void CollectJoinOnKeysMatcher::Data::addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos) @@ -107,11 +108,11 @@ void CollectJoinOnKeysMatcher::visit(const ASTFunction & func, const ASTPtr & as if (func.name == "or") { // throw Exception("JOIN ON does not support OR. Unexpected '" + queryToString(ast) + "'", ErrorCodes::NOT_IMPLEMENTED); - data.setDisjuncts(func); + data.setDisjuncts(ast); return; } - data.addDisjunct(func); + data.addDisjunct(ast); if (func.name == "and") return; /// go into children diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index 58946b9fdcc..c1960f8bfbd 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -51,8 +51,8 @@ public: void addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos); void addAsofJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos, const ASOF::Inequality & asof_inequality); - void setDisjuncts(const ASTFunction & or_func); - void addDisjunct(const ASTFunction & func); + void setDisjuncts(const ASTPtr & or_func); + void addDisjunct(const ASTPtr & func); void asofToJoinKeys(); }; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index f49b2a3d826..23c441eddce 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -209,7 +209,8 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s if (multiple_disjuncts) { - // required_right_keys_sources concept does not work well if multiple disjuncts + /// required right keys concept does not work well if multiple disjuncts, + /// we need all keys sample_block_with_columns_to_add = right_table_keys = materializeBlock(right_sample_block); } else @@ -1052,22 +1053,23 @@ struct JoinFeatures template class KnownRowsHolder; -// TODO: helper to clean, instead of recreating - +/// Keep already joined rows to prevent duplication if many disjuncts +/// if for a particular pair of rows condition looks like TRUE or TRUE or TRUE +/// we want to have it once in resultset template<> class KnownRowsHolder { public: - using Type = std::pair; + using Type = std::pair; private: - static const size_t MAX_LINEAR = 16; - using LinearHolder = std::array; - using LogHolder = std::set; - using LogHolderPtr = std::unique_ptr; + static const size_t MAX_LINEAR = 16; // threshold to switch from Array to Set + using ArrayHolder = std::array; + using SetHolder = std::set; + using SetHolderPtr = std::unique_ptr; - LinearHolder linh; - LogHolderPtr logh_ptr; + ArrayHolder array_holder; + SetHolderPtr set_holder_ptr; size_t items; @@ -1081,19 +1083,19 @@ public: template void add(InputIt from, InputIt to) { - size_t new_items = std::distance(from, to); + const size_t new_items = std::distance(from, to); if (items + new_items <= MAX_LINEAR) { - std::copy(from, to, &linh[items]); + std::copy(from, to, &array_holder[items]); } else { if (items <= MAX_LINEAR) { - logh_ptr = std::make_unique(); - logh_ptr->insert(std::cbegin(linh), std::cbegin(linh) + items); + set_holder_ptr = std::make_unique(); + set_holder_ptr->insert(std::cbegin(array_holder), std::cbegin(array_holder) + items); } - logh_ptr->insert(from, to); + set_holder_ptr->insert(from, to); } items += new_items; } @@ -1102,8 +1104,8 @@ public: bool isKnown(const Needle & needle) { return items <= MAX_LINEAR - ? std::find(std::cbegin(linh), std::cbegin(linh) + items, needle) != std::cbegin(linh) + items - : logh_ptr->find(needle) != logh_ptr->end(); + ? std::find(std::cbegin(array_holder), std::cbegin(array_holder) + items, needle) != std::cbegin(array_holder) + items + : set_holder_ptr->find(needle) != set_holder_ptr->end(); } }; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 0ec581ad29f..86a68d12471 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -103,10 +103,12 @@ void TableJoin::addUsingKey(const ASTPtr & ast) right_key = renames[right_key]; } - -void TableJoin::addDisjunct(const IAST* addr) +/// create new disjunct when see a child of a previously descovered OR +void TableJoin::addDisjunct(const ASTPtr & ast) { - if (std::find(disjuncts.begin(), disjuncts.end(), addr) != disjuncts.end()) + const IAST * addr = ast.get(); + + if (std::find_if(disjuncts.begin(), disjuncts.end(), [addr](const ASTPtr & ast_){return ast_.get() == addr;}) != disjuncts.end()) { assert(key_names_left.size() == disjunct_num + 1); @@ -121,9 +123,10 @@ void TableJoin::addDisjunct(const IAST* addr) } } -void TableJoin::setDisjuncts(std::vector&& disjuncts_) +/// remember OR's children +void TableJoin::setDisjuncts(std::vector&& disjuncts_) { - disjuncts = disjuncts_; + disjuncts = std::move(disjuncts_); } void TableJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 333c81b1b86..786595fd95f 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -79,7 +79,7 @@ private: ASTsVector on_filter_condition_asts_right; private: size_t disjunct_num = 0; - std::vector disjuncts; + std::vector disjuncts; ASTs key_asts_left; ASTs key_asts_right; @@ -177,8 +177,8 @@ public: void resetCollected(); void addUsingKey(const ASTPtr & ast); - void addDisjunct(const IAST*); - void setDisjuncts(std::vector&&); + void setDisjuncts(std::vector&&); + void addDisjunct(const ASTPtr &); void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast); /* Conditions for left/right table from JOIN ON section. diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 696a2dd9ed2..2e35acaeb4b 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -512,24 +512,23 @@ class DNF void normTree(ASTPtr node) { - auto *function = node->as(); - - // LOG_TRACE(&Poco::Logger::get("TreeRewrite"), "top of normTree: {}", node->dumpTree()); - if (function && function->children.size() == 1) + auto * func = node->as(); + if (func && func->children.size() == 1) { for (bool touched = true; touched;) { touched = false; ASTs new_children; - const auto * expression_list = function->children[0]->as(); - for (const auto & child : expression_list->children) + const auto * func_args = func->arguments->as(); + for (const auto & child : func_args->children) { - auto *f = child->as(); - if (f && function->children.size() == 1 && ((function->name == "or" && f->name == "or") || (function->name == "and" && f->name == "and"))) + auto * child_func = child->as(); + if (child_func && func->children.size() == 1 + && ((func->name == "or" && child_func->name == "or") || (func->name == "and" && child_func->name == "and"))) { - std::copy(child->children[0]->children.begin(), - child->children[0]->children.end(), + std::copy(child_func->arguments->children.begin(), + child_func->arguments->children.end(), std::back_inserter(new_children)); touched = true; } @@ -539,10 +538,10 @@ class DNF } } - function->arguments->children = std::move(new_children); + func->arguments->children = std::move(new_children); } - for (auto & child : function->arguments->children) + for (auto & child : func->arguments->children) { normTree(child); } @@ -558,25 +557,25 @@ class DNF { if (function->name == "and") { - const auto * expression_list = function->children[0]->as(); - if (!expression_list) + const auto * func_args = function->arguments->as(); + if (!func_args) { return node; } - auto or_child = std::find_if(expression_list->children.begin(), expression_list->children.end(), [](ASTPtr arg) + auto or_child = std::find_if(func_args->children.begin(), func_args->children.end(), [](ASTPtr arg) { const auto * f = arg->as(); return f && f->name == "or" && f->children.size() == 1; }); - if (or_child == expression_list->children.end()) + if (or_child == func_args->children.end()) { return node; } ASTs rest_children; - for (const auto & arg : expression_list->children) + for (const auto & arg : func_args->children) { // LOG_DEBUG(&Poco::Logger::get("toDNF"), "IDs {} vs. {}", arg->getTreeHash(), (*or_child)->getTreeHash()); From 634eb2a0a7ab23aeff7b10cc97b0edb91fdd8e7d Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 2 Aug 2021 02:41:47 +0300 Subject: [PATCH 256/317] no errors on OR in join_on_conditions test --- .../0_stateless/01881_join_on_conditions.reference | 12 +++++++++++- .../queries/0_stateless/01881_join_on_conditions.sql | 8 ++++---- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01881_join_on_conditions.reference b/tests/queries/0_stateless/01881_join_on_conditions.reference index e1fac0e7dc3..b270b8424c5 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions.reference +++ b/tests/queries/0_stateless/01881_join_on_conditions.reference @@ -49,6 +49,12 @@ 2 222 2 0 2 222 222 2 AAA AAA 3 333 333 3 BBB BBB +2 222 2 2 AAA AAA +2 222 222 2 AAA AAA +3 333 333 3 BBB BBB +2 222 2 2 AAA AAA +2 222 222 2 AAA AAA +3 333 333 3 BBB BBB -- 2 222 2 2 AAA a 2 222 222 2 AAA AAA @@ -72,7 +78,6 @@ 1 1 1 -1 -- 2 2 @@ -103,6 +108,11 @@ 2 222 2 0 2 222 222 2 AAA AAA 3 333 333 3 BBB BBB +2 222 2 2 AAA AAA +2 222 222 2 AAA AAA +3 333 333 3 BBB BBB +2 222 222 2 AAA AAA +3 333 333 3 BBB BBB -- 2 222 2 2 AAA a 2 222 222 2 AAA AAA diff --git a/tests/queries/0_stateless/01881_join_on_conditions.sql b/tests/queries/0_stateless/01881_join_on_conditions.sql index e074397ff5e..510c261fdbf 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions.sql +++ b/tests/queries/0_stateless/01881_join_on_conditions.sql @@ -63,8 +63,8 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverE SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; +SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- non-equi condition containing columns from different tables doesn't supported yet SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } @@ -125,8 +125,8 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverE SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; +SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- non-equi condition containing columns from different tables doesn't supported yet SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } From a18c6f7f8e20bc3c301cb077525e5503a392bed3 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 2 Aug 2021 03:18:44 +0300 Subject: [PATCH 257/317] merge error fixed --- src/Interpreters/HashJoin.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 23c441eddce..ba6df55949d 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -205,6 +205,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s , right_sample_block(right_sample_block_) , log(&Poco::Logger::get("HashJoin")) { + LOG_DEBUG(log, "Right sample block: {}", right_sample_block.dumpStructure()); bool multiple_disjuncts = key_names_right.size() > 1; if (multiple_disjuncts) @@ -742,10 +743,10 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) BlockWithFlags structured_block = structureRightBlock(block); bool multiple_disjuncts = disjuncts_num > 1; - // if (nullable_right_side && multiple_disjuncts) - // { - // JoinCommon::convertColumnsToNullable(structured_block.block); - // } + if (nullable_right_side && multiple_disjuncts) + { + JoinCommon::convertColumnsToNullable(structured_block.block); + } std::vector join_mask_col_vector(disjuncts_num); // std::vector join_mask_vector(disjuncts_num); bool use_join_mask_col = false; From f94b2cba2d74155c4f292ae4dc27d0ee9b9caed3 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 4 Aug 2021 00:05:59 +0300 Subject: [PATCH 258/317] Fixed bug if OR together with filter conditions, some comments --- src/Interpreters/HashJoin.cpp | 7 +++++-- src/Interpreters/HashJoin.h | 1 - src/Interpreters/MergeJoin.cpp | 3 +++ src/Interpreters/TableJoin.cpp | 10 +++++++++- .../01881_join_on_conditions.reference | 12 ++++-------- .../0_stateless/01881_join_on_conditions.sql | 18 +++++++++++++----- 6 files changed, 34 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index ba6df55949d..a170d767abf 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -206,7 +206,8 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s , log(&Poco::Logger::get("HashJoin")) { LOG_DEBUG(log, "Right sample block: {}", right_sample_block.dumpStructure()); - bool multiple_disjuncts = key_names_right.size() > 1; + const size_t disjuncts_num = key_names_right.size(); + const bool multiple_disjuncts = disjuncts_num > 1; if (multiple_disjuncts) { @@ -235,7 +236,6 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s initRightBlockStructure(data->sample_block); - const size_t disjuncts_num = key_names_right.size(); JoinCommon::createMissedColumns(sample_block_with_columns_to_add); if (table_join->getDictionaryReader()) @@ -1919,6 +1919,9 @@ struct AdderNonJoined /// Stream from not joined earlier rows of the right table. +/// Based on +/// map' offsetInternal saved in used_flags for single disjuncts +/// flags in BlockWithFlags for multiple disjuncts template class NotJoinedHash final : public NotJoinedBlocks::RightColumnsFiller { diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 3d523a6346f..2ad97e70fa8 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -327,7 +327,6 @@ public: std::vector maps; Block sample_block; /// Block as it would appear in the BlockList - // BlocksList blocks; /// Blocks of "right" table. BlocksWithFlagsList blocks; BlockNullmapList blocks_nullmaps; /// Nullmaps for blocks of "right" table (if needed) diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 651a5093508..da775551aad 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -507,6 +507,9 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right ErrorCodes::PARAMETER_OUT_OF_BOUND); } + if (table_join->keyNamesLeft().size() > 1) + throw Exception("MergeJoin does not support OR", ErrorCodes::NOT_IMPLEMENTED); + std::tie(mask_column_name_left, mask_column_name_right) = table_join->joinConditionColumnNames(0); /// Add auxiliary joining keys to join only rows where conditions from JOIN ON sections holds diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 86a68d12471..ffdfe228174 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -112,7 +112,7 @@ void TableJoin::addDisjunct(const ASTPtr & ast) { assert(key_names_left.size() == disjunct_num + 1); - if (!key_names_left[disjunct_num].empty()) + if (!key_names_left[disjunct_num].empty() || !on_filter_condition_asts_left[disjunct_num].empty() || !on_filter_condition_asts_right[disjunct_num].empty()) { disjunct_num++; key_names_left.resize(disjunct_num+1); @@ -120,6 +120,14 @@ void TableJoin::addDisjunct(const ASTPtr & ast) on_filter_condition_asts_left.resize(disjunct_num+1); on_filter_condition_asts_right.resize(disjunct_num+1); } +#ifndef NDEBUG + else + { + /// we already have disjunct #0 , + /// that is why we are skipping left side of the very first AND + assert(!disjunct_num); + } +#endif } } diff --git a/tests/queries/0_stateless/01881_join_on_conditions.reference b/tests/queries/0_stateless/01881_join_on_conditions.reference index b270b8424c5..c85d9240c37 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions.reference +++ b/tests/queries/0_stateless/01881_join_on_conditions.reference @@ -52,12 +52,13 @@ 2 222 2 2 AAA AAA 2 222 222 2 AAA AAA 3 333 333 3 BBB BBB -2 222 2 2 AAA AAA -2 222 222 2 AAA AAA -3 333 333 3 BBB BBB -- 2 222 2 2 AAA a 2 222 222 2 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA -- partial_merge -- -- 222 2 @@ -108,11 +109,6 @@ 2 222 2 0 2 222 222 2 AAA AAA 3 333 333 3 BBB BBB -2 222 2 2 AAA AAA -2 222 222 2 AAA AAA -3 333 333 3 BBB BBB -2 222 222 2 AAA AAA -3 333 333 3 BBB BBB -- 2 222 2 2 AAA a 2 222 222 2 AAA AAA diff --git a/tests/queries/0_stateless/01881_join_on_conditions.sql b/tests/queries/0_stateless/01881_join_on_conditions.sql index 510c261fdbf..46433e27ab2 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions.sql +++ b/tests/queries/0_stateless/01881_join_on_conditions.sql @@ -2,16 +2,18 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t2_nullable; DROP TABLE IF EXISTS t2_lc; +DROP TABLE IF EXISTS t22; CREATE TABLE t1 (`id` Int32, key String, key2 String) ENGINE = TinyLog; CREATE TABLE t2 (`id` Int32, key String, key2 String) ENGINE = TinyLog; CREATE TABLE t2_nullable (`id` Int32, key String, key2 Nullable(String)) ENGINE = TinyLog; CREATE TABLE t2_lc (`id` Int32, key String, key2 LowCardinality(String)) ENGINE = TinyLog; - +CREATE TABLE t22 (`id` Int32, `idd` Int32, `key` String, `key2` String) ENGINE = TinyLog; INSERT INTO t1 VALUES (1, '111', '111'),(2, '222', '2'),(2, '222', '222'),(3, '333', '333'); INSERT INTO t2 VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', 'BBB'),(4, 'CCC', 'CCC'); INSERT INTO t2_nullable VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', NULL),(4, 'CCC', 'CCC'); INSERT INTO t2_lc VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', 'BBB'),(4, 'CCC', 'CCC'); +INSERT INTO t22 VALUES (2, 1, 'AAA', 'AAA'),(2, 3, 'AAA', 'a'),(3, 100, 'BBB', 'BBB'),(4, 101, 'CCC', 'CCC'); SELECT '-- hash_join --'; @@ -63,7 +65,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverE SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- non-equi condition containing columns from different tables doesn't supported yet SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } @@ -73,6 +75,11 @@ SELECT '--'; -- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id); +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2); +SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; +SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; + SET join_algorithm = 'partial_merge'; SELECT '-- partial_merge --'; @@ -90,7 +97,7 @@ SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id == 3; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.key2 == 'BBB'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333'; +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333'; -- { serverError 48 } SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_lc as t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND isNull(t2.key2); SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND t1.key2 like '33%'; @@ -125,8 +132,8 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverE SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } +SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 48 } -- non-equi condition containing columns from different tables doesn't supported yet SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } @@ -139,3 +146,4 @@ DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t2_nullable; DROP TABLE IF EXISTS t2_lc; +DROP TABLE IF EXISTS t22; From b03415bd0e1f5b630784ccd98461f2df7b93e243 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 5 Aug 2021 14:35:16 +0300 Subject: [PATCH 259/317] Fixed other bug if OR together with filter conditions --- src/Interpreters/CollectJoinOnKeysVisitor.cpp | 7 +-- src/Interpreters/CollectJoinOnKeysVisitor.h | 4 +- src/Interpreters/HashJoin.cpp | 63 ++++++++----------- src/Interpreters/TableJoin.cpp | 4 +- src/Interpreters/TableJoin.h | 7 ++- .../01881_join_on_conditions.reference | 6 ++ .../0_stateless/01881_join_on_conditions.sql | 1 + 7 files changed, 44 insertions(+), 48 deletions(-) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/src/Interpreters/CollectJoinOnKeysVisitor.cpp index dc0422ea38f..fc15d242847 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -32,11 +32,11 @@ bool isRightIdentifier(JoinIdentifierPos pos) } -void CollectJoinOnKeysMatcher::Data::setDisjuncts(const ASTPtr & ast) +void CollectJoinOnKeysMatcher::Data::setDisjuncts(const ASTPtr & or_func_ast) { - auto * func = ast->as(); + const auto * func = or_func_ast->as(); const auto * func_args = func->arguments->as(); - std::vector v; + TableJoin::Disjuncts v; for (const auto & child : func_args->children) { v.push_back(child); @@ -107,7 +107,6 @@ void CollectJoinOnKeysMatcher::visit(const ASTFunction & func, const ASTPtr & as { if (func.name == "or") { - // throw Exception("JOIN ON does not support OR. Unexpected '" + queryToString(ast) + "'", ErrorCodes::NOT_IMPLEMENTED); data.setDisjuncts(ast); return; } diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index c1960f8bfbd..54b4ee39478 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -51,8 +51,8 @@ public: void addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos); void addAsofJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos, const ASOF::Inequality & asof_inequality); - void setDisjuncts(const ASTPtr & or_func); - void addDisjunct(const ASTPtr & func); + void setDisjuncts(const ASTPtr & or_func_ast); + void addDisjunct(const ASTPtr & ast); void asofToJoinKeys(); }; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index a170d767abf..59b46a82215 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -759,38 +759,6 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) } - /// Save blocks that do not hold conditions in ON section - ColumnUInt8::MutablePtr not_joined_map = nullptr; - if (isRightOrFull(kind) && use_join_mask_col) - { - /// Save rows that do not hold conditions - not_joined_map = ColumnUInt8::create(block.rows(), 0); - const size_t sz = assert_cast(*(join_mask_col_vector[0])).getData().size(); - for (size_t i = 0; i < sz; ++i) - { - bool add_to_not_joined_map = true; - - /// Condition hold, do not save row - for (size_t d = 0; d < disjuncts_num; ++d) - { - const auto & join_mask = assert_cast(*(join_mask_col_vector[d])).getData(); - if (join_mask[i]) - { - add_to_not_joined_map = false; - break; - } - } - - - // !!! - // /// NULL key will be saved anyway because, do not save twice - // if (save_nullmap && (*null_map)[i]) - // continue; - if (add_to_not_joined_map) - not_joined_map->getData()[i] = 1; - } - } - std::vector null_map_vector; Columns null_map_holder_vector; @@ -847,9 +815,6 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) }); } - if (not_joined_map) - data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map)); - if (!check_limits) return true; @@ -860,9 +825,33 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) } - if (save_a_nullmap && !multiple_disjuncts) + if (!multiple_disjuncts) { - data->blocks_nullmaps.emplace_back(stored_block, null_map_holder_vector[0]); + /// Save blocks that do not hold conditions in ON section + ColumnUInt8::MutablePtr not_joined_map = nullptr; + if (!multiple_disjuncts && isRightOrFull(kind) && join_mask_col_vector[0]) + { + const auto & join_mask = assert_cast(*join_mask_col_vector[0]).getData(); + /// Save rows that do not hold conditions + not_joined_map = ColumnUInt8::create(block.rows(), 0); + for (size_t i = 0, sz = join_mask.size(); i < sz; ++i) + { + /// Condition hold, do not save row + if (join_mask[i]) + continue; + + /// NULL key will be saved anyway because, do not save twice + if (save_a_nullmap && (*null_map_vector[0])[i]) + continue; + + not_joined_map->getData()[i] = 1; + } + } + + if (save_a_nullmap) + data->blocks_nullmaps.emplace_back(stored_block, null_map_holder_vector[0]); + if (not_joined_map) + data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map)); } } diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index ffdfe228174..d4cee1f7527 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -103,7 +103,7 @@ void TableJoin::addUsingKey(const ASTPtr & ast) right_key = renames[right_key]; } -/// create new disjunct when see a child of a previously descovered OR +/// create new disjunct when see a child of a previously discovered OR void TableJoin::addDisjunct(const ASTPtr & ast) { const IAST * addr = ast.get(); @@ -132,7 +132,7 @@ void TableJoin::addDisjunct(const ASTPtr & ast) } /// remember OR's children -void TableJoin::setDisjuncts(std::vector&& disjuncts_) +void TableJoin::setDisjuncts(Disjuncts&& disjuncts_) { disjuncts = std::move(disjuncts_); } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 786595fd95f..263bf5103e8 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -14,7 +14,7 @@ #include #include - +#include namespace DB { @@ -46,6 +46,7 @@ class TableJoin public: using NameToTypeMap = std::unordered_map; + using Disjuncts = std::deque; private: /** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k` @@ -79,7 +80,7 @@ private: ASTsVector on_filter_condition_asts_right; private: size_t disjunct_num = 0; - std::vector disjuncts; + Disjuncts disjuncts; ASTs key_asts_left; ASTs key_asts_right; @@ -177,7 +178,7 @@ public: void resetCollected(); void addUsingKey(const ASTPtr & ast); - void setDisjuncts(std::vector&&); + void setDisjuncts(Disjuncts &&); void addDisjunct(const ASTPtr &); void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast); diff --git a/tests/queries/0_stateless/01881_join_on_conditions.reference b/tests/queries/0_stateless/01881_join_on_conditions.reference index c85d9240c37..e8b6253f04d 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions.reference +++ b/tests/queries/0_stateless/01881_join_on_conditions.reference @@ -49,6 +49,12 @@ 2 222 2 0 2 222 222 2 AAA AAA 3 333 333 3 BBB BBB +0 4 101 CCC CCC +1 111 111 2 1 AAA AAA +2 222 2 0 0 +2 222 222 2 1 AAA AAA +3 333 333 2 3 AAA a +3 333 333 3 100 BBB BBB 2 222 2 2 AAA AAA 2 222 222 2 AAA AAA 3 333 333 3 BBB BBB diff --git a/tests/queries/0_stateless/01881_join_on_conditions.sql b/tests/queries/0_stateless/01881_join_on_conditions.sql index 46433e27ab2..e4379846ffa 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions.sql +++ b/tests/queries/0_stateless/01881_join_on_conditions.sql @@ -57,6 +57,7 @@ SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id = SELECT '--'; SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id, t2.id; +SELECT t1.id, t1.key, t1.key2, t22.id, t22.idd, t22.key, t22.key2 FROM t1 FULL JOIN t22 ON t1.id == t22.id AND t22.key == t22.key2 AND t1.key == t1.key2 OR t1.id = t22.idd AND t1.key = t1.key2 ORDER BY t1.id, t22.id; SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 } From d0990e270a7171d49d329e7de4d050a3d76e7eed Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sat, 7 Aug 2021 00:52:34 +0300 Subject: [PATCH 260/317] making gcc happy, minor improvements in ORs in JOIN --- src/Interpreters/CollectJoinOnKeysVisitor.cpp | 6 +----- src/Interpreters/HashJoin.cpp | 7 ------- src/Interpreters/TableJoin.cpp | 16 ++++------------ src/Interpreters/TableJoin.h | 2 +- 4 files changed, 6 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/src/Interpreters/CollectJoinOnKeysVisitor.cpp index fc15d242847..e6e9c37f3fc 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -36,11 +36,7 @@ void CollectJoinOnKeysMatcher::Data::setDisjuncts(const ASTPtr & or_func_ast) { const auto * func = or_func_ast->as(); const auto * func_args = func->arguments->as(); - TableJoin::Disjuncts v; - for (const auto & child : func_args->children) - { - v.push_back(child); - } + TableJoin::Disjuncts v = func_args->children; analyzed_join.setDisjuncts(std::move(v)); } diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 59b46a82215..b0a3a7e57f9 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -748,15 +748,8 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) JoinCommon::convertColumnsToNullable(structured_block.block); } std::vector join_mask_col_vector(disjuncts_num); - // std::vector join_mask_vector(disjuncts_num); - bool use_join_mask_col = false; for (size_t d = 0; d < disjuncts_num; ++d) - { join_mask_col_vector[d] = JoinCommon::getColumnAsMask(block, condition_mask_column_name_right[d]); - // join_mask_vector[d] = assert_cast(*(join_mask_col_vector[d])).getData(); - if (join_mask_col_vector[d]) - use_join_mask_col = true; - } std::vector null_map_vector; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index d4cee1f7527..a77f8cbb026 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -115,19 +115,11 @@ void TableJoin::addDisjunct(const ASTPtr & ast) if (!key_names_left[disjunct_num].empty() || !on_filter_condition_asts_left[disjunct_num].empty() || !on_filter_condition_asts_right[disjunct_num].empty()) { disjunct_num++; - key_names_left.resize(disjunct_num+1); - key_names_right.resize(disjunct_num+1); - on_filter_condition_asts_left.resize(disjunct_num+1); - on_filter_condition_asts_right.resize(disjunct_num+1); + key_names_left.resize(disjunct_num + 1); + key_names_right.resize(disjunct_num + 1); + on_filter_condition_asts_left.resize(disjunct_num + 1); + on_filter_condition_asts_right.resize(disjunct_num + 1); } -#ifndef NDEBUG - else - { - /// we already have disjunct #0 , - /// that is why we are skipping left side of the very first AND - assert(!disjunct_num); - } -#endif } } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 263bf5103e8..820b5d80390 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -46,7 +46,7 @@ class TableJoin public: using NameToTypeMap = std::unordered_map; - using Disjuncts = std::deque; + using Disjuncts = ASTs; private: /** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k` From 4bddd0bc7185318990e4d866fefda5e0f1021218 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sun, 8 Aug 2021 02:14:22 +0300 Subject: [PATCH 261/317] make PVS check happy with ORs in JOIN --- src/Interpreters/HashJoin.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index b0a3a7e57f9..12267ace74b 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -822,7 +822,7 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) { /// Save blocks that do not hold conditions in ON section ColumnUInt8::MutablePtr not_joined_map = nullptr; - if (!multiple_disjuncts && isRightOrFull(kind) && join_mask_col_vector[0]) + if (isRightOrFull(kind) && join_mask_col_vector[0]) { const auto & join_mask = assert_cast(*join_mask_col_vector[0]).getData(); /// Save rows that do not hold conditions From 68d6a0ee8549582d7da12f44fddd893c13ddbf1d Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 9 Aug 2021 18:58:41 +0300 Subject: [PATCH 262/317] prohibit storage join if ORs --- src/Interpreters/HashJoin.cpp | 5 +++++ tests/queries/0_stateless/00118_storage_join.sql | 2 ++ .../queries/0_stateless/00855_join_with_array_join.reference | 2 ++ tests/queries/0_stateless/00855_join_with_array_join.sql | 2 +- 4 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 12267ace74b..25a289418be 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -2085,6 +2085,11 @@ void HashJoin::reuseJoinedData(const HashJoin & join) { data = join.data; from_storage_join = true; + + bool multiple_disjuncts = key_names_left.size() > 1; + if (multiple_disjuncts) + throw Exception("StorageJoin with ORs is not supported", ErrorCodes::NOT_IMPLEMENTED); + for (auto & map : data->maps) { joinDispatch(kind, strictness, map, [this](auto kind_, auto strictness_, auto & map_) diff --git a/tests/queries/0_stateless/00118_storage_join.sql b/tests/queries/0_stateless/00118_storage_join.sql index 47896d3316c..a855dcd56d8 100644 --- a/tests/queries/0_stateless/00118_storage_join.sql +++ b/tests/queries/0_stateless/00118_storage_join.sql @@ -16,4 +16,6 @@ SELECT k, js1.s, t2.s FROM (SELECT toUInt64(number / 3) AS k, sum(number) as s F SELECT k, js1.s, t2.s FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k; SELECT k, t2.k, js1.s, t2.s FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k; +SELECT k, js1.s, t2.s FROM (SELECT number AS k, number AS s FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN t2 ON js1.k == t2.k OR js1.s == t2.k; -- { serverError 48 } + DROP TABLE t2; diff --git a/tests/queries/0_stateless/00855_join_with_array_join.reference b/tests/queries/0_stateless/00855_join_with_array_join.reference index acb0240fbd2..fe9bd148afd 100644 --- a/tests/queries/0_stateless/00855_join_with_array_join.reference +++ b/tests/queries/0_stateless/00855_join_with_array_join.reference @@ -4,6 +4,8 @@ 4 0 5 0 6 0 +7 1 0 +8 1 0 - 1 0 2 2 a2 diff --git a/tests/queries/0_stateless/00855_join_with_array_join.sql b/tests/queries/0_stateless/00855_join_with_array_join.sql index 642f90d4c6c..05180573525 100644 --- a/tests/queries/0_stateless/00855_join_with_array_join.sql +++ b/tests/queries/0_stateless/00855_join_with_array_join.sql @@ -3,6 +3,7 @@ SET joined_subquery_requires_alias = 0; SELECT ax, c FROM (SELECT [1,2] ax, 0 c) ARRAY JOIN ax JOIN (SELECT 0 c) USING (c); SELECT ax, c FROM (SELECT [3,4] ax, 0 c) JOIN (SELECT 0 c) USING (c) ARRAY JOIN ax; SELECT ax, c FROM (SELECT [5,6] ax, 0 c) s1 JOIN system.one s2 ON s1.c = s2.dummy ARRAY JOIN ax; +SELECT ax, c, d FROM (SELECT [7,8] ax, 1 c, 0 d) s1 JOIN system.one s2 ON s1.c = s2.dummy OR s1.d = s2.dummy ARRAY JOIN ax; SELECT ax, c FROM (SELECT [101,102] ax, 0 c) s1 @@ -48,4 +49,3 @@ JOIN ( SELECT toInt32(dummy) AS dummy FROM system.one ) AS y USING dummy GROUP B DROP TABLE IF EXISTS f; DROP TABLE IF EXISTS d; - From 6b6592fda776d248eb9e2fa7307c9e1f4fb6cdd7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 17 Aug 2021 13:27:23 +0300 Subject: [PATCH 263/317] Use JoinUsedFlags with multiple dijuncts --- src/Interpreters/HashJoin.cpp | 172 +++++++++++++++++++++------------- src/Interpreters/HashJoin.h | 38 +++++--- src/Interpreters/RowRefs.h | 7 ++ 3 files changed, 141 insertions(+), 76 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 25a289418be..396069b4f25 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -57,62 +57,107 @@ struct NotProcessedCrossJoin : public ExtraBlock namespace JoinStuff { - /// Version of `getUsed` with dynamic dispatch + /// for single disjunct bool JoinUsedFlags::getUsedSafe(size_t i) const { - if (flags.empty()) - return !need_flags; - return flags[i].load(); + return getUsedSafe(nullptr, i); } + /// for multiple disjuncts + bool JoinUsedFlags::getUsedSafe(const Block * block_ptr, size_t row_idx) const + { + if (auto it = flags.find(block_ptr); it != flags.end()) + return it->second[row_idx].load(); + return !need_flags; + } + + /// for single disjunct template void JoinUsedFlags::reinit(size_t size) { if constexpr (MapGetter::flagged) { - assert(flags.size() <= size); + assert(flags[nullptr].size() <= size); need_flags = true; - flags = std::vector(size); + flags[nullptr] = std::vector(size); + } + } + + /// for multiple disjuncts + template + void JoinUsedFlags::reinit(const Block * block_ptr) + { + if constexpr (MapGetter::flagged) + { + assert(flags[block_ptr].size() <= block_ptr->rows()); + need_flags = true; + flags[block_ptr] = std::vector(block_ptr->rows()); } } template void JoinUsedFlags::setUsed(const FindResult & f) { - if constexpr (use_flags) + if constexpr (!use_flags) + return; + + /// Could be set simultaneously from different threads. + if constexpr (multiple_disjuncts) { - /// Could be set simultaneously from different threads. - if constexpr (!multiple_disjuncts) - { - flags[f.getOffset()].store(true, std::memory_order_relaxed); - } + auto & mapped = f.getMapped(); + flags[mapped.block][mapped.row_num].store(true, std::memory_order_relaxed); + } + else + { + flags[nullptr][f.getOffset()].store(true, std::memory_order_relaxed); } } template bool JoinUsedFlags::getUsed(const FindResult & f) { - if constexpr (use_flags) + if constexpr (!use_flags) + return true; + + if constexpr (multiple_disjuncts) { - return flags[f.getOffset()].load(); + auto & mapped = f.getMapped(); + return flags[mapped.block][mapped.row_num].load(); + } + else + { + return flags[nullptr][f.getOffset()].load(); } - return true; } template bool JoinUsedFlags::setUsedOnce(const FindResult & f) { - if constexpr (use_flags) + if constexpr (!use_flags) + return true; + + if constexpr (multiple_disjuncts) { - size_t off = f.getOffset(); + auto & mapped = f.getMapped(); + /// fast check to prevent heavy CAS with seq_cst order - if (flags[off].load(std::memory_order_relaxed)) + if (flags[mapped.block][mapped.row_num].load(std::memory_order_relaxed)) return false; bool expected = false; - return flags[off].compare_exchange_strong(expected, true); + return flags[mapped.block][mapped.row_num].compare_exchange_strong(expected, true); + } + else + { + auto off = f.getOffset(); + + /// fast check to prevent heavy CAS with seq_cst order + if (flags[nullptr][off].load(std::memory_order_relaxed)) + return false; + + bool expected = false; + return flags[nullptr][off].compare_exchange_strong(expected, true); } - return true; } } @@ -389,11 +434,14 @@ static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes return KeyGetter(key_columns, key_sizes, nullptr); } +template +using FindResultImpl = ColumnsHashing::columns_hashing_impl::FindResultImpl; + class KeyGetterForDict { public: using Mapped = RowRef; - using FindResult = ColumnsHashing::columns_hashing_impl::FindResultImpl; + using FindResult = FindResultImpl; KeyGetterForDict(const TableJoin & table_join, const ColumnRawPtrs & key_columns) { @@ -681,15 +729,15 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) } -HashJoin::BlockWithFlags HashJoin::structureRightBlock(const Block & block) const +Block HashJoin::structureRightBlock(const Block & block) const { - BlockWithFlags structured_block; + Block structured_block; for (const auto & sample_column : savedBlockSample().getColumnsWithTypeAndName()) { ColumnWithTypeAndName column = block.getByName(sample_column.name); if (sample_column.column->isNullable()) JoinCommon::convertColumnToNullable(column); - structured_block.block.insert(column); + structured_block.insert(column); } return structured_block; @@ -741,12 +789,13 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) ColumnRawPtrs all_key_columns = JoinCommon::materializeColumnsInplace(block, all_key_names_right); - BlockWithFlags structured_block = structureRightBlock(block); + Block structured_block = structureRightBlock(block); bool multiple_disjuncts = disjuncts_num > 1; if (nullable_right_side && multiple_disjuncts) { - JoinCommon::convertColumnsToNullable(structured_block.block); + JoinCommon::convertColumnsToNullable(structured_block); } + std::vector join_mask_col_vector(disjuncts_num); for (size_t d = 0; d < disjuncts_num; ++d) join_mask_col_vector[d] = JoinCommon::getColumnAsMask(block, condition_mask_column_name_right[d]); @@ -760,10 +809,8 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) throw DB::Exception("addJoinedBlock called when HashJoin locked to prevent updates", ErrorCodes::LOGICAL_ERROR); - data->blocks.emplace_back(std::move(structured_block)); - BlockWithFlags & stored_block_with_flags = data->blocks.back(); + BlockWithFlags & stored_block_with_flags = data->blocks.emplace_back(std::move(structured_block)); Block * stored_block = &stored_block_with_flags.block; - stored_block_with_flags.flags = std::vector(stored_block->rows()); if (rows) data->empty = false; @@ -773,7 +820,10 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) for (size_t d = 0; d < disjuncts_num; ++d) { ColumnRawPtrs key_columns(key_names_right_indexes[d].size()); - std::transform(std::cbegin(key_names_right_indexes[d]), std::cend(key_names_right_indexes[d]), std::begin(key_columns), [&](size_t ind){return all_key_columns[ind];}); + std::transform(std::cbegin(key_names_right_indexes[d]), + std::cend(key_names_right_indexes[d]), + std::begin(key_columns), + [&all_key_columns](size_t i){ return all_key_columns[i]; }); /// We will insert to the map only keys, where all components are not NULL. @@ -799,12 +849,12 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) *this, data->type, map, rows, key_columns, key_sizes[d], stored_block, null_map_vector.back(), join_mask_col_vector[d] ? &assert_cast(*join_mask_col_vector[d]).getData() : nullptr, data->pool); - /// Number of buckets + 1 value from zero storage - if (!d) - { + if (multiple_disjuncts) + used_flags.reinit(stored_block); + else + /// Number of buckets + 1 value from zero storage used_flags.reinit(size + 1); - } }); } @@ -1109,7 +1159,12 @@ public: }; template -void addFoundRowAll(const typename Map::mapped_type & mapped, AddedColumns & added, IColumn::Offset & current_offset, KnownRowsHolder & known_rows [[maybe_unused]]) +void addFoundRowAll( + const typename Map::mapped_type & mapped, + AddedColumns & added, + IColumn::Offset & current_offset, + KnownRowsHolder & known_rows [[maybe_unused]], + JoinStuff::JoinUsedFlags * used_flags [[maybe_unused]]) { if constexpr (add_missing) added.applyLazyDefaults(); @@ -1129,9 +1184,11 @@ void addFoundRowAll(const typename Map::mapped_type & mapped, AddedColumns & add new_known_rows_ptr = std::make_unique::Type>>(); } new_known_rows_ptr->push_back(std::make_pair(it->block, it->row_num)); - const HashJoin::BlockWithFlags * block_with_flags = reinterpret_cast(it->block); - - block_with_flags->flags[it->row_num].store(true, std::memory_order_relaxed); + if (used_flags) + { + used_flags->JoinStuff::JoinUsedFlags::setUsedOnce( + FindResultImpl(*it, true, 0)); + } } } @@ -1178,7 +1235,7 @@ NO_INLINE IColumn::Filter joinRightColumns( const std::vector & null_map [[maybe_unused]], JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]]) { - JoinFeatures jf; + constexpr JoinFeatures jf; size_t rows = added_columns.rows_to_add; IColumn::Filter filter; @@ -1192,14 +1249,6 @@ NO_INLINE IColumn::Filter joinRightColumns( size_t disjunct_num = added_columns.key_columns.size(); - // std::vector key_getter_vector; - - // for (size_t d = 0; d < disjunct_num; ++d) - // { - // auto key_getter = createKeyGetter(added_columns.key_columns[d], added_columns.key_sizes[d]); - // key_getter_vector.push_back(std::move(key_getter)); - // } - IColumn::Offset current_offset = 0; for (size_t i = 0; i < rows; ++i) @@ -1237,7 +1286,11 @@ NO_INLINE IColumn::Filter joinRightColumns( if (const RowRef * found = mapped.findAsof(asof_type, asof_inequality, left_asof_key, i)) { setUsed(filter, i); - used_flags.template setUsed(find_result); + if constexpr (multiple_disjuncts) + used_flags.template setUsed(FindResultImpl(found, true, 0)); + else + used_flags.template setUsed(find_result); + added_columns.appendFromBlock(*found->block, found->row_num); } else @@ -1247,17 +1300,18 @@ NO_INLINE IColumn::Filter joinRightColumns( { setUsed(filter, i); used_flags.template setUsed(find_result); - addFoundRowAll(mapped, added_columns, current_offset, known_rows); + auto used_flags_opt = jf.need_flags ? &used_flags : nullptr; + addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); } else if constexpr ((jf.is_any_join || jf.is_semi_join) && jf.right) { /// Use first appeared left key + it needs left columns replication bool used_once = used_flags.template setUsedOnce(find_result); - if (used_once) { + auto used_flags_opt = jf.need_flags ? &used_flags : nullptr; setUsed(filter, i); - addFoundRowAll(mapped, added_columns, current_offset, known_rows); + addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); } } else if constexpr (jf.is_any_join && KIND == ASTTableJoin::Kind::Inner) @@ -1287,12 +1341,6 @@ NO_INLINE IColumn::Filter joinRightColumns( setUsed(filter, i); used_flags.template setUsed(find_result); added_columns.appendFromBlock(*mapped.block, mapped.row_num); - if constexpr (multiple_disjuncts) - { - const HashJoin::BlockWithFlags * block_with_flags = reinterpret_cast(mapped.block); - block_with_flags->flags[mapped.row_num].store(true, std::memory_order_relaxed); - } - if (jf.is_any_join) { @@ -1979,16 +2027,15 @@ private: for (auto & it = *used_position; it != end && rows_added < max_block_size; ++it) { - const HashJoin::BlockWithFlags & block_with_flags = *it; + const Block & mapped_block = it->block; - for (size_t row = 0; row < block_with_flags.flags.size(); ++row) + for (size_t row = 0; row < mapped_block.rows(); ++row) { - if (!block_with_flags.flags[row]) + if (!parent.isUsed(&mapped_block, row)) { for (size_t colnum = 0; colnum < columns_keys_and_right.size(); ++colnum) { - auto clmn = block_with_flags.block.getByPosition(colnum).column; - columns_keys_and_right[colnum]->insertFrom(*clmn, row); + columns_keys_and_right[colnum]->insertFrom(*mapped_block.getByPosition(colnum).column, row); } ++rows_added; @@ -2017,7 +2064,6 @@ private: continue; AdderNonJoined::add(mapped, rows_added, columns_keys_and_right); - if (rows_added >= max_block_size) { ++it; diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 2ad97e70fa8..35282585d3e 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -35,18 +35,27 @@ namespace JoinStuff /// Flags needed to implement RIGHT and FULL JOINs. class JoinUsedFlags { - std::vector flags; + using RawBlockPtr = const Block *; + using UsedFlagsForBlock = std::vector; + + /// For multiple dijuncts each empty in hashmap stores flags for particular block + /// For single dicunct we store all flags in `nullptr` entry, index is the offset in FindResult + std::unordered_map flags; + bool need_flags; public: - /// Update size for vector with flags. /// Calling this method invalidates existing flags. /// It can be called several times, but all of them should happen before using this structure. template void reinit(size_t size_); + template + void reinit(const Block * block_ptr); + bool getUsedSafe(size_t i) const; + bool getUsedSafe(const Block * block_ptr, size_t row_idx) const; template void setUsed(const T & f); @@ -309,17 +318,22 @@ public: using MapsAsof = MapsTemplate; using MapsVariant = std::variant; - using BlockNullmapList = std::deque>; - using BlockUsedmapList = std::deque>; - struct BlockWithFlags : public ExtraBlock + using RawBlockPtr = const Block *; + using BlockNullmapList = std::deque>; + + struct BlockWithFlags { - mutable std::vector flags; + explicit BlockWithFlags(Block && block_) + : block(std::move(block_)) + {} + + Block block; + bool empty() const { return !block; } }; using BlocksWithFlagsList = std::list; - struct RightTableData { Type type = Type::EMPTY; @@ -334,7 +348,7 @@ public: Arena pool; }; - using RightTableDataPtr = std::shared_ptr; + using RightTableDataPtr = std::shared_ptr; /// We keep correspondence between used_flags and hash table internal buffer. /// Hash table cannot be modified during HashJoin lifetime and must be protected with lock. @@ -345,12 +359,10 @@ public: void reuseJoinedData(const HashJoin & join); - std::shared_ptr getJoinedData() const - { - return data; - } + RightTableDataPtr getJoinedData() const { return data; } bool isUsed(size_t off) const { return used_flags.getUsedSafe(off); } + bool isUsed(const Block * block_ptr, size_t row_idx) const { return used_flags.getUsedSafe(block_ptr, row_idx); } private: template friend class NotJoinedHash; @@ -414,7 +426,7 @@ private: const Block & savedBlockSample() const { return data->sample_block; } /// Modify (structure) right block to save it in block list - BlockWithFlags structureRightBlock(const Block & stored_block) const; + Block structureRightBlock(const Block & stored_block) const; void initRightBlockStructure(Block & saved_block_sample); template diff --git a/src/Interpreters/RowRefs.h b/src/Interpreters/RowRefs.h index 8e5141be199..047146d569c 100644 --- a/src/Interpreters/RowRefs.h +++ b/src/Interpreters/RowRefs.h @@ -77,6 +77,13 @@ struct RowRefList : RowRef return &batch->row_refs[position]; } + const RowRef * operator * () const + { + if (first) + return root; + return &batch->row_refs[position]; + } + void operator ++ () { if (first) From 24e3e04203cc9f1198115ada816b7ee6b6d7ad9f Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 18 Aug 2021 16:28:15 +0300 Subject: [PATCH 264/317] BlockWithFlags removed from vdimir's change --- src/Interpreters/HashJoin.cpp | 22 +++++++--------------- src/Interpreters/HashJoin.h | 14 +------------- src/Interpreters/JoinSwitcher.cpp | 10 +++++----- 3 files changed, 13 insertions(+), 33 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 396069b4f25..29194861b85 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -548,7 +548,7 @@ size_t HashJoin::getTotalRowCount() const if (data->type == Type::CROSS) { for (const auto & block : data->blocks) - res += block.block.rows(); + res += block.rows(); } else if (data->type != Type::DICT) { @@ -568,7 +568,7 @@ size_t HashJoin::getTotalByteCount() const if (data->type == Type::CROSS) { for (const auto & block : data->blocks) - res += block.block.bytes(); + res += block.bytes(); } else if (data->type != Type::DICT) { @@ -809,8 +809,8 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) throw DB::Exception("addJoinedBlock called when HashJoin locked to prevent updates", ErrorCodes::LOGICAL_ERROR); - BlockWithFlags & stored_block_with_flags = data->blocks.emplace_back(std::move(structured_block)); - Block * stored_block = &stored_block_with_flags.block; + data->blocks.emplace_back(std::move(structured_block)); + Block * stored_block = &data->blocks.back(); if (rows) data->empty = false; @@ -1694,10 +1694,8 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) for (size_t left_row = start_left_row; left_row < rows_left; ++left_row) { size_t block_number = 0; - for (const auto & block_wrapper : data->blocks) + for (const Block & block_right : data->blocks) { - const Block & block_right = block_wrapper.block; - ++block_number; if (block_number < start_right_block) continue; @@ -1964,11 +1962,6 @@ public: size_t fillColumns(MutableColumns & columns_right) override { - // if (multiple_disjuncts && parent.nullable_right_side) - // { - // JoinCommon::convertColumnsToNullable(columns_right); - // } - size_t rows_added = 0; auto fill_callback = [&](auto, auto strictness, auto & map) { @@ -1990,10 +1983,9 @@ private: const HashJoin & parent; UInt64 max_block_size; - std::any position; std::optional nulls_position; - std::optional used_position; + std::optional used_position; template size_t fillColumnsFromMap(const Maps & maps, MutableColumns & columns_keys_and_right) @@ -2027,7 +2019,7 @@ private: for (auto & it = *used_position; it != end && rows_added < max_block_size; ++it) { - const Block & mapped_block = it->block; + const Block & mapped_block = *it; for (size_t row = 0; row < mapped_block.rows(); ++row) { diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 35282585d3e..52ca382377f 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -322,18 +322,6 @@ public: using RawBlockPtr = const Block *; using BlockNullmapList = std::deque>; - struct BlockWithFlags - { - explicit BlockWithFlags(Block && block_) - : block(std::move(block_)) - {} - - Block block; - bool empty() const { return !block; } - }; - - using BlocksWithFlagsList = std::list; - struct RightTableData { Type type = Type::EMPTY; @@ -341,7 +329,7 @@ public: std::vector maps; Block sample_block; /// Block as it would appear in the BlockList - BlocksWithFlagsList blocks; + BlocksList blocks; BlockNullmapList blocks_nullmaps; /// Nullmaps for blocks of "right" table (if needed) /// Additional data - strings for string keys and continuation elements of single-linked lists of references to rows. diff --git a/src/Interpreters/JoinSwitcher.cpp b/src/Interpreters/JoinSwitcher.cpp index 599755e43ea..480d105ebb6 100644 --- a/src/Interpreters/JoinSwitcher.cpp +++ b/src/Interpreters/JoinSwitcher.cpp @@ -51,7 +51,7 @@ bool JoinSwitcher::addJoinedBlock(const Block & block, bool) void JoinSwitcher::switchJoin() { std::shared_ptr joined_data = static_cast(*join).getJoinedData(); - HashJoin::BlocksWithFlagsList right_blocks = std::move(joined_data->blocks); + BlocksList right_blocks = std::move(joined_data->blocks); /// Destroy old join & create new one. Early destroy for memory saving. join = std::make_shared(table_join, right_sample_block); @@ -62,20 +62,20 @@ void JoinSwitcher::switchJoin() if (!right_blocks.empty()) { positions.reserve(right_sample_block.columns()); - const HashJoin::BlockWithFlags & tmp_block = *right_blocks.begin(); + const Block & tmp_block = *right_blocks.begin(); for (const auto & sample_column : right_sample_block) { - positions.emplace_back(tmp_block.block.getPositionByName(sample_column.name)); + positions.emplace_back(tmp_block.getPositionByName(sample_column.name)); is_nullable.emplace_back(sample_column.type->isNullable()); } } - for (HashJoin::BlockWithFlags & saved_block : right_blocks) + for (Block & saved_block : right_blocks) { Block restored_block; for (size_t i = 0; i < positions.size(); ++i) { - auto & column = saved_block.block.getByPosition(positions[i]); + auto & column = saved_block.getByPosition(positions[i]); restored_block.insert(correctNullability(std::move(column), is_nullable[i])); } join->addJoinedBlock(restored_block); From a8e1de51e1076a8483ff4fa68be67b600de89d95 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 20 Aug 2021 17:32:17 +0300 Subject: [PATCH 265/317] fix TableJoin::leftToRightKeyRemap --- src/Interpreters/TableJoin.cpp | 27 ++++++++++++++++++++------- src/Interpreters/TableJoin.h | 5 +++++ 2 files changed, 25 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index a77f8cbb026..2564b00077d 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -582,6 +582,25 @@ void TableJoin::addJoinCondition(const ASTPtr & ast, bool is_left) on_filter_condition_asts_right[disjunct_num].push_back(ast); } +void TableJoin::leftToRightKeyRemap( + const Names & left_keys, + const Names & right_keys, + const NameSet & required_right_keys, + std::unordered_map & key_map) const +{ + if (hasUsing()) + { + for (size_t i = 0; i < left_keys.size(); ++i) + { + const String & left_key_name = left_keys[i]; + const String & right_key_name = right_keys[i]; + + if (!required_right_keys.contains(right_key_name)) + key_map[left_key_name] = right_key_name; + } + } +} + std::unordered_map TableJoin::leftToRightKeyRemap() const { std::unordered_map left_to_right_key_remap; @@ -589,13 +608,7 @@ std::unordered_map TableJoin::leftToRightKeyRemap() const { const auto & required_right_keys = requiredRightKeys(); for (size_t i = 0; i < key_names_left.size(); ++i) - { - const String & left_key_name = key_names_left[i]; - const String & right_key_name = key_names_right[i]; - - if (!required_right_keys.contains(right_key_name)) - left_to_right_key_remap[left_key_name] = right_key_name; - } + TableJoin::leftToRightKeyRemap(key_names_left[i], key_names_right[i], required_right_keys, left_to_right_key_remap); } return left_to_right_key_remap; } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 820b5d80390..9624134b1eb 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -123,6 +123,11 @@ private: bool inferJoinKeyCommonType(const LeftNamesAndTypes & left, const RightNamesAndTypes & right, bool allow_right); NamesAndTypesList correctedColumnsAddedByJoin() const; + void leftToRightKeyRemap( + const Names & left_keys, + const Names & right_keys, + const NameSet & required_right_keys, + std::unordered_map & key_map) const; public: TableJoin() From 38374ba07b52e0051e9b34c4f426990185c4ae66 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 23 Aug 2021 14:09:33 +0300 Subject: [PATCH 266/317] fix join->getNonJoinedBlocks --- src/Interpreters/HashJoin.cpp | 9 ++++++--- src/Interpreters/HashJoin.h | 3 ++- src/Interpreters/IJoin.h | 2 +- src/Interpreters/JoinSwitcher.h | 5 +++-- src/Interpreters/MergeJoin.cpp | 6 ++++-- src/Interpreters/MergeJoin.h | 2 +- src/Processors/Transforms/JoiningTransform.cpp | 3 ++- 7 files changed, 19 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 29194861b85..ebaef2d7bb9 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -2092,7 +2092,9 @@ private: } }; -std::shared_ptr HashJoin::getNonJoinedBlocks(const Block & result_sample_block, UInt64 max_block_size) const +std::shared_ptr HashJoin::getNonJoinedBlocks(const Block & left_sample_block, + const Block & result_sample_block, + UInt64 max_block_size) const { if (table_join->strictness() == ASTTableJoin::Strictness::Asof || table_join->strictness() == ASTTableJoin::Strictness::Semi || @@ -2106,14 +2108,15 @@ std::shared_ptr HashJoin::getNonJoinedBlocks(const Block & resu { /// ... calculate `left_columns_count` ... // throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "TODO"); - size_t left_columns_count = result_sample_block.columns() - required_right_keys.columns() - sample_block_with_columns_to_add.columns(); + size_t left_columns_count = left_sample_block.columns(); auto non_joined = std::make_unique>(*this, max_block_size); return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); } else { - size_t left_columns_count = result_sample_block.columns() - required_right_keys.columns() - sample_block_with_columns_to_add.columns(); + size_t left_columns_count = left_sample_block.columns(); + assert(left_columns_count == result_sample_block.columns() - required_right_keys.columns() - sample_block_with_columns_to_add.columns()); auto non_joined = std::make_unique>(*this, max_block_size); return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); } diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 52ca382377f..64fa26218a3 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -178,7 +178,8 @@ public: * Use only after all calls to joinBlock was done. * left_sample_block is passed without account of 'use_nulls' setting (columns will be converted to Nullable inside). */ - std::shared_ptr getNonJoinedBlocks(const Block & result_sample_block, UInt64 max_block_size) const override; + std::shared_ptr getNonJoinedBlocks( + const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; /// Number of keys in all built JOIN maps. size_t getTotalRowCount() const final; diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index c48524e2031..620ec6107c4 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -45,7 +45,7 @@ public: /// Different query plan is used for such joins. virtual bool isFilled() const { return false; } - virtual std::shared_ptr getNonJoinedBlocks(const Block &, UInt64) const = 0; + virtual std::shared_ptr getNonJoinedBlocks(const Block &, const Block &, UInt64) const = 0; }; using JoinPtr = std::shared_ptr; diff --git a/src/Interpreters/JoinSwitcher.h b/src/Interpreters/JoinSwitcher.h index 83ac70044a8..86d8f9d1ba0 100644 --- a/src/Interpreters/JoinSwitcher.h +++ b/src/Interpreters/JoinSwitcher.h @@ -61,9 +61,10 @@ public: return join->alwaysReturnsEmptySet(); } - std::shared_ptr getNonJoinedBlocks(const Block & block, UInt64 max_block_size) const override + std::shared_ptr getNonJoinedBlocks( + const Block & left_sample_block, const Block & result_block, UInt64 max_block_size) const override { - return join->getNonJoinedBlocks(block, max_block_size); + return join->getNonJoinedBlocks(left_sample_block, result_block, max_block_size); } private: diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index da775551aad..9a54f64af72 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -1101,11 +1101,13 @@ private: }; -std::shared_ptr MergeJoin::getNonJoinedBlocks(const Block & result_sample_block, UInt64 max_block_size) const +std::shared_ptr MergeJoin::getNonJoinedBlocks( + const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const { if (table_join->strictness() == ASTTableJoin::Strictness::All && (is_right || is_full)) { - size_t left_columns_count = result_sample_block.columns() - right_columns_to_add.columns(); + size_t left_columns_count = left_sample_block.columns(); + assert(left_columns_count == result_sample_block.columns() - right_columns_to_add.columns()); auto non_joined = std::make_unique(*this, max_block_size); return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); } diff --git a/src/Interpreters/MergeJoin.h b/src/Interpreters/MergeJoin.h index 53cde2bff7c..8782a2f7535 100644 --- a/src/Interpreters/MergeJoin.h +++ b/src/Interpreters/MergeJoin.h @@ -36,7 +36,7 @@ public: /// Has to be called only after setTotals()/mergeRightBlocks() bool alwaysReturnsEmptySet() const override { return (is_right || is_inner) && min_max_right_blocks.empty(); } - std::shared_ptr getNonJoinedBlocks(const Block & result_sample_block, UInt64 max_block_size) const override; + std::shared_ptr getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override; private: friend class NotJoinedMerge; diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index aceb04d7994..08f70f28a7f 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -124,7 +124,8 @@ void JoiningTransform::work() return; } - non_joined_blocks = join->getNonJoinedBlocks(outputs.front().getHeader(), max_block_size); + non_joined_blocks = join->getNonJoinedBlocks( + inputs.front().getHeader(), outputs.front().getHeader(), max_block_size); if (!non_joined_blocks) { process_non_joined = false; From d37b88fb078d9acaa89257b6cf9e6a3b0929a798 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 23 Aug 2021 15:59:47 +0300 Subject: [PATCH 267/317] fix HashJoin::initRightBlockStructure --- src/Interpreters/HashJoin.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index ebaef2d7bb9..e96ba9f7b75 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -701,8 +701,9 @@ namespace void HashJoin::initRightBlockStructure(Block & saved_block_sample) { + bool multiple_disjuncts = key_names_right.size() > 1; /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). - bool save_key_columns = !table_join->forceHashJoin() || isRightOrFull(kind) || key_names_right.size() > 1; + bool save_key_columns = !table_join->forceHashJoin() || isRightOrFull(kind) || multiple_disjuncts; if (save_key_columns) { saved_block_sample = right_table_keys.cloneEmpty(); @@ -724,7 +725,7 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) if (nullable_right_side) { - JoinCommon::convertColumnsToNullable(saved_block_sample, (isFull(kind) ? right_table_keys.columns() : 0)); + JoinCommon::convertColumnsToNullable(saved_block_sample, (isFull(kind) && !multiple_disjuncts ? right_table_keys.columns() : 0)); } } From fc6b8ed655e2c86ea03a8f7090250e70bd99e952 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 24 Aug 2021 11:00:07 +0300 Subject: [PATCH 268/317] Test join_use_nulls in 01881_join_on_conditions --- .../01881_join_on_conditions.reference | 120 ----------------- .../01881_join_on_conditions.reference.j2 | 126 ++++++++++++++++++ ...ns.sql => 01881_join_on_conditions.sql.j2} | 15 ++- 3 files changed, 138 insertions(+), 123 deletions(-) delete mode 100644 tests/queries/0_stateless/01881_join_on_conditions.reference create mode 100644 tests/queries/0_stateless/01881_join_on_conditions.reference.j2 rename tests/queries/0_stateless/{01881_join_on_conditions.sql => 01881_join_on_conditions.sql.j2} (97%) diff --git a/tests/queries/0_stateless/01881_join_on_conditions.reference b/tests/queries/0_stateless/01881_join_on_conditions.reference deleted file mode 100644 index e8b6253f04d..00000000000 --- a/tests/queries/0_stateless/01881_join_on_conditions.reference +++ /dev/null @@ -1,120 +0,0 @@ --- hash_join -- --- -222 2 -222 222 -333 333 --- -222 222 -333 333 --- -222 -333 --- -1 -1 -1 -1 -1 -1 -1 -1 -1 --- -2 -2 -3 -2 -3 -2 -3 -2 -3 -2 -3 -2 -3 -2 -3 -2 -3 --- -222 2 -333 3 -222 2 -333 3 --- -0 2 AAA a -0 4 CCC CCC -1 111 111 0 -2 222 2 0 -2 222 222 2 AAA AAA -3 333 333 3 BBB BBB -0 4 101 CCC CCC -1 111 111 2 1 AAA AAA -2 222 2 0 0 -2 222 222 2 1 AAA AAA -3 333 333 2 3 AAA a -3 333 333 3 100 BBB BBB -2 222 2 2 AAA AAA -2 222 222 2 AAA AAA -3 333 333 3 BBB BBB --- -2 222 2 2 AAA a -2 222 222 2 AAA AAA -t22 1 111 111 2 1 AAA AAA -t22 1 111 111 2 1 AAA AAA -t22 1 111 111 2 1 AAA AAA -t22 1 111 111 2 1 AAA AAA --- partial_merge -- --- -222 2 -222 222 -333 333 --- -222 222 -333 333 --- -222 -333 --- -1 -1 -1 -1 -1 -1 -1 -1 --- -2 -2 -3 -2 -3 -2 -3 -2 -3 -2 -3 -2 -3 -2 -3 -2 -3 --- -222 2 -333 3 -222 2 -333 3 --- -0 2 AAA a -0 4 CCC CCC -1 111 111 0 -2 222 2 0 -2 222 222 2 AAA AAA -3 333 333 3 BBB BBB --- -2 222 2 2 AAA a -2 222 222 2 AAA AAA diff --git a/tests/queries/0_stateless/01881_join_on_conditions.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions.reference.j2 new file mode 100644 index 00000000000..b2f162d471c --- /dev/null +++ b/tests/queries/0_stateless/01881_join_on_conditions.reference.j2 @@ -0,0 +1,126 @@ +{%- macro jnull(join_use_nulls, value='') -%} +{#- default value or null if join_use_nulls is enabled -#} +{% if join_use_nulls == 1 %}\N{% else %}{{ value }}{% endif %} +{%- endmacro -%} +{% for jn in [0, 1] -%} +-- hash_join -- +-- +222 2 +222 222 +333 333 +-- +222 222 +333 333 +-- +222 +333 +-- +1 +1 +1 +1 +1 +1 +1 +1 +1 +-- +2 +2 +3 +2 +3 +2 +3 +2 +3 +2 +3 +2 +3 +2 +3 +2 +3 +-- +222 2 +333 3 +222 2 +333 3 +-- +{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 2 AAA a +{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 4 CCC CCC +1 111 111 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} +2 222 2 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} +2 222 222 2 AAA AAA +3 333 333 3 BBB BBB +{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 4 101 CCC CCC +1 111 111 2 1 AAA AAA +2 222 2 {{ jnull(jn, 0) }} {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} +2 222 222 2 1 AAA AAA +3 333 333 2 3 AAA a +3 333 333 3 100 BBB BBB +2 222 2 2 AAA AAA +2 222 222 2 AAA AAA +3 333 333 3 BBB BBB +-- +2 222 2 2 AAA a +2 222 222 2 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +-- partial_merge -- +-- +222 2 +222 222 +333 333 +-- +222 222 +333 333 +-- +222 +333 +-- +1 +1 +1 +1 +1 +1 +1 +1 +-- +2 +2 +3 +2 +3 +2 +3 +2 +3 +2 +3 +2 +3 +2 +3 +2 +3 +-- +222 2 +333 3 +222 2 +333 3 +-- +{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 2 AAA a +{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 4 CCC CCC +1 111 111 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} +2 222 2 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} +2 222 222 2 AAA AAA +3 333 333 3 BBB BBB +-- +2 222 2 2 AAA a +2 222 222 2 AAA AAA +{% endfor -%} diff --git a/tests/queries/0_stateless/01881_join_on_conditions.sql b/tests/queries/0_stateless/01881_join_on_conditions.sql.j2 similarity index 97% rename from tests/queries/0_stateless/01881_join_on_conditions.sql rename to tests/queries/0_stateless/01881_join_on_conditions.sql.j2 index e4379846ffa..5981512f8c9 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions.sql +++ b/tests/queries/0_stateless/01881_join_on_conditions.sql.j2 @@ -15,6 +15,12 @@ INSERT INTO t2_nullable VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', NULL INSERT INTO t2_lc VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', 'BBB'),(4, 'CCC', 'CCC'); INSERT INTO t22 VALUES (2, 1, 'AAA', 'AAA'),(2, 3, 'AAA', 'a'),(3, 100, 'BBB', 'BBB'),(4, 101, 'CCC', 'CCC'); +{% for join_use_nulls in [0, 1] -%} + +SET join_use_nulls = {{ join_use_nulls }}; + +SET join_algorithm = 'hash'; + SELECT '-- hash_join --'; SELECT '--'; @@ -56,8 +62,8 @@ SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id = SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toUInt8(e + 1); SELECT '--'; -SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id, t2.id; -SELECT t1.id, t1.key, t1.key2, t22.id, t22.idd, t22.key, t22.key2 FROM t1 FULL JOIN t22 ON t1.id == t22.id AND t22.key == t22.key2 AND t1.key == t1.key2 OR t1.id = t22.idd AND t1.key = t1.key2 ORDER BY t1.id, t22.id; +SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id NULLS FIRST, t2.id NULLS FIRST; +SELECT t1.id, t1.key, t1.key2, t22.id, t22.idd, t22.key, t22.key2 FROM t1 FULL JOIN t22 ON t1.id == t22.id AND t22.key == t22.key2 AND t1.key == t1.key2 OR t1.id = t22.idd AND t1.key = t1.key2 ORDER BY t1.id NULLS FIRST, t22.id NULLS FIRST; SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 } @@ -124,7 +130,7 @@ SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id = SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toUInt8(e + 1); SELECT '--'; -SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id, t2.id; +SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id NULLS FIRST, t2.id NULLS FIRST; SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 } @@ -143,6 +149,9 @@ SELECT '--'; -- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; + +{% endfor -%} + DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2; DROP TABLE IF EXISTS t2_nullable; From c7b93e3324f62993f8fe9ca5d32df02f69c99a35 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 24 Aug 2021 12:39:32 +0300 Subject: [PATCH 269/317] Fix non deterministic behaviour in join_or_all/any tests --- .../0_stateless/01660_join_or_all.reference | 12 ++--- .../queries/0_stateless/01660_join_or_all.sql | 47 +++++++++---------- .../0_stateless/01660_join_or_any.reference | 14 +++--- .../queries/0_stateless/01660_join_or_any.sql | 31 ++++++------ 4 files changed, 52 insertions(+), 52 deletions(-) diff --git a/tests/queries/0_stateless/01660_join_or_all.reference b/tests/queries/0_stateless/01660_join_or_all.reference index 9c70d462540..d746b5fb899 100644 --- a/tests/queries/0_stateless/01660_join_or_all.reference +++ b/tests/queries/0_stateless/01660_join_or_all.reference @@ -31,14 +31,14 @@ join on OR chain (all right) 6 4 5 4 6 4 100 4 == -1 0 +1 == -1 3 0 4 +1 3 == -1 2 2 3 0 0 6 4 +1 2 2 3 == 2 4 6 5 @@ -55,14 +55,14 @@ join on OR chain (full) 6 4 5 4 6 4 100 4 == -1 0 +1 == -1 3 0 4 +1 3 == -1 2 2 3 0 0 6 4 +1 2 2 3 == 2 4 6 5 diff --git a/tests/queries/0_stateless/01660_join_or_all.sql b/tests/queries/0_stateless/01660_join_or_all.sql index be05ea840ca..a9c4d8e1102 100644 --- a/tests/queries/0_stateless/01660_join_or_all.sql +++ b/tests/queries/0_stateless/01660_join_or_all.sql @@ -1,5 +1,4 @@ SET joined_subquery_requires_alias = 0; -SET max_threads = 1; drop table if exists tab1; drop table if exists tab2; @@ -10,58 +9,58 @@ create table tab2 (a2 Int32, b2 Int32) engine = MergeTree order by a2; create table tab3 (a3 Int32, b3 Int32) engine = MergeTree order by a3; insert into tab1 values (1, 2); + insert into tab2 values (2, 3); insert into tab2 values (6, 4); + insert into tab3 values (2, 3); insert into tab3 values (5, 4); insert into tab3 values (100, 4); select 'join on OR chain (all left)'; -select a2, b2 from tab2 all left join tab3 on a2 = a3 or b2 = b3; +select a2, b2 from tab2 all left join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2; select '=='; -select a3, b3 from tab2 all left join tab3 on a2 = a3 or b2 = b3; +select a3, b3 from tab2 all left join tab3 on a2 = a3 or b2 = b3 ORDER BY a3, b3; select '=='; -select a2, b2, a3, b3 from tab2 all left join tab3 on a2 = a3 or b2 = b3; +select a2, b2, a3, b3 from tab2 all left join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2, a3, b3; select '=='; -select a1 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1; select '=='; -select a1, b2 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1, b2 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b2; select '=='; -select a1, b1, a2, b2 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1, b1, a2, b2 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b1, a2, b2; select '=='; -select a2, b2 + 1 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a2, b2 + 1 from tab1 all left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a2, b2 + 1; select 'join on OR chain (all right)'; -select a2, b2 from tab2 all right join tab3 on a2 = a3 or b2 = b3; +select a2, b2 from tab2 all right join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2; select '=='; -select a3, b3 from tab2 all right join tab3 on a2 = a3 or b2 = b3; +select a3, b3 from tab2 all right join tab3 on a2 = a3 or b2 = b3 ORDER BY a3, b3; select '=='; -select a2, b2, a3, b3 from tab2 all right join tab3 on a2 = a3 or b2 = b3; +select a2, b2, a3, b3 from tab2 all right join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2, a3, b3; select '=='; -select a1 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1; select '=='; -select a1, b2 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1, b2 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b2; select '=='; -select a1, b1, a2, b2 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1, b1, a2, b2 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b1, a2, b2; select '=='; -select a2, b2 + 1 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; - +select a2, b2 + 1 from tab1 all right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a2, b2 + 1; select 'join on OR chain (full)'; -select a2, b2 from tab2 full join tab3 on a2 = a3 or b2 = b3; +select a2, b2 from tab2 full join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2; select '=='; -select a3, b3 from tab2 full join tab3 on a2 = a3 or b2 = b3; +select a3, b3 from tab2 full join tab3 on a2 = a3 or b2 = b3 ORDER BY a3, b3; select '=='; -select a2, b2, a3, b3 from tab2 full join tab3 on a2 = a3 or b2 = b3; +select a2, b2, a3, b3 from tab2 full join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2, a3, b3; select '=='; -select a1 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1; select '=='; -select a1, b2 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1, b2 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b2; select '=='; -select a1, b1, a2, b2 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1, b1, a2, b2 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b1, a2, b2; select '=='; -select a2, b2 + 1 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; - +select a2, b2 + 1 from tab1 full join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a2, b2 + 1; drop table tab1; drop table tab2; diff --git a/tests/queries/0_stateless/01660_join_or_any.reference b/tests/queries/0_stateless/01660_join_or_any.reference index c049a75762f..8724cbe7ba5 100644 --- a/tests/queries/0_stateless/01660_join_or_any.reference +++ b/tests/queries/0_stateless/01660_join_or_any.reference @@ -2,11 +2,11 @@ join on OR chain (any left) 2 3 6 4 == -2 3 -5 4 +0 3 +1 4 == -2 3 2 3 -6 4 5 4 +2 3 0 3 +6 4 1 4 == 1 == @@ -28,14 +28,14 @@ join on OR chain (any right) 6 4 5 4 6 4 100 4 == -1 0 +1 == -1 3 0 4 +1 3 == -1 2 2 3 0 0 6 4 +1 2 2 3 == 2 4 6 5 diff --git a/tests/queries/0_stateless/01660_join_or_any.sql b/tests/queries/0_stateless/01660_join_or_any.sql index 3c5a39bd056..deb1260e9ee 100644 --- a/tests/queries/0_stateless/01660_join_or_any.sql +++ b/tests/queries/0_stateless/01660_join_or_any.sql @@ -1,5 +1,4 @@ SET joined_subquery_requires_alias = 0; -SET max_threads = 1; drop table if exists tab1; drop table if exists tab2; @@ -10,42 +9,44 @@ create table tab2 (a2 Int32, b2 Int32) engine = MergeTree order by a2; create table tab3 (a3 Int32, b3 Int32) engine = MergeTree order by a3; insert into tab1 values (1, 2); + insert into tab2 values (2, 3); insert into tab2 values (6, 4); + insert into tab3 values (2, 3); insert into tab3 values (5, 4); insert into tab3 values (100, 4); select 'join on OR chain (any left)'; -select a2, b2 from tab2 any left join tab3 on a2 = a3 or b2 = b3; +select a2, b2 from tab2 any left join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2; select '=='; -select a3, b3 from tab2 any left join tab3 on a2 = a3 or b2 = b3; +select a3 == 5 OR a3 == 100, b3 from tab2 any left join tab3 on a2 = a3 or b2 = b3 ORDER BY a3, b3; select '=='; -select a2, b2, a3, b3 from tab2 any left join tab3 on a2 = a3 or b2 = b3; +select a2, b2, a3 == 5 OR a3 == 100, b3 from tab2 any left join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2, b3; select '=='; -select a1 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1; select '=='; -select a1, b2 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1, b2 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b2; select '=='; -select a1, b1, a2, b2 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1, b1, a2, b2 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b1, a2, b2; select '=='; -select a2, b2 + 1 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a2, b2 + 1 from tab1 any left join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a2, b2 + 1; select 'join on OR chain (any right)'; -select a2, b2 from tab2 any right join tab3 on a2 = a3 or b2 = b3; +select a2, b2 from tab2 any right join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2; select '=='; -select a3, b3 from tab2 any right join tab3 on a2 = a3 or b2 = b3; +select a3, b3 from tab2 any right join tab3 on a2 = a3 or b2 = b3 ORDER BY a3, b3; select '=='; -select a2, b2, a3, b3 from tab2 any right join tab3 on a2 = a3 or b2 = b3; +select a2, b2, a3, b3 from tab2 any right join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2, a3, b3; select '=='; -select a1 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1; select '=='; -select a1, b2 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1, b2 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b2; select '=='; -select a1, b1, a2, b2 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a1, b1, a2, b2 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a1, b1, a2, b2; select '=='; -select a2, b2 + 1 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2; +select a2, b2 + 1 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a2, b2 + 1; drop table tab1; drop table tab2; From be034444bad4326ba55bca2c4d747cced723abf1 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 25 Aug 2021 01:22:44 +0300 Subject: [PATCH 270/317] fix any_join_distinct_right_table_keys crash if ORs in JOIN --- src/Interpreters/HashJoin.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index e96ba9f7b75..3d6f3a44378 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1066,6 +1066,7 @@ template struct JoinFeatures { static constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any; + static constexpr bool is_any_or_old_join = STRICTNESS == ASTTableJoin::Strictness::Any || STRICTNESS == ASTTableJoin::Strictness::RightAny; static constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All; static constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof; static constexpr bool is_semi_join = STRICTNESS == ASTTableJoin::Strictness::Semi; @@ -1343,7 +1344,7 @@ NO_INLINE IColumn::Filter joinRightColumns( used_flags.template setUsed(find_result); added_columns.appendFromBlock(*mapped.block, mapped.row_num); - if (jf.is_any_join) + if (jf.is_any_or_old_join) { break; } From 5ed6b26c9dd1a06737d29800b91007e02dadf54c Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 25 Aug 2021 13:18:29 +0300 Subject: [PATCH 271/317] any_join_distinct_right_table_keys tests and fix for ORs in JOIN --- src/Interpreters/HashJoin.cpp | 4 +- .../0_stateless/01660_join_or_any.reference | 8 ++++ .../queries/0_stateless/01660_join_or_any.sql | 9 +++++ .../0_stateless/01660_join_or_inner.reference | 12 ++++++ .../0_stateless/01660_join_or_inner.sql | 39 +++++++++++++++++++ 5 files changed, 70 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01660_join_or_inner.reference create mode 100644 tests/queries/0_stateless/01660_join_or_inner.sql diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 3d6f3a44378..a0048c47c1d 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1066,7 +1066,7 @@ template struct JoinFeatures { static constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any; - static constexpr bool is_any_or_old_join = STRICTNESS == ASTTableJoin::Strictness::Any || STRICTNESS == ASTTableJoin::Strictness::RightAny; + static constexpr bool is_any_or_semi_join = STRICTNESS == ASTTableJoin::Strictness::Any || STRICTNESS == ASTTableJoin::Strictness::RightAny || (STRICTNESS == ASTTableJoin::Strictness::Semi && KIND == ASTTableJoin::Kind::Left); static constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All; static constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof; static constexpr bool is_semi_join = STRICTNESS == ASTTableJoin::Strictness::Semi; @@ -1344,7 +1344,7 @@ NO_INLINE IColumn::Filter joinRightColumns( used_flags.template setUsed(find_result); added_columns.appendFromBlock(*mapped.block, mapped.row_num); - if (jf.is_any_or_old_join) + if (jf.is_any_or_semi_join) { break; } diff --git a/tests/queries/0_stateless/01660_join_or_any.reference b/tests/queries/0_stateless/01660_join_or_any.reference index 8724cbe7ba5..39123f6ef9b 100644 --- a/tests/queries/0_stateless/01660_join_or_any.reference +++ b/tests/queries/0_stateless/01660_join_or_any.reference @@ -39,3 +39,11 @@ join on OR chain (any right) == 2 4 6 5 +any_join_distinct_right_table_keys = 1 +0 0 +2 3 +6 4 +== +2 3 +5 4 +100 4 diff --git a/tests/queries/0_stateless/01660_join_or_any.sql b/tests/queries/0_stateless/01660_join_or_any.sql index deb1260e9ee..62c9dd680be 100644 --- a/tests/queries/0_stateless/01660_join_or_any.sql +++ b/tests/queries/0_stateless/01660_join_or_any.sql @@ -1,4 +1,6 @@ SET joined_subquery_requires_alias = 0; +SET any_join_distinct_right_table_keys = 0; + drop table if exists tab1; drop table if exists tab2; @@ -48,6 +50,13 @@ select a1, b1, a2, b2 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 select '=='; select a2, b2 + 1 from tab1 any right join tab2 on b1 + 1 = a2 + 1 or a1 + 4 = b2 + 2 ORDER BY a2, b2 + 1; +SET any_join_distinct_right_table_keys = 1; +select 'any_join_distinct_right_table_keys = 1'; +select a2, b2 from tab2 any right join tab3 on a2 = a3 or b2 = b3 ORDER BY a2, b2; +select '=='; +select a3, b3 from tab2 any right join tab3 on a2 = a3 or b2 = b3 ORDER BY a3, b3; + + drop table tab1; drop table tab2; drop table tab3; diff --git a/tests/queries/0_stateless/01660_join_or_inner.reference b/tests/queries/0_stateless/01660_join_or_inner.reference new file mode 100644 index 00000000000..eb12265358f --- /dev/null +++ b/tests/queries/0_stateless/01660_join_or_inner.reference @@ -0,0 +1,12 @@ +any_join_distinct_right_table_keys = 0 +2 3 2 3 +6 4 5 4 +== +2 3 2 3 +6 4 5 4 +any_join_distinct_right_table_keys = 1 +2 3 2 3 +6 4 5 4 +== +2 3 2 3 +6 4 5 4 diff --git a/tests/queries/0_stateless/01660_join_or_inner.sql b/tests/queries/0_stateless/01660_join_or_inner.sql new file mode 100644 index 00000000000..9b93940631d --- /dev/null +++ b/tests/queries/0_stateless/01660_join_or_inner.sql @@ -0,0 +1,39 @@ +SET joined_subquery_requires_alias = 0; + +drop table if exists tab1; +drop table if exists tab2; +drop table if exists tab3; + +create table tab1 (a1 Int32, b1 Int32) engine = MergeTree order by a1; +create table tab2 (a2 Int32, b2 Int32) engine = MergeTree order by a2; +create table tab3 (a3 Int32, b3 Int32) engine = MergeTree order by a3; + +insert into tab1 values (1, 2); + +insert into tab2 values (2, 3); +insert into tab2 values (6, 4); +insert into tab2 values (998, 999); + +insert into tab3 values (2, 3); +insert into tab3 values (5, 4); +insert into tab3 values (100, 4); +insert into tab3 values (1998, 1999); + +set max_threads = 1; + +SET any_join_distinct_right_table_keys = 0; +select 'any_join_distinct_right_table_keys = 0'; +select tab2.*, tab3.* from tab2 any join tab3 on a2 = a3 or b2 = b3; +select '=='; +select tab2.*, tab3.* from tab2 any join tab3 on b2 = b3 or a2 = a3; + +SET any_join_distinct_right_table_keys = 1; +select 'any_join_distinct_right_table_keys = 1'; +select tab2.*, tab3.* from tab2 any join tab3 on a2 = a3 or b2 = b3; +select '=='; +select tab2.*, tab3.* from tab2 any join tab3 on b2 = b3 or a2 = a3; + + +drop table tab1; +drop table tab2; +drop table tab3; From aa3d8a431a4cde7293d344f1da66d4b7f65cf9a3 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 27 Aug 2021 17:03:39 +0300 Subject: [PATCH 272/317] No redundant convertColumnsToNullable if ORs in JOIN --- src/Interpreters/HashJoin.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index a0048c47c1d..f6e95825f38 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -792,10 +792,6 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) Block structured_block = structureRightBlock(block); bool multiple_disjuncts = disjuncts_num > 1; - if (nullable_right_side && multiple_disjuncts) - { - JoinCommon::convertColumnsToNullable(structured_block); - } std::vector join_mask_col_vector(disjuncts_num); for (size_t d = 0; d < disjuncts_num; ++d) From 8339cfc8e1bc4ade7c7254ac3946791871f3eb74 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sun, 29 Aug 2021 00:29:12 +0300 Subject: [PATCH 273/317] split long lasting tests for JOINs --- .../00875_join_right_nulls.reference | 27 ------ .../0_stateless/00875_join_right_nulls.sql | 17 ---- .../00875_join_right_nulls_ors.reference | 27 ++++++ .../00875_join_right_nulls_ors.sql | 31 +++++++ ...1881_join_on_conditions_hash.reference.j2} | 53 ----------- ...2 => 01881_join_on_conditions_hash.sql.j2} | 63 ------------- ...1881_join_on_conditions_merge.reference.j2 | 59 ++++++++++++ .../01881_join_on_conditions_merge.sql.j2 | 91 +++++++++++++++++++ 8 files changed, 208 insertions(+), 160 deletions(-) create mode 100644 tests/queries/0_stateless/00875_join_right_nulls_ors.reference create mode 100644 tests/queries/0_stateless/00875_join_right_nulls_ors.sql rename tests/queries/0_stateless/{01881_join_on_conditions.reference.j2 => 01881_join_on_conditions_hash.reference.j2} (69%) rename tests/queries/0_stateless/{01881_join_on_conditions.sql.j2 => 01881_join_on_conditions_hash.sql.j2} (58%) create mode 100644 tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 create mode 100644 tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 diff --git a/tests/queries/0_stateless/00875_join_right_nulls.reference b/tests/queries/0_stateless/00875_join_right_nulls.reference index 54b6a6abc11..2751528db29 100644 --- a/tests/queries/0_stateless/00875_join_right_nulls.reference +++ b/tests/queries/0_stateless/00875_join_right_nulls.reference @@ -17,33 +17,6 @@ n rj t id id n fj t 1 1 n fj t id id n fj t \N \N -on with or -n rj n 1 1 -n rj n id id -n rj n \N \N -n a rj n 1 1 -n a rj n id id -n a rj n \N \N -n fj n 1 1 -n fj n id id -n fj n \N \N -n fj n \N \N -t rj n 1 1 -t rj n id id -t rj n \N \N -t fj n 1 1 -t fj n id id -t fj n \N \N -n rj t 1 1 -n rj t id id -n a rj t 1 1 -n a rj t id id -n fj t 1 1 -n fj t id id -n fj t \N \N -n fj t 1 1 -n fj t id id -n fj t \N \N using n rj n 1 1 n rj n id id diff --git a/tests/queries/0_stateless/00875_join_right_nulls.sql b/tests/queries/0_stateless/00875_join_right_nulls.sql index 7d9f7fdf3c9..8aa7aee8c15 100644 --- a/tests/queries/0_stateless/00875_join_right_nulls.sql +++ b/tests/queries/0_stateless/00875_join_right_nulls.sql @@ -1,14 +1,11 @@ DROP TABLE IF EXISTS t; DROP TABLE IF EXISTS nt; -DROP TABLE IF EXISTS ntxy; CREATE TABLE t (x String) ENGINE = Log(); CREATE TABLE nt (x Nullable(String)) ENGINE = Log(); -CREATE TABLE ntxy (x Nullable(String), y Nullable(String)) ENGINE = Log(); INSERT INTO t (x) VALUES ('id'), ('1'); INSERT INTO nt (x) VALUES ('id'), (NULL), ('1'); -INSERT INTO ntxy (x, y) VALUES ('id', 'id'), (NULL, NULL), ('1', '1'); SET join_use_nulls = 1; @@ -24,19 +21,6 @@ SELECT 't fj n', t1.x, t2.x FROM t AS t1 FULL JOIN nt AS t2 ON t1.x = t2.x ORDER SELECT 'n rj t', t1.x, t2.x FROM nt AS t1 RIGHT JOIN t AS t2 ON t1.x = t2.x ORDER BY t1.x; SELECT 'n fj t', t1.x, t2.x FROM nt AS t1 FULL JOIN t AS t2 ON t1.x = t2.x ORDER BY t1.x; -SELECT 'on with or'; -SELECT 'n rj n', t1.x, t2.x FROM nt AS t1 RIGHT JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; -SELECT 'n a rj n', t1.x, t2.x FROM nt AS t1 ANY RIGHT JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; -SELECT 'n fj n', t1.x, t2.x FROM nt AS t1 FULL JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; - -SELECT 't rj n', t1.x, t2.x FROM t AS t1 RIGHT JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; -SELECT 't fj n', t1.x, t2.x FROM t AS t1 FULL JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; - -SELECT 'n rj t', t1.x, t2.x FROM ntxy AS t1 RIGHT JOIN t AS t2 ON t1.x = t2.x OR t1.y = t2.x ORDER BY t1.x; -SELECT 'n a rj t', t1.x, t2.x FROM ntxy AS t1 ANY RIGHT JOIN t AS t2 ON t1.x = t2.x OR t1.y = t2.x ORDER BY t1.x; -SELECT 'n fj t', t1.x, t2.x FROM ntxy AS t1 FULL JOIN t AS t2 ON t1.x = t2.x OR t2.x = t1.y ORDER BY t1.x; -SELECT 'n fj t', t1.x, t2.x FROM ntxy AS t1 FULL JOIN t AS t2 ON t2.x = t1.y OR t1.x = t2.x ORDER BY t1.x; - SELECT 'using'; SELECT 'n rj n', t1.x, t2.x FROM nt AS t1 RIGHT JOIN nt AS t2 USING(x) ORDER BY t1.x; @@ -63,4 +47,3 @@ SELECT sum(isNull(t1.x)), count(t1.x) FROM nt AS t1 FULL JOIN nt AS t2 USING(x); DROP TABLE t; DROP TABLE nt; -DROP TABLE ntxy; diff --git a/tests/queries/0_stateless/00875_join_right_nulls_ors.reference b/tests/queries/0_stateless/00875_join_right_nulls_ors.reference new file mode 100644 index 00000000000..05febe97d7a --- /dev/null +++ b/tests/queries/0_stateless/00875_join_right_nulls_ors.reference @@ -0,0 +1,27 @@ +on with or +n rj n 1 1 +n rj n id id +n rj n \N \N +n a rj n 1 1 +n a rj n id id +n a rj n \N \N +n fj n 1 1 +n fj n id id +n fj n \N \N +n fj n \N \N +t rj n 1 1 +t rj n id id +t rj n \N \N +t fj n 1 1 +t fj n id id +t fj n \N \N +n rj t 1 1 +n rj t id id +n a rj t 1 1 +n a rj t id id +n fj t 1 1 +n fj t id id +n fj t \N \N +n fj t 1 1 +n fj t id id +n fj t \N \N diff --git a/tests/queries/0_stateless/00875_join_right_nulls_ors.sql b/tests/queries/0_stateless/00875_join_right_nulls_ors.sql new file mode 100644 index 00000000000..7f41565fbf7 --- /dev/null +++ b/tests/queries/0_stateless/00875_join_right_nulls_ors.sql @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS t; +DROP TABLE IF EXISTS nt; +DROP TABLE IF EXISTS ntxy; + +CREATE TABLE t (x String) ENGINE = Log(); +CREATE TABLE nt (x Nullable(String)) ENGINE = Log(); +CREATE TABLE ntxy (x Nullable(String), y Nullable(String)) ENGINE = Log(); + +INSERT INTO t (x) VALUES ('id'), ('1'); +INSERT INTO nt (x) VALUES ('id'), (NULL), ('1'); +INSERT INTO ntxy (x, y) VALUES ('id', 'id'), (NULL, NULL), ('1', '1'); + + +SET join_use_nulls = 1; + +SELECT 'on with or'; +SELECT 'n rj n', t1.x, t2.x FROM nt AS t1 RIGHT JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; +SELECT 'n a rj n', t1.x, t2.x FROM nt AS t1 ANY RIGHT JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; +SELECT 'n fj n', t1.x, t2.x FROM nt AS t1 FULL JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; + +SELECT 't rj n', t1.x, t2.x FROM t AS t1 RIGHT JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; +SELECT 't fj n', t1.x, t2.x FROM t AS t1 FULL JOIN ntxy AS t2 ON t1.x = t2.x OR t1.x = t2.y ORDER BY t1.x; + +SELECT 'n rj t', t1.x, t2.x FROM ntxy AS t1 RIGHT JOIN t AS t2 ON t1.x = t2.x OR t1.y = t2.x ORDER BY t1.x; +SELECT 'n a rj t', t1.x, t2.x FROM ntxy AS t1 ANY RIGHT JOIN t AS t2 ON t1.x = t2.x OR t1.y = t2.x ORDER BY t1.x; +SELECT 'n fj t', t1.x, t2.x FROM ntxy AS t1 FULL JOIN t AS t2 ON t1.x = t2.x OR t2.x = t1.y ORDER BY t1.x; +SELECT 'n fj t', t1.x, t2.x FROM ntxy AS t1 FULL JOIN t AS t2 ON t2.x = t1.y OR t1.x = t2.x ORDER BY t1.x; + +DROP TABLE t; +DROP TABLE nt; +DROP TABLE ntxy; diff --git a/tests/queries/0_stateless/01881_join_on_conditions.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 similarity index 69% rename from tests/queries/0_stateless/01881_join_on_conditions.reference.j2 rename to tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 index b2f162d471c..94b3e19483f 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions.reference.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 @@ -70,57 +70,4 @@ t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA --- partial_merge -- --- -222 2 -222 222 -333 333 --- -222 222 -333 333 --- -222 -333 --- -1 -1 -1 -1 -1 -1 -1 -1 --- -2 -2 -3 -2 -3 -2 -3 -2 -3 -2 -3 -2 -3 -2 -3 -2 -3 --- -222 2 -333 3 -222 2 -333 3 --- -{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 2 AAA a -{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 4 CCC CCC -1 111 111 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} -2 222 2 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} -2 222 222 2 AAA AAA -3 333 333 3 BBB BBB --- -2 222 2 2 AAA a -2 222 222 2 AAA AAA {% endfor -%} diff --git a/tests/queries/0_stateless/01881_join_on_conditions.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 similarity index 58% rename from tests/queries/0_stateless/01881_join_on_conditions.sql.j2 rename to tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index 5981512f8c9..9230db75c03 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -87,69 +87,6 @@ SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22 SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -SET join_algorithm = 'partial_merge'; - -SELECT '-- partial_merge --'; - -SELECT '--'; -SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2; -SELECT '--'; -SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2; - -SELECT '--'; -SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2; - -SELECT '--'; -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id > 2; -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id == 3; -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.key2 == 'BBB'; -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333'; -- { serverError 48 } -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_lc as t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND isNull(t2.key2); -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND t1.key2 like '33%'; -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t1.key); - --- DISTINCT is used to remove the difference between 'hash' and 'merge' join: 'merge' doesn't support `any_join_distinct_right_table_keys` - -SELECT '--'; -SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND t2.key2 != ''; -SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(t2.key2 != ''); -SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(t2.key2 != ''); -SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(toNullable(t2.key2 != '')); -SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(toLowCardinality(t2.key2 != '')); -SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(t1.key2 != ''); -SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(t1.key2 != ''); -SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(toNullable(t1.key2 != '')); -SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(toLowCardinality(t1.key2 != '')); - -SELECT '--'; -SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND e; --- `e + 1` is UInt16 -SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND e + 1; -- { serverError 403 } -SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toUInt8(e + 1); - -SELECT '--'; -SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id NULLS FIRST, t2.id NULLS FIRST; - -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError 403 } -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 } -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id + 2; -- { serverError 403 } -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverError 403 } -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 403 } -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 48 } --- non-equi condition containing columns from different tables doesn't supported yet -SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } -SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } - -SELECT '--'; --- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key -SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; - - {% endfor -%} DROP TABLE IF EXISTS t1; diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 new file mode 100644 index 00000000000..fdc9ecfbbef --- /dev/null +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 @@ -0,0 +1,59 @@ +{%- macro jnull(join_use_nulls, value='') -%} +{#- default value or null if join_use_nulls is enabled -#} +{% if join_use_nulls == 1 %}\N{% else %}{{ value }}{% endif %} +{%- endmacro -%} +{% for jn in [0, 1] -%} +-- partial_merge -- +-- +222 2 +222 222 +333 333 +-- +222 222 +333 333 +-- +222 +333 +-- +1 +1 +1 +1 +1 +1 +1 +1 +-- +2 +2 +3 +2 +3 +2 +3 +2 +3 +2 +3 +2 +3 +2 +3 +2 +3 +-- +222 2 +333 3 +222 2 +333 3 +-- +{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 2 AAA a +{{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 4 CCC CCC +1 111 111 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} +2 222 2 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} +2 222 222 2 AAA AAA +3 333 333 3 BBB BBB +-- +2 222 2 2 AAA a +2 222 222 2 AAA AAA +{% endfor -%} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 new file mode 100644 index 00000000000..ba0f644504d --- /dev/null +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 @@ -0,0 +1,91 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t2_nullable; +DROP TABLE IF EXISTS t2_lc; +DROP TABLE IF EXISTS t22; + +CREATE TABLE t1 (`id` Int32, key String, key2 String) ENGINE = TinyLog; +CREATE TABLE t2 (`id` Int32, key String, key2 String) ENGINE = TinyLog; +CREATE TABLE t2_nullable (`id` Int32, key String, key2 Nullable(String)) ENGINE = TinyLog; +CREATE TABLE t2_lc (`id` Int32, key String, key2 LowCardinality(String)) ENGINE = TinyLog; +CREATE TABLE t22 (`id` Int32, `idd` Int32, `key` String, `key2` String) ENGINE = TinyLog; +INSERT INTO t1 VALUES (1, '111', '111'),(2, '222', '2'),(2, '222', '222'),(3, '333', '333'); +INSERT INTO t2 VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', 'BBB'),(4, 'CCC', 'CCC'); +INSERT INTO t2_nullable VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', NULL),(4, 'CCC', 'CCC'); +INSERT INTO t2_lc VALUES (2, 'AAA', 'AAA'),(2, 'AAA', 'a'),(3, 'BBB', 'BBB'),(4, 'CCC', 'CCC'); +INSERT INTO t22 VALUES (2, 1, 'AAA', 'AAA'),(2, 3, 'AAA', 'a'),(3, 100, 'BBB', 'BBB'),(4, 101, 'CCC', 'CCC'); + +{% for join_use_nulls in [0, 1] -%} + +SET join_use_nulls = {{ join_use_nulls }}; + +SET join_algorithm = 'partial_merge'; + +SELECT '-- partial_merge --'; + +SELECT '--'; +SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2; +SELECT '--'; +SELECT t1.key, t1.key2 FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2; + +SELECT '--'; +SELECT t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2; + +SELECT '--'; +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id > 2; +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id == 3; +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.key2 == 'BBB'; +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333'; -- { serverError 48 } +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_lc as t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND isNull(t2.key2); +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND t1.key2 like '33%'; +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t1.key); + +-- DISTINCT is used to remove the difference between 'hash' and 'merge' join: 'merge' doesn't support `any_join_distinct_right_table_keys` + +SELECT '--'; +SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND t2.key2 != ''; +SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(t2.key2 != ''); +SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(t2.key2 != ''); +SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(toNullable(t2.key2 != '')); +SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(toLowCardinality(t2.key2 != '')); +SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(t1.key2 != ''); +SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(t1.key2 != ''); +SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toLowCardinality(toNullable(t1.key2 != '')); +SELECT DISTINCT t1.id FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toNullable(toLowCardinality(t1.key2 != '')); + +SELECT '--'; +SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND e; +-- `e + 1` is UInt16 +SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND e + 1; -- { serverError 403 } +SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND toUInt8(e + 1); + +SELECT '--'; +SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id NULLS FIRST, t2.id NULLS FIRST; + +SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError 403 } +SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 } +SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id + 2; -- { serverError 403 } +SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id + 2; -- { serverError 403 } +SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverError 403 } +SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 } +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } +SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 48 } +-- non-equi condition containing columns from different tables doesn't supported yet +SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } +SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } + +SELECT '--'; +-- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key +SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; + + +{% endfor -%} + +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS t2_nullable; +DROP TABLE IF EXISTS t2_lc; +DROP TABLE IF EXISTS t22; From 8e2637aab2b7d22b97d9100aaccfb14da5ed470f Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 2 Sep 2021 14:40:04 +0300 Subject: [PATCH 274/317] Store all related to one join disjunct in JoinOnClause, pt1 --- src/Interpreters/CollectJoinOnKeysVisitor.cpp | 7 +- src/Interpreters/CollectJoinOnKeysVisitor.h | 1 - .../LogicalExpressionsOptimizer.cpp | 1 - src/Interpreters/MergeJoin.cpp | 8 +- src/Interpreters/TableJoin.cpp | 141 ++++++++---------- src/Interpreters/TableJoin.h | 76 +++++++--- .../Optimizations/filterPushDown.cpp | 3 +- src/Storages/StorageJoin.cpp | 2 +- 8 files changed, 130 insertions(+), 109 deletions(-) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/src/Interpreters/CollectJoinOnKeysVisitor.cpp index e6e9c37f3fc..99f1fbc0082 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -41,11 +41,6 @@ void CollectJoinOnKeysMatcher::Data::setDisjuncts(const ASTPtr & or_func_ast) analyzed_join.setDisjuncts(std::move(v)); } -void CollectJoinOnKeysMatcher::Data::addDisjunct(const ASTPtr & ast) -{ - analyzed_join.addDisjunct(std::move(ast)); -} - void CollectJoinOnKeysMatcher::Data::addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos) { ASTPtr left = left_ast->clone(); @@ -107,7 +102,7 @@ void CollectJoinOnKeysMatcher::visit(const ASTFunction & func, const ASTPtr & as return; } - data.addDisjunct(ast); + data.analyzed_join.addDisjunct(ast); if (func.name == "and") return; /// go into children diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index 54b4ee39478..61e526b3b4d 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -52,7 +52,6 @@ public: void addAsofJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos, const ASOF::Inequality & asof_inequality); void setDisjuncts(const ASTPtr & or_func_ast); - void addDisjunct(const ASTPtr & ast); void asofToJoinKeys(); }; diff --git a/src/Interpreters/LogicalExpressionsOptimizer.cpp b/src/Interpreters/LogicalExpressionsOptimizer.cpp index ad3a1b8424a..936ed0149d2 100644 --- a/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -1,4 +1,3 @@ -#include #include #include diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 9a54f64af72..2aced4e72be 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -507,7 +507,9 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right ErrorCodes::PARAMETER_OUT_OF_BOUND); } - if (table_join->keyNamesLeft().size() > 1) + const auto & key_names_left_all = table_join->keyNamesLeft(); + const auto & key_names_right_all = table_join->keyNamesRight(); + if (key_names_left_all.size() != 1 || key_names_right_all.size() != 1) throw Exception("MergeJoin does not support OR", ErrorCodes::NOT_IMPLEMENTED); std::tie(mask_column_name_left, mask_column_name_right) = table_join->joinConditionColumnNames(0); @@ -522,8 +524,8 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right key_names_right.push_back(deriveTempName(mask_column_name_right)); } - key_names_left.insert(key_names_left.end(), table_join->keyNamesLeft().front().begin(), table_join->keyNamesLeft().front().end()); - key_names_right.insert(key_names_right.end(), table_join->keyNamesRight().front().begin(), table_join->keyNamesRight().front().end()); + key_names_left.insert(key_names_left.end(), key_names_left_all.front().begin(), key_names_left_all.front().end()); + key_names_right.insert(key_names_right.end(), key_names_right_all.front().begin(), key_names_right_all.front().end()); addConditionJoinColumn(right_sample_block, JoinTableSide::Right); JoinCommon::splitAdditionalColumns(NamesVector{key_names_right}, right_sample_block, right_table_keys, right_columns_to_add); diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 2564b00077d..edc03a794e2 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -62,22 +62,17 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) , partial_merge_join_left_table_buffer_bytes(settings.partial_merge_join_left_table_buffer_bytes) , max_files_to_merge(settings.join_on_disk_max_files_to_merge) , temporary_files_codec(settings.temporary_files_codec) - , key_names_left(1) - , key_names_right(1) - , on_filter_condition_asts_left(1) - , on_filter_condition_asts_right(1) + , left_clauses(1) + , right_clauses(1) , tmp_volume(tmp_volume_) { } void TableJoin::resetCollected() { - key_names_left.clear(); - key_names_right.clear(); - key_asts_left.clear(); - key_asts_right.clear(); - on_filter_condition_asts_left.clear(); - on_filter_condition_asts_right.clear(); + left_clauses = std::vector(1); + right_clauses = std::vector(1); + columns_from_joined_table.clear(); columns_added_by_join.clear(); original_names.clear(); @@ -92,33 +87,21 @@ void TableJoin::resetCollected() void TableJoin::addUsingKey(const ASTPtr & ast) { - key_names_left.front().push_back(ast->getColumnName()); - key_names_right.front().push_back(ast->getAliasOrColumnName()); - - key_asts_left.push_back(ast); - key_asts_right.push_back(ast); - - auto & right_key = key_names_right.front().back(); - if (renames.count(right_key)) - right_key = renames[right_key]; + left_clauses.back().addKey(ast->getColumnName(), ast); + right_clauses.back().addKey(renamedRightColumnName(ast->getAliasOrColumnName()), ast); } -/// create new disjunct when see a child of a previously discovered OR +/// create new disjunct when see a direct child of a previously discovered OR void TableJoin::addDisjunct(const ASTPtr & ast) { const IAST * addr = ast.get(); if (std::find_if(disjuncts.begin(), disjuncts.end(), [addr](const ASTPtr & ast_){return ast_.get() == addr;}) != disjuncts.end()) { - assert(key_names_left.size() == disjunct_num + 1); - - if (!key_names_left[disjunct_num].empty() || !on_filter_condition_asts_left[disjunct_num].empty() || !on_filter_condition_asts_right[disjunct_num].empty()) + if (!left_clauses.back().key_names.empty() || !left_clauses.back().on_filter_conditions.empty() || !right_clauses.back().on_filter_conditions.empty()) { - disjunct_num++; - key_names_left.resize(disjunct_num + 1); - key_names_right.resize(disjunct_num + 1); - on_filter_condition_asts_left.resize(disjunct_num + 1); - on_filter_condition_asts_right.resize(disjunct_num + 1); + left_clauses.emplace_back(); + right_clauses.emplace_back(); } } } @@ -131,11 +114,8 @@ void TableJoin::setDisjuncts(Disjuncts&& disjuncts_) void TableJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast) { - key_names_left[disjunct_num].push_back(left_table_ast->getColumnName()); - key_names_right[disjunct_num].push_back(right_table_ast->getAliasOrColumnName()); - - key_asts_left.push_back(left_table_ast); - key_asts_right.push_back(right_table_ast); + left_clauses.back().addKey(left_table_ast->getColumnName(), left_table_ast); + right_clauses.back().addKey(right_table_ast->getAliasOrColumnName(), right_table_ast); } /// @return how many times right key appears in ON section. @@ -145,9 +125,8 @@ size_t TableJoin::rightKeyInclusion(const String & name) const return 0; size_t count = 0; - for (const auto & key_names : key_names_right) - count += std::count(key_names.begin(), key_names.end(), name); - + for (const auto & clause : right_clauses) + count += std::count(clause.key_names.begin(), clause.key_names.end(), name); return count; } @@ -194,31 +173,39 @@ NamesWithAliases TableJoin::getNamesWithAliases(const NameSet & required_columns ASTPtr TableJoin::leftKeysList() const { ASTPtr keys_list = std::make_shared(); - keys_list->children = key_asts_left; - const size_t disjuncts_num = key_names_left.size(); - for (size_t d = 0; d < disjuncts_num; ++d) - if (ASTPtr extra_cond = joinConditionColumn(JoinTableSide::Left, d)) + for (size_t i = 0; i < left_clauses.size(); ++i) + { + const auto & clause = left_clauses[i]; + keys_list->children.insert(keys_list->children.end(), clause.key_asts.begin(), clause.key_asts.end()); + if (ASTPtr extra_cond = joinConditionColumn(JoinTableSide::Left, i)) keys_list->children.push_back(extra_cond); + } return keys_list; } ASTPtr TableJoin::rightKeysList() const { ASTPtr keys_list = std::make_shared(); - if (hasOn()) - keys_list->children = key_asts_right; - const size_t disjuncts_num = key_names_left.size(); - for (size_t d = 0; d < disjuncts_num; ++d) - if (ASTPtr extra_cond = joinConditionColumn(JoinTableSide::Right, d)) + for (size_t i = 0; i < right_clauses.size(); ++i) + { + if (hasOn()) + { + const auto & clause = right_clauses[i]; + keys_list->children.insert(keys_list->children.end(), clause.key_asts.begin(), clause.key_asts.end()); + } + + if (ASTPtr extra_cond = joinConditionColumn(JoinTableSide::Right, i)) keys_list->children.push_back(extra_cond); + } return keys_list; } Names TableJoin::requiredJoinedNames() const { NameSet required_columns_set; - for (const auto& key_names_right_part : key_names_right) - required_columns_set.insert(key_names_right_part.begin(), key_names_right_part.end()); + for (const auto & clause : right_clauses) + required_columns_set.insert(clause.key_names.begin(), clause.key_names.end()); + for (const auto & joined_column : columns_added_by_join) required_columns_set.insert(joined_column.name); @@ -228,9 +215,9 @@ Names TableJoin::requiredJoinedNames() const NameSet TableJoin::requiredRightKeys() const { NameSet required; - for (const auto & key_names_right_part : key_names_right) + for (const auto & clause : right_clauses) { - for (const auto & name : key_names_right_part) + for (const auto & name : clause.key_names) { auto rename = renamedRightColumnName(name); for (const auto & column : columns_added_by_join) @@ -369,7 +356,7 @@ bool TableJoin::allowMergeJoin() const bool all_join = is_all && (isInner(kind()) || isLeft(kind()) || isRight(kind()) || isFull(kind())); bool special_left = isLeft(kind()) && (is_any || is_semi); - bool no_ors = (key_names_right.size() == 1); + bool no_ors = (left_clauses.size() == 1); return (all_join || special_left) && no_ors; } @@ -407,7 +394,7 @@ bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context) if (!allowed_inner && !allowed_left) return false; - const Names & right_keys = keyNamesRight().front(); + const Names & right_keys = right_clauses.front().key_names; if (right_keys.size() != 1) return false; @@ -470,12 +457,14 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig for (const auto & col : right) right_types[renamedRightColumnName(col.name)] = col.type; - for (size_t d = 0; d < key_names_left.size(); ++d) + for (size_t d = 0; d < left_clauses.size(); ++d) { - for (size_t i = 0; i < key_names_left[d].size(); ++i) + auto & key_names_left = left_clauses[d].key_names; + auto & key_names_right = right_clauses[d].key_names; + for (size_t i = 0; i < key_names_left.size(); ++i) { - auto ltype = left_types.find(key_names_left[d][i]); - auto rtype = right_types.find(key_names_right[d][i]); + auto ltype = left_types.find(key_names_left[i]); + auto rtype = right_types.find(key_names_right[i]); if (ltype == left_types.end() || rtype == right_types.end()) { /// Name mismatch, give up @@ -495,13 +484,14 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig } catch (DB::Exception & ex) { - throw DB::Exception(ErrorCodes::TYPE_MISMATCH, - "Can't infer common type for joined columns: {}: {} at left, {}: {} at right. {}", - key_names_left[d][i], ltype->second->getName(), - key_names_right[d][i], rtype->second->getName(), - ex.message()); + throw Exception( + "Type mismatch of columns to JOIN by: " + + key_names_left[d][i] + ": " + ltype->second->getName() + " at left, " + + key_names_right[d][i] + ": " + rtype->second->getName() + " at right. " + + "Can't get supertype: " + ex.message(), + ErrorCodes::TYPE_MISMATCH); } - left_type_map[key_names_left[d][i]] = right_type_map[key_names_right[d][i]] = supertype; + left_type_map[key_names_left[i]] = right_type_map[key_names_right[i]] = supertype; } } @@ -518,7 +508,7 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig } ActionsDAGPtr TableJoin::applyKeyConvertToTable( - const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, NamesVector & names_vector_to_rename) const + const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, std::vector & join_clause) const { bool has_some_to_do = false; @@ -540,9 +530,9 @@ ActionsDAGPtr TableJoin::applyKeyConvertToTable( auto dag = ActionsDAG::makeConvertingActions( cols_src, cols_dst, ActionsDAG::MatchColumnsMode::Name, true, !hasUsing(), &key_column_rename); - for (auto & disjunct_names : names_vector_to_rename) + for (auto & clause : join_clause) { - for (auto & name : disjunct_names) + for (auto & name : clause.key_names) { const auto it = key_column_rename.find(name); if (it != key_column_rename.end()) @@ -577,9 +567,9 @@ String TableJoin::renamedRightColumnName(const String & name) const void TableJoin::addJoinCondition(const ASTPtr & ast, bool is_left) { if (is_left) - on_filter_condition_asts_left[disjunct_num].push_back(ast); + left_clauses.back().on_filter_conditions.push_back(ast); else - on_filter_condition_asts_right[disjunct_num].push_back(ast); + right_clauses.back().on_filter_conditions.push_back(ast); } void TableJoin::leftToRightKeyRemap( @@ -607,33 +597,34 @@ std::unordered_map TableJoin::leftToRightKeyRemap() const if (hasUsing()) { const auto & required_right_keys = requiredRightKeys(); - for (size_t i = 0; i < key_names_left.size(); ++i) - TableJoin::leftToRightKeyRemap(key_names_left[i], key_names_right[i], required_right_keys, left_to_right_key_remap); + for (size_t i = 0; i < left_clauses.size(); ++i) + TableJoin::leftToRightKeyRemap(left_clauses[i].key_names, right_clauses[i].key_names, required_right_keys, left_to_right_key_remap); } return left_to_right_key_remap; } /// Returns all conditions related to one table joined with 'and' function -static ASTPtr buildJoinConditionColumn(const ASTsVector & on_filter_condition_asts, size_t disjunct) +static ASTPtr buildJoinConditionColumn(const ASTs & on_filter_condition_asts) { - if (on_filter_condition_asts[disjunct].empty()) + if (on_filter_condition_asts.empty()) return nullptr; - if (on_filter_condition_asts[disjunct].size() == 1) - return on_filter_condition_asts[disjunct][0]; + + if (on_filter_condition_asts.size() == 1) + return on_filter_condition_asts[0]; auto function = std::make_shared(); function->name = "and"; function->arguments = std::make_shared(); function->children.push_back(function->arguments); - function->arguments->children = on_filter_condition_asts[disjunct]; + function->arguments->children = on_filter_condition_asts; return function; } ASTPtr TableJoin::joinConditionColumn(JoinTableSide side, size_t disjunct) const { if (side == JoinTableSide::Left) - return buildJoinConditionColumn(on_filter_condition_asts_left, disjunct); - return buildJoinConditionColumn(on_filter_condition_asts_right, disjunct); + return buildJoinConditionColumn(left_clauses[disjunct].on_filter_conditions); + return buildJoinConditionColumn(right_clauses[disjunct].on_filter_conditions); } std::pair TableJoin::joinConditionColumnNames(size_t disjunct) const diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 9624134b1eb..77bf6a2215f 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -74,16 +74,31 @@ private: const size_t max_files_to_merge = 0; const String temporary_files_codec = "LZ4"; - NamesVector key_names_left; - NamesVector key_names_right; /// Duplicating names are qualified. - ASTsVector on_filter_condition_asts_left; - ASTsVector on_filter_condition_asts_right; -private: - size_t disjunct_num = 0; + /// Corresponds to one disjunct + struct JoinOnClause + { + Names key_names; + ASTs key_asts; + + ASTs on_filter_conditions; + + JoinOnClause() = default; + + explicit JoinOnClause(const Names & names) + : key_names(names) + {} + + void addKey(const String & name, const ASTPtr & ast) + { + key_names.emplace_back(name); + key_asts.emplace_back(ast); + } + }; + Disjuncts disjuncts; - ASTs key_asts_left; - ASTs key_asts_right; + std::vector left_clauses; + std::vector right_clauses; /// Duplicating key_names are qualified. ASTTableJoin table_join; @@ -116,7 +131,7 @@ private: /// Create converting actions and change key column names if required ActionsDAGPtr applyKeyConvertToTable( - const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, NamesVector & names_vector_to_rename) const; + const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, std::vector & join_clause) const; /// Calculates common supertypes for corresponding join key columns. template @@ -131,10 +146,8 @@ private: public: TableJoin() - : key_names_left(1) - , key_names_right(1) - , on_filter_condition_asts_left(1) - , on_filter_condition_asts_right(1) + : left_clauses(1) + , right_clauses(1) { } @@ -142,16 +155,14 @@ public: /// for StorageJoin TableJoin(SizeLimits limits, bool use_nulls, ASTTableJoin::Kind kind, ASTTableJoin::Strictness strictness, - const NamesVector & key_names_right_) + const Names & key_names_right) : size_limits(limits) , default_max_bytes(0) , join_use_nulls(use_nulls) , join_algorithm(JoinAlgorithm::HASH) - , key_names_left(1) - , key_names_right(key_names_right_) - , on_filter_condition_asts_left(1) - , on_filter_condition_asts_right(1) + , left_clauses(1) { + right_clauses.emplace_back(key_names_right); table_join.kind = kind; table_join.strictness = strictness; } @@ -232,8 +243,26 @@ public: ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only - const NamesVector & keyNamesLeft() const { return key_names_left; } - const NamesVector & keyNamesRight() const { return key_names_right; } + NamesVector keyNamesLeft() const + { + NamesVector key_names; + for (const auto & clause : left_clauses) + { + key_names.push_back(clause.key_names); + } + return key_names; + } + + NamesVector keyNamesRight() const + { + NamesVector key_names; + for (const auto & clause : right_clauses) + { + key_names.push_back(clause.key_names); + } + return key_names; + } + const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; } Names columnsAddedByJoin() const @@ -245,7 +274,12 @@ public: } /// StorageJoin overrides key names (cause of different names qualification) - void setRightKeys(const Names & keys) { key_names_right.clear(); key_names_right.push_back(keys); } + void setRightKeys(const Names & keys) + { + // assert(right_clauses.size() <= 1); + right_clauses.clear(); + right_clauses.emplace_back(keys); + } Block getRequiredRightKeys(const Block & right_table_keys, std::vector & keys_sources) const; diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 10cfad82dd6..6f599ddd85c 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -202,7 +202,8 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes const auto & left_header = join->getInputStreams().front().header; const auto & res_header = join->getOutputStream().header; Names allowed_keys; - for (const auto & name : table_join.keyNamesLeft().front()) + const auto & key_names_left = table_join.keyNamesLeft(); + for (const auto & name : key_names_left.front()) { /// Skip key if it is renamed. /// I don't know if it is possible. Just in case. diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 39e81e6cf75..fdc026bb6c2 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -62,7 +62,7 @@ StorageJoin::StorageJoin( if (!metadata_snapshot->getColumns().hasPhysical(key)) throw Exception{"Key column (" + key + ") does not exist in table declaration.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE}; - table_join = std::make_shared(limits, use_nulls, kind, strictness, NamesVector{key_names}); + table_join = std::make_shared(limits, use_nulls, kind, strictness, key_names); join = std::make_shared(table_join, metadata_snapshot->getSampleBlock().sortColumns(), overwrite); restore(); } From 46187a73eefea033264b42560ed6268b71066bdc Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 6 Sep 2021 13:59:18 +0300 Subject: [PATCH 275/317] wip --- src/Core/Names.h | 1 - src/Interpreters/ExpressionActions.cpp | 13 +- src/Interpreters/HashJoin.cpp | 564 +++++++----------- src/Interpreters/HashJoin.h | 54 +- src/Interpreters/MergeJoin.cpp | 21 +- src/Interpreters/NullableUtils.cpp | 1 + src/Interpreters/TableJoin.cpp | 301 +++++----- src/Interpreters/TableJoin.h | 121 ++-- src/Interpreters/TreeRewriter.cpp | 4 +- src/Interpreters/join_common.cpp | 33 +- src/Interpreters/join_common.h | 8 +- .../Optimizations/filterPushDown.cpp | 6 +- 12 files changed, 504 insertions(+), 623 deletions(-) diff --git a/src/Core/Names.h b/src/Core/Names.h index 869530b1a16..3281daa560e 100644 --- a/src/Core/Names.h +++ b/src/Core/Names.h @@ -11,7 +11,6 @@ namespace DB { using Names = std::vector; -using NamesVector = std::vector; using NameSet = std::unordered_set; using NameOrderedSet = std::set; using NameToNameMap = std::unordered_map; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index a56f7c66bf1..73c191d39ab 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -1056,15 +1056,12 @@ void ExpressionActionsChain::JoinStep::finalize(const NameSet & required_output_ /// That's an input columns we need. NameSet required_names = required_output_; - if (!analyzed_join->keyNamesLeft().empty()) // why do we need this condition ? - for (const auto & name_part : analyzed_join->keyNamesLeft()) - for (const auto & name : name_part) - required_names.emplace(name); + for (const auto & name : analyzed_join->getAllNames(JoinTableSide::Left)) + required_names.emplace(name); - const size_t disjuncts = analyzed_join->keyNamesLeft().size(); - for (size_t d = 0; d < disjuncts; ++d) - if (ASTPtr extra_condition_column = analyzed_join->joinConditionColumn(JoinTableSide::Left, d)) - required_names.emplace(extra_condition_column->getColumnName()); + for (const auto & onexpr : analyzed_join->getClauses()) + if (const auto & cond_name = onexpr.condColumnNames().first; !cond_name.empty()) + required_names.emplace(cond_name); for (const auto & column : required_columns) { diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index f6e95825f38..f39abf11c99 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1,6 +1,9 @@ #include +#include #include #include +#include +#include #include @@ -25,8 +28,12 @@ #include #include +#include +#include #include #include +#include "Columns/IColumn.h" +#include "Interpreters/IJoin.h" namespace DB { @@ -228,11 +235,16 @@ static ColumnWithTypeAndName correctNullability(ColumnWithTypeAndName && column, return std::move(column); } -static std::string formatKeysDebug(const NamesVector & key_names) +static std::string formatKeysDebug(const std::vector & caluses) { std::vector res; - for (const auto & keys : key_names) - res.emplace_back(fmt::format("{}", fmt::join(keys, ", "))); + for (const auto & clause : caluses) + { + std::vector current; + for (size_t i = 0; i < clause.keysCount(); ++i) + current.emplace_back(fmt::format("{} == {}", clause.key_names_left[i], clause.key_names_right[i])); + res.emplace_back(fmt::format("{}", fmt::join(current, ", "))); + } return fmt::format("{}", fmt::join(res, " | ")); } @@ -240,8 +252,6 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s : table_join(table_join_) , kind(table_join->kind()) , strictness(table_join->strictness()) - , key_names_right(table_join->keyNamesRight()) - , key_names_left(table_join->keyNamesLeft()) , nullable_right_side(table_join->forceNullableRight()) , nullable_left_side(table_join->forceNullableLeft()) , any_take_last_row(any_take_last_row_) @@ -251,10 +261,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s , log(&Poco::Logger::get("HashJoin")) { LOG_DEBUG(log, "Right sample block: {}", right_sample_block.dumpStructure()); - const size_t disjuncts_num = key_names_right.size(); - const bool multiple_disjuncts = disjuncts_num > 1; - - if (multiple_disjuncts) + if (!table_join->oneDisjunct()) { /// required right keys concept does not work well if multiple disjuncts, /// we need all keys @@ -262,55 +269,50 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s } else { + const auto & key_names_right = table_join->getOnlyClause().key_names_right; JoinCommon::splitAdditionalColumns(key_names_right, right_sample_block, right_table_keys, sample_block_with_columns_to_add); required_right_keys = table_join->getRequiredRightKeys(right_table_keys, required_right_keys_sources); } - LOG_DEBUG(log, "Right keys: [{}] (required: [{}]), left keys: [{}]", - formatKeysDebug(key_names_right), - fmt::join(required_right_keys.getNames(), ", "), - formatKeysDebug(key_names_left)); + LOG_DEBUG(log, "Join keys: [{}], required right: [{}])", formatKeysDebug(table_join->getClauses()), fmt::join(required_right_keys.getNames(), ", ")); LOG_DEBUG(log, "Columns to add: [{}]", sample_block_with_columns_to_add.dumpStructure()); JoinCommon::removeLowCardinalityInplace(right_table_keys); - key_sizes.resize(key_names_right.size()); - - Type join_method = Type::EMPTY; initRightBlockStructure(data->sample_block); JoinCommon::createMissedColumns(sample_block_with_columns_to_add); - if (table_join->getDictionaryReader()) - { - data->maps.resize(disjuncts_num); - } - condition_mask_column_name_left.resize(disjuncts_num); - condition_mask_column_name_right.resize(disjuncts_num); - if (nullable_right_side) - { JoinCommon::convertColumnsToNullable(sample_block_with_columns_to_add); - } - for (size_t d = 0; d < disjuncts_num; ++d) + + size_t disjuncts_num = table_join->getClauses().size(); + data->maps.reserve(disjuncts_num); + key_sizes.reserve(disjuncts_num); + + for (const auto & clause : table_join->getClauses()) { - std::tie(condition_mask_column_name_left[d], condition_mask_column_name_right[d]) = table_join->joinConditionColumnNames(d); - ColumnRawPtrs key_columns = JoinCommon::extractKeysForJoin(right_table_keys, key_names_right[d]); + const auto & key_names_right = clause.key_names_right; + ColumnRawPtrs key_columns = JoinCommon::extractKeysForJoin(right_table_keys, key_names_right); if (table_join->dictionary_reader) { + assert(disjuncts_num == 1); LOG_DEBUG(log, "Performing join over dict"); - join_method = Type::DICT; + data->type = Type::DICT; - std::get(data->maps[d]).create(Type::DICT); - chooseMethod(key_columns, key_sizes[d]); /// init key_sizes - continue; // break ? + data->maps.resize(disjuncts_num); + std::get(data->maps[0]).create(Type::DICT); + key_sizes.resize(1); + chooseMethod(key_columns, key_sizes[0]); /// init key_sizes } else if (strictness == ASTTableJoin::Strictness::Asof) { + assert(disjuncts_num == 1); + /// @note ASOF JOIN is not INNER. It's better avoid use of 'INNER ASOF' combination in messages. /// In fact INNER means 'LEFT SEMI ASOF' while LEFT means 'LEFT OUTER ASOF'. if (!isLeft(kind) && !isInner(kind)) @@ -319,7 +321,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s if (key_columns.size() <= 1) throw Exception("ASOF join needs at least one equi-join column", ErrorCodes::SYNTAX_ERROR); - if (right_table_keys.getByName(key_names_right[0].back()).type->isNullable()) + if (right_table_keys.getByName(key_names_right.back()).type->isNullable()) throw Exception("ASOF join over right table Nullable column is not implemented", ErrorCodes::NOT_IMPLEMENTED); size_t asof_size; @@ -330,34 +332,22 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s /// However, this does not depend on the size of the asof join key (as that goes into the BST) /// Therefore, add it back in such that it can be extracted appropriately from the full stored /// key_columns and key_sizes - key_sizes[d].push_back(asof_size); + key_sizes.resize(1); + key_sizes[0].push_back(asof_size); } else { /// Choose data structure to use for JOIN. - } - - auto current_join_method = chooseMethod(key_columns, key_sizes[d]); - if (join_method == Type::EMPTY) - { - join_method = current_join_method; - } - else if (join_method != current_join_method) - { - join_method = Type::hashed; + auto current_join_method = chooseMethod(key_columns, key_sizes.emplace_back()); + if (data->type == Type::EMPTY) + data->type = current_join_method; + else if (data->type != current_join_method) + data->type = Type::hashed; } } - data->type = join_method; - if (join_method != Type::DICT) - { - data->maps.resize(key_names_right.size()); - - for (size_t d = 0; d < disjuncts_num; ++d) - { - data_map_init(data->maps[d]); - } - } + for (auto & maps : data->maps) + dataMapInit(maps); } HashJoin::Type HashJoin::chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes) @@ -453,7 +443,7 @@ public: JoinCommon::convertColumnToNullable(column); } - FindResult findKey(const TableJoin & /* void * */, size_t row, const Arena &) + FindResult findKey(const TableJoin &, size_t row, const Arena &) { result.block = &read_result; result.row_num = positions[row]; @@ -518,8 +508,10 @@ struct KeyGetterForType using Type = typename KeyGetterForTypeImpl::Type; }; -void HashJoin::data_map_init(MapsVariant & map) +void HashJoin::dataMapInit(MapsVariant & map) { + if (data->type == Type::DICT) + return; if (kind == ASTTableJoin::Kind::Cross) return; joinDispatchInit(kind, strictness, map); @@ -650,13 +642,9 @@ namespace if constexpr (is_asof_join) Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); else if constexpr (mapped_one) - { Inserter::insertOne(join, map, key_getter, stored_block, i, pool); - } else - { Inserter::insertAll(join, map, key_getter, stored_block, i, pool); - } } return map.getBufferSizeInCells(); } @@ -701,7 +689,7 @@ namespace void HashJoin::initRightBlockStructure(Block & saved_block_sample) { - bool multiple_disjuncts = key_names_right.size() > 1; + bool multiple_disjuncts = !table_join->oneDisjunct(); /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). bool save_key_columns = !table_join->forceHashJoin() || isRightOrFull(kind) || multiple_disjuncts; if (save_key_columns) @@ -760,47 +748,17 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) Block block = materializeBlock(source_block); size_t rows = block.rows(); - size_t total_rows = 0; - size_t total_bytes = 0; - - // Collect all keys in all_key_names_right - // and lists of indexes in this vector for all disjuncts - Names all_key_names_right = key_names_right.front(); - const size_t disjuncts_num = key_names_right.size(); - std::vector> key_names_right_indexes(disjuncts_num); - key_names_right_indexes[0].resize(all_key_names_right.size()); - std::iota(std::begin(key_names_right_indexes[0]), std::end(key_names_right_indexes[0]), 0); - - for (size_t d = 1; d < disjuncts_num; ++d) + ColumnRawPtrMap all_key_columns; { - for (size_t i = 0; i < key_names_right[d].size(); ++i) - { - auto it = std::find(std::cbegin(all_key_names_right), std::cend(all_key_names_right), key_names_right[d][i]); - if (it == std::cend(all_key_names_right)) - { - key_names_right_indexes[d].push_back(all_key_names_right.size()); - all_key_names_right.push_back(key_names_right[d][i]); - } - else - { - key_names_right_indexes[d].push_back(std::distance(std::cbegin(all_key_names_right), it)); - } - } + Names all_key_names_right; + for (const auto & clause : table_join->getClauses()) + all_key_names_right.insert(all_key_names_right.end(), clause.key_names_right.begin(), clause.key_names_right.end()); + all_key_columns = JoinCommon::materializeColumnsInplaceMap(block, all_key_names_right); } - ColumnRawPtrs all_key_columns = JoinCommon::materializeColumnsInplace(block, all_key_names_right); - Block structured_block = structureRightBlock(block); - bool multiple_disjuncts = disjuncts_num > 1; - - std::vector join_mask_col_vector(disjuncts_num); - for (size_t d = 0; d < disjuncts_num; ++d) - join_mask_col_vector[d] = JoinCommon::getColumnAsMask(block, condition_mask_column_name_right[d]); - - - std::vector null_map_vector; - Columns null_map_holder_vector; - + size_t total_rows = 0; + size_t total_bytes = 0; { if (storage_join_lock.mutex()) throw DB::Exception("addJoinedBlock called when HashJoin locked to prevent updates", @@ -812,66 +770,33 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) if (rows) data->empty = false; - bool save_a_nullmap = false; - - for (size_t d = 0; d < disjuncts_num; ++d) + bool multiple_disjuncts = !table_join->oneDisjunct(); + const auto & onexprs = table_join->getClauses(); + for (size_t onexpr_idx = 0; onexpr_idx < onexprs.size(); ++onexpr_idx) { - ColumnRawPtrs key_columns(key_names_right_indexes[d].size()); - std::transform(std::cbegin(key_names_right_indexes[d]), - std::cend(key_names_right_indexes[d]), - std::begin(key_columns), - [&all_key_columns](size_t i){ return all_key_columns[i]; }); + ColumnRawPtrs key_columns; + for (const auto & name : onexprs[onexpr_idx].key_names_right) + key_columns.push_back(all_key_columns[name]); /// We will insert to the map only keys, where all components are not NULL. + ConstNullMapPtr null_map{}; + ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); - null_map_vector.emplace_back(); - null_map_holder_vector.push_back(extractNestedColumnsAndNullMap(key_columns, null_map_vector.back())); - - - /// If RIGHT or FULL save blocks with nulls for NonJoinedBlockInputStream + /// If RIGHT or FULL save blocks with nulls for NotJoinedBlocks UInt8 save_nullmap = 0; - if (isRightOrFull(kind) && null_map_vector.back()) + if (isRightOrFull(kind) && null_map) { - for (size_t i = 0; !save_nullmap && i < null_map_vector.back()->size(); ++i) - save_nullmap |= (*null_map_vector.back())[i]; + /// Save rows with NULL keys + for (size_t i = 0; !save_nullmap && i < null_map->size(); ++i) + save_nullmap |= (*null_map)[i]; } - save_a_nullmap |= save_nullmap; - { - if (kind != ASTTableJoin::Kind::Cross) - { - joinDispatch(kind, strictness, data->maps[d], [&](auto kind_, auto strictness_, auto & map) - { - size_t size = insertFromBlockImpl( - *this, data->type, map, rows, key_columns, key_sizes[d], stored_block, null_map_vector.back(), - join_mask_col_vector[d] ? &assert_cast(*join_mask_col_vector[d]).getData() : nullptr, - data->pool); - - if (multiple_disjuncts) - used_flags.reinit(stored_block); - else - /// Number of buckets + 1 value from zero storage - used_flags.reinit(size + 1); - }); - } - - if (!check_limits) - return true; - - /// TODO: Do not calculate them every time - total_rows = getTotalRowCount(); - total_bytes = getTotalByteCount(); - } - } - - - if (!multiple_disjuncts) - { + auto join_mask_col = JoinCommon::getColumnAsMask(block, onexprs[onexpr_idx].condColumnNames().second); /// Save blocks that do not hold conditions in ON section ColumnUInt8::MutablePtr not_joined_map = nullptr; - if (isRightOrFull(kind) && join_mask_col_vector[0]) + if (!multiple_disjuncts && isRightOrFull(kind) && join_mask_col) { - const auto & join_mask = assert_cast(*join_mask_col_vector[0]).getData(); + const auto & join_mask = assert_cast(*join_mask_col).getData(); /// Save rows that do not hold conditions not_joined_map = ColumnUInt8::create(block.rows(), 0); for (size_t i = 0, sz = join_mask.size(); i < sz; ++i) @@ -881,25 +806,87 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) continue; /// NULL key will be saved anyway because, do not save twice - if (save_a_nullmap && (*null_map_vector[0])[i]) + if (save_nullmap && (*null_map)[i]) continue; not_joined_map->getData()[i] = 1; } } - if (save_a_nullmap) - data->blocks_nullmaps.emplace_back(stored_block, null_map_holder_vector[0]); - if (not_joined_map) + if (kind != ASTTableJoin::Kind::Cross) + { + joinDispatch(kind, strictness, data->maps[onexpr_idx], [&](auto kind_, auto strictness_, auto & map) + { + size_t size = insertFromBlockImpl( + *this, data->type, map, rows, key_columns, key_sizes[onexpr_idx], stored_block, null_map, + join_mask_col ? &assert_cast(*join_mask_col).getData() : nullptr, + data->pool); + + if (multiple_disjuncts) + used_flags.reinit(stored_block); + else + /// Number of buckets + 1 value from zero storage + used_flags.reinit(size + 1); + }); + } + + if (!multiple_disjuncts && save_nullmap) + data->blocks_nullmaps.emplace_back(stored_block, null_map_holder); + + if (!multiple_disjuncts && not_joined_map) data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map)); + + if (!check_limits) + return true; + + /// TODO: Do not calculate them every time + total_rows = getTotalRowCount(); + total_bytes = getTotalByteCount(); } } return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); } -using ColumnRawPtrsVector = std::vector; -using SizesVector = std::vector; + +namespace +{ + +struct JoinOnKeyColumns +{ + const Names & key_names; + + Columns materialized_keys_holder; + ColumnRawPtrs key_columns; + + ColumnPtr null_map_holder; + ConstNullMapPtr null_map; + + /// Only rows where mask == true can be joined + ColumnPtr join_mask_column; + + Sizes key_sizes; + + + explicit JoinOnKeyColumns(const Block & block, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_) + : key_names(key_names_) + , materialized_keys_holder(JoinCommon::materializeColumns(block, key_names)) /// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them. + , key_columns(JoinCommon::getRawPointers(materialized_keys_holder)) + , null_map_holder(extractNestedColumnsAndNullMap(key_columns, null_map)) + , join_mask_column(JoinCommon::getColumnAsMask(block, cond_column_name)) + , key_sizes(key_sizes_) + {} + + bool isRowFiltered(size_t i) const + { + if (join_mask_column) + { + UInt8ColumnDataPtr mask = &assert_cast(*(join_mask_column)).getData(); + return !(*mask)[i]; + } + return false; + } +}; class AddedColumns { @@ -921,17 +908,13 @@ public: const Block & block, const Block & saved_block_sample, const HashJoin & join, - const ColumnRawPtrsVector & key_columns_, - const SizesVector & key_sizes_, - const std::vector & join_mask_column_, + std::vector && join_on_keys_, bool is_asof_join, bool is_join_get_) - : key_columns(key_columns_) - , key_sizes(key_sizes_) + : join_on_keys(join_on_keys_) , rows_to_add(block.rows()) , asof_type(join.getAsofType()) , asof_inequality(join.getAsofInequality()) - , join_mask_column(join_mask_column_) , is_join_get(is_join_get_) { size_t num_columns_to_add = block_with_columns_to_add.columns(); @@ -954,9 +937,10 @@ public: if (is_asof_join) { - const ColumnWithTypeAndName & right_asof_column = join.rightAsofKeyColumn(); + assert(join_on_keys.size() == 1); + const auto & right_asof_column = join.savedBlockSample().getByName(join_on_keys[0].key_names.front()); addColumn(right_asof_column, right_asof_column.name); - left_asof_key = key_columns.front().back(); + left_asof_key = join_on_keys[0].key_columns.front(); } for (auto & tn : type_name) @@ -1016,24 +1000,10 @@ public: ASOF::Inequality asofInequality() const { return asof_inequality; } const IColumn & leftAsofKey() const { return *left_asof_key; } - bool isRowFiltered(size_t i, size_t d) - { - if (join_mask_column[d]) - { - - UInt8ColumnDataPtr jmc = &assert_cast(*(join_mask_column[d])).getData(); - return !(*jmc)[i]; - } - - return false; - } - - const ColumnRawPtrsVector key_columns; - const SizesVector key_sizes; + std::vector join_on_keys; size_t rows_to_add; std::unique_ptr offsets_to_replicate; bool need_filter = false; - IColumn::Filter row_filter; private: std::vector type_name; @@ -1044,7 +1014,6 @@ private: std::optional asof_type; ASOF::Inequality asof_inequality; const IColumn * left_asof_key = nullptr; - std::vector join_mask_column; bool is_join_get; void addColumn(const ColumnWithTypeAndName & src_column, const std::string & qualified_name) @@ -1054,10 +1023,7 @@ private: type_name.emplace_back(src_column.type, src_column.name, qualified_name); } }; -using AddedColumnsV = std::vector>; -namespace -{ template struct JoinFeatures { @@ -1230,7 +1196,6 @@ NO_INLINE IColumn::Filter joinRightColumns( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, - const std::vector & null_map [[maybe_unused]], JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]]) { constexpr JoinFeatures jf; @@ -1245,8 +1210,6 @@ NO_INLINE IColumn::Filter joinRightColumns( if constexpr (jf.need_replication) added_columns.offsets_to_replicate = std::make_unique(rows); - size_t disjunct_num = added_columns.key_columns.size(); - IColumn::Offset current_offset = 0; for (size_t i = 0; i < rows; ++i) @@ -1255,21 +1218,21 @@ NO_INLINE IColumn::Filter joinRightColumns( bool null_element_found = false; KnownRowsHolder known_rows; - size_t d = 0; - do + for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) { + const auto & join_keys = added_columns.join_on_keys[onexpr_idx]; if constexpr (has_null_map) { - if (null_map[d] && (*null_map[d])[i]) + if (join_keys.null_map && (*join_keys.null_map)[i]) { null_element_found = true; continue; } } - bool row_acceptable = !added_columns.isRowFiltered(i, d); + bool row_acceptable = !join_keys.isRowFiltered(i); using FindResult = typename KeyGetter::FindResult; - auto find_result = row_acceptable ? key_getter_vector[d].findKey(*(mapv[d]), i, pool) : FindResult(); + auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult(); if (find_result.isFound()) { @@ -1346,7 +1309,7 @@ NO_INLINE IColumn::Filter joinRightColumns( } } } - } while (multiple_disjuncts && ++d < disjunct_num); + } if constexpr (has_null_map) { @@ -1356,7 +1319,7 @@ NO_INLINE IColumn::Filter joinRightColumns( if constexpr (jf.need_replication) { - (*added_columns.offsets_to_replicate)[i] = current_offset; + (*added_columns.offsets_to_replicate)[i] = current_offset; } continue; } @@ -1371,7 +1334,7 @@ NO_INLINE IColumn::Filter joinRightColumns( if constexpr (jf.need_replication) { - (*added_columns.offsets_to_replicate)[i] = current_offset; + (*added_columns.offsets_to_replicate)[i] = current_offset; } } @@ -1384,44 +1347,43 @@ IColumn::Filter joinRightColumnsSwitchMultipleDisjuncts( std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, - const std::vector & null_map [[maybe_unused]], JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]]) { return mapv.size() > 1 - ? joinRightColumns(std::forward>(key_getter_vector), mapv, added_columns, null_map, used_flags) - : joinRightColumns(std::forward>(key_getter_vector), mapv, added_columns, null_map, used_flags); + ? joinRightColumns(std::forward>(key_getter_vector), mapv, added_columns, used_flags) + : joinRightColumns(std::forward>(key_getter_vector), mapv, added_columns, used_flags); } template IColumn::Filter joinRightColumnsSwitchNullability( std::vector && key_getter_vector, - const std::vector/***/ & mapv, + const std::vector & mapv, AddedColumns & added_columns, - const std::vector & null_map, JoinStuff::JoinUsedFlags & used_flags) { + bool has_null_map = std::any_of(added_columns.join_on_keys.begin(), added_columns.join_on_keys.end(), + [](const auto & k) { return k.null_map; }); if (added_columns.need_filter) { - if (!null_map.empty()) - return joinRightColumnsSwitchMultipleDisjuncts(std::forward>(key_getter_vector), mapv, added_columns, null_map, used_flags); + if (has_null_map) + return joinRightColumnsSwitchMultipleDisjuncts(std::forward>(key_getter_vector), mapv, added_columns, used_flags); else - return joinRightColumnsSwitchMultipleDisjuncts(std::forward>(key_getter_vector), mapv, added_columns, null_map, used_flags); + return joinRightColumnsSwitchMultipleDisjuncts(std::forward>(key_getter_vector), mapv, added_columns, used_flags); } else { - if (!null_map.empty()) - return joinRightColumnsSwitchMultipleDisjuncts(std::forward>(key_getter_vector), mapv, added_columns, null_map, used_flags); + if (has_null_map) + return joinRightColumnsSwitchMultipleDisjuncts(std::forward>(key_getter_vector), mapv, added_columns, used_flags); else - return joinRightColumnsSwitchMultipleDisjuncts(std::forward>(key_getter_vector), mapv, added_columns, null_map, used_flags); + return joinRightColumnsSwitchMultipleDisjuncts(std::forward>(key_getter_vector), mapv, added_columns, used_flags); } } template IColumn::Filter switchJoinRightColumns( - const std::vector & mapv, + const std::vector & mapv, AddedColumns & added_columns, HashJoin::Type type, - const std::vector & null_map, JoinStuff::JoinUsedFlags & used_flags) { constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof; @@ -1430,18 +1392,18 @@ IColumn::Filter switchJoinRightColumns( #define M(TYPE) \ case HashJoin::Type::TYPE: \ { \ - using AMapTypeVal = const typename std::remove_reference_t::element_type; \ - using KeyGetter = typename KeyGetterForType::Type; \ - std::vector a_map_type_vector(mapv.size()); \ + using MapTypeVal = const typename std::remove_reference_t::element_type; \ + using KeyGetter = typename KeyGetterForType::Type; \ + std::vector a_map_type_vector(mapv.size()); \ std::vector key_getter_vector; \ - size_t disjunct_num = added_columns.key_columns.size(); \ - for (size_t d = 0; d < disjunct_num; ++d) \ - { \ + for (size_t d = 0; d < added_columns.join_on_keys.size(); ++d) \ + { \ + const auto & join_on_key = added_columns.join_on_keys[d]; \ a_map_type_vector[d] = mapv[d]->TYPE.get(); \ - key_getter_vector.push_back(std::move(createKeyGetter(added_columns.key_columns[d], added_columns.key_sizes[d]))); \ + key_getter_vector.push_back(std::move(createKeyGetter(join_on_key.key_columns, join_on_key.key_sizes))); \ } \ return joinRightColumnsSwitchNullability( \ - std::move(key_getter_vector), a_map_type_vector, added_columns, null_map, used_flags); \ + std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); \ } APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -1452,30 +1414,22 @@ IColumn::Filter switchJoinRightColumns( } template -IColumn::Filter dictionaryJoinRightColumns(const TableJoin & table_join, AddedColumns & added_columns, const ConstNullMapPtr & null_map) +IColumn::Filter dictionaryJoinRightColumns(const TableJoin & table_join, AddedColumns & added_columns) { if constexpr (KIND == ASTTableJoin::Kind::Left && (STRICTNESS == ASTTableJoin::Strictness::Any || STRICTNESS == ASTTableJoin::Strictness::Semi || STRICTNESS == ASTTableJoin::Strictness::Anti)) { - assert(added_columns.key_columns.size() == 1); + assert(added_columns.join_on_keys.size() == 1); - // JoinStuff::JoinUsedFlags flags; - // KeyGetterForDict key_getter(table_join, added_columns.key_columns); - // return joinRightColumnsSwitchNullability( - // std::move(key_getter), nullptr, added_columns, null_map, flags); - std::vector maps_vector; + std::vector maps_vector; maps_vector.push_back(&table_join); - std::vector null_maps_vector; - null_maps_vector.push_back(null_map); - JoinStuff::JoinUsedFlags flags; std::vector key_getter_vector; - key_getter_vector.push_back(KeyGetterForDict(table_join, added_columns.key_columns[0])); - // KeyGetterForDict key_getter(table_join, added_columns.key_columns); - return joinRightColumnsSwitchNullability(std::move(key_getter_vector), maps_vector, added_columns, null_maps_vector, flags); + key_getter_vector.push_back(KeyGetterForDict(table_join, added_columns.join_on_keys[0].key_columns)); + return joinRightColumnsSwitchNullability(std::move(key_getter_vector), maps_vector, added_columns, flags); } throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", STRICTNESS, KIND); @@ -1484,36 +1438,22 @@ IColumn::Filter dictionaryJoinRightColumns(const TableJoin & table_join, AddedCo } /// nameless template -std::unique_ptr HashJoin::makeAddedColumns( +void HashJoin::joinBlockImpl( Block & block, - const NamesVector & key_names_left_vector, const Block & block_with_columns_to_add, - const std::vector & maps_, + const std::vector & maps_, bool is_join_get) const { constexpr JoinFeatures jf; - ColumnRawPtrsVector left_key_columns_vector; - std::vector null_map_vector; - std::vector null_map_holder_vector; - std::vector materialized_keys_vector; - std::vector join_mask_column_vector; /// Only rows where mask == true can be joined - - size_t disjunct = 0; - - for (const auto & key_names_left_part : key_names_left_vector) + std::vector join_on_keys; + const auto & onexprs = table_join->getClauses(); + for (size_t i = 0; i < onexprs.size(); ++i) { - /// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them. - materialized_keys_vector.emplace_back(JoinCommon::materializeColumns(block, key_names_left_part)); - ColumnRawPtrs left_key_columns = JoinCommon::getRawPointers(materialized_keys_vector.back()); - left_key_columns_vector.push_back(std::move(left_key_columns)); - - /// Keys with NULL value in any column won't join to anything. - null_map_vector.emplace_back(); - null_map_holder_vector.push_back(extractNestedColumnsAndNullMap(left_key_columns_vector.back(), null_map_vector.back())); - - join_mask_column_vector.push_back(JoinCommon::getColumnAsMask(block, condition_mask_column_name_left[disjunct++])); + const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right; + join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, key_sizes[i]); } + size_t existing_columns = block.columns(); /** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized. * Because if they are constants, then in the "not joined" rows, they may have different values @@ -1532,47 +1472,34 @@ std::unique_ptr HashJoin::makeAddedColumns( * but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped. * For ASOF, the last column is used as the ASOF column */ - auto added_columns = std::make_unique( + AddedColumns added_columns( block_with_columns_to_add, - block, savedBlockSample(), + block, + savedBlockSample(), *this, - left_key_columns_vector, - key_sizes, - join_mask_column_vector, + std::move(join_on_keys), jf.is_asof_join, is_join_get); bool has_required_right_keys = (required_right_keys.columns() != 0); - added_columns->need_filter = jf.need_filter || has_required_right_keys; + added_columns.need_filter = jf.need_filter || has_required_right_keys; - added_columns->row_filter = overDictionary() ? - dictionaryJoinRightColumns(*table_join, *added_columns, null_map_vector[0]): - switchJoinRightColumns(maps_, *added_columns, data->type, null_map_vector, used_flags); + IColumn::Filter row_filter = overDictionary() ? + dictionaryJoinRightColumns(*table_join, added_columns) : + switchJoinRightColumns(maps_, added_columns, data->type, used_flags); - for (size_t i = 0; i < added_columns->size(); ++i) - block.insert(added_columns->moveColumn(i)); + for (size_t i = 0; i < added_columns.size(); ++i) + block.insert(added_columns.moveColumn(i)); - return added_columns; -} - -template -void HashJoin::joinBlockImpl( - Block & block, - std::unique_ptr added_columns, - size_t existing_columns) const -{ - JoinFeatures jf; - bool has_required_right_keys = (required_right_keys.columns() != 0); std::vector right_keys_to_replicate [[maybe_unused]]; if constexpr (jf.need_filter) { /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. for (size_t i = 0; i < existing_columns; ++i) - block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(added_columns->row_filter, -1); + block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(row_filter, -1); - /// Add join key columns from right block if needed - /// using value from left table because of equality + /// Add join key columns from right block if needed using value from left table because of equality for (size_t i = 0; i < required_right_keys.columns(); ++i) { const auto & right_key = required_right_keys.getByPosition(i); @@ -1582,7 +1509,7 @@ void HashJoin::joinBlockImpl( const auto & left_name = required_right_keys_sources[i]; /// asof column is already in block. - if (jf.is_asof_join && right_key.name == key_names_right[0].back()) + if (jf.is_asof_join && right_key.name == table_join->getOnlyClause().key_names_right.back()) continue; const auto & col = block.getByName(left_name); @@ -1601,8 +1528,9 @@ void HashJoin::joinBlockImpl( /// Some trash to represent IColumn::Filter as ColumnUInt8 needed for ColumnNullable::applyNullMap() auto null_map_filter_ptr = ColumnUInt8::create(); ColumnUInt8 & null_map_filter = assert_cast(*null_map_filter_ptr); - null_map_filter.getData().swap(added_columns->row_filter); + null_map_filter.getData().swap(row_filter); const IColumn::Filter & filter = null_map_filter.getData(); + /// Add join key columns from right block if needed. for (size_t i = 0; i < required_right_keys.columns(); ++i) { @@ -1611,8 +1539,9 @@ void HashJoin::joinBlockImpl( if (!block.findByName(right_col_name /*right_key.name*/)) { const auto & left_name = required_right_keys_sources[i]; + /// asof column is already in block. - if (jf.is_asof_join && right_key.name == key_names_right[0].back()) + if (jf.is_asof_join && right_key.name == table_join->getOnlyClause().key_names_right.back()) continue; const auto & col = block.getByName(left_name); @@ -1634,7 +1563,7 @@ void HashJoin::joinBlockImpl( if constexpr (jf.need_replication) { - std::unique_ptr & offsets_to_replicate = added_columns->offsets_to_replicate; + std::unique_ptr & offsets_to_replicate = added_columns.offsets_to_replicate; /// If ALL ... JOIN - we replicate all the columns except the new ones. for (size_t i = 0; i < existing_columns; ++i) @@ -1642,10 +1571,7 @@ void HashJoin::joinBlockImpl( /// Replicate additional right keys for (size_t pos : right_keys_to_replicate) - { block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate); - } - } } @@ -1654,7 +1580,6 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) size_t max_joined_block_rows = table_join->maxJoinedBlockRows(); size_t start_left_row = 0; size_t start_right_block = 0; - if (not_processed) { auto & continuation = static_cast(*not_processed); @@ -1767,31 +1692,24 @@ ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block && kind == ASTTableJoin::Kind::Left; if (!is_valid) throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN); + const auto & key_names_right = table_join->getOnlyClause().key_names_right; /// Assemble the key block with correct names. Block keys; for (size_t i = 0; i < block.columns(); ++i) { auto key = block.getByPosition(i); - key.name = key_names_right.front()[i]; + key.name = key_names_right[i]; keys.insert(std::move(key)); } static_assert(!MapGetter::flagged, "joinGet are not protected from hash table changes between block processing"); - - size_t existing_columns = block.columns(); - - std::vector maps_vector; + std::vector maps_vector; maps_vector.push_back(&std::get(data->maps[0])); - - auto added_columns = makeAddedColumns( - keys, key_names_right, block_with_columns_to_add, maps_vector, /* is_join_get */ true); - - joinBlockImpl( - keys, std::move(added_columns), existing_columns); + keys, block_with_columns_to_add, maps_vector, true); return keys.getByPosition(keys.columns() - 1); } @@ -1802,10 +1720,12 @@ void HashJoin::checkTypesOfKeys(const Block & block) const void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) { - for (size_t i = 0; i < key_names_left.size(); ++i) + for (const auto & onexpr : table_join->getClauses()) { - JoinCommon::checkTypesOfKeys(block, key_names_left[i], condition_mask_column_name_left[i], - right_sample_block, key_names_right[i], condition_mask_column_name_right[i]); + auto cond_column_name = onexpr.condColumnNames(); + JoinCommon::checkTypesOfKeys( + block, onexpr.key_names_left, cond_column_name.first, + right_sample_block, onexpr.key_names_right, cond_column_name.second); } if (kind == ASTTableJoin::Kind::Cross) @@ -1821,9 +1741,6 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) JoinCommon::convertColumnsToNullable(block); } - AddedColumnsV added_columns_v; - size_t existing_columns = block.columns(); - if (overDictionary()) { using Kind = ASTTableJoin::Kind; @@ -1839,37 +1756,20 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) { case Strictness::Any: case Strictness::All: - { - auto added_columns = makeAddedColumns( - block, key_names_left, sample_block_with_columns_to_add, maps_vector); - joinBlockImpl(block, std::move(added_columns), existing_columns); + joinBlockImpl(block, sample_block_with_columns_to_add, maps_vector); break; - } - case Strictness::Semi: - { - auto added_columns = makeAddedColumns( - block, key_names_left, sample_block_with_columns_to_add, maps_vector); - joinBlockImpl(block, std::move(added_columns), existing_columns); + joinBlockImpl(block, sample_block_with_columns_to_add, maps_vector); break; - } case Strictness::Anti: - { - auto added_columns = makeAddedColumns( - block, key_names_left, sample_block_with_columns_to_add, maps_vector); - joinBlockImpl(block, std::move(added_columns), existing_columns); + joinBlockImpl(block, sample_block_with_columns_to_add, maps_vector); break; - } default: throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: dictionary + {} {}", strictness, kind); } } else if (kind == Kind::Inner && strictness == Strictness::All) - { - auto added_columns = makeAddedColumns( - block, key_names_left/*[0]*/, sample_block_with_columns_to_add, maps_vector); - joinBlockImpl(block, std::move(added_columns), existing_columns); - } + joinBlockImpl(block, sample_block_with_columns_to_add, maps_vector); else throw throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind); @@ -1877,25 +1777,15 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) else { - // MapsVariantPtrVector maps_vector; - std::vectormaps[0])>* > maps_vector; - - for (size_t i = 0; i < key_names_left.size(); ++i) + std::vectormaps[0])> * > maps_vector; + for (size_t i = 0; i < table_join->getClauses().size(); ++i) { - // JoinCommon::checkTypesOfKeys(block, key_names_left[i], condition_mask_column_name_left[i], - // right_table_keys, key_names_right[i], condition_mask_column_name_right[i]); maps_vector.push_back(&data->maps[i]); } - std::unique_ptr added_columns; - joinDispatch(kind, strictness, maps_vector, [&](auto kind_, auto strictness_, auto & maps_vector_) + if (joinDispatch(kind, strictness, maps_vector, [&](auto kind_, auto strictness_, auto & maps_vector_) { - added_columns = makeAddedColumns(block, key_names_left, sample_block_with_columns_to_add, maps_vector_); - }); - - if (joinDispatch(kind, strictness, data->maps[0], [&](auto kind_, auto strictness_, auto &) - { - joinBlockImpl(block, std::move(added_columns), existing_columns); + joinBlockImpl(block, sample_block_with_columns_to_add, maps_vector_); })) { /// Joined @@ -1945,9 +1835,9 @@ struct AdderNonJoined /// Stream from not joined earlier rows of the right table. -/// Based on -/// map' offsetInternal saved in used_flags for single disjuncts -/// flags in BlockWithFlags for multiple disjuncts +/// Based on: +/// - map offsetInternal saved in used_flags for single disjuncts +/// - flags in BlockWithFlags for multiple disjuncts template class NotJoinedHash final : public NotJoinedBlocks::RightColumnsFiller { @@ -2100,7 +1990,7 @@ std::shared_ptr HashJoin::getNonJoinedBlocks(const Block & left { return {}; } - bool multiple_disjuncts = key_names_right.size() > 1; + bool multiple_disjuncts = !table_join->oneDisjunct(); if (multiple_disjuncts) { @@ -2125,7 +2015,7 @@ void HashJoin::reuseJoinedData(const HashJoin & join) data = join.data; from_storage_join = true; - bool multiple_disjuncts = key_names_left.size() > 1; + bool multiple_disjuncts = !table_join->oneDisjunct(); if (multiple_disjuncts) throw Exception("StorageJoin with ORs is not supported", ErrorCodes::NOT_IMPLEMENTED); diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 64fa26218a3..d309e9f5deb 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -69,10 +70,6 @@ public: } -using SizesVector = std::vector; - -class AddedColumns; - /** Data structure for implementation of JOIN. * It is just a hash table: keys -> rows of joined ("right") table. * Additionally, CROSS JOIN is supported: instead of hash table, it use just set of blocks without keys. @@ -194,11 +191,7 @@ public: ASOF::Inequality getAsofInequality() const { return asof_inequality; } bool anyTakeLastRow() const { return any_take_last_row; } - const ColumnWithTypeAndName & rightAsofKeyColumn() const - { - /// It should be nullable if nullable_right_side is true - return savedBlockSample().getByName(key_names_right.front().back()); - } + const Block & savedBlockSample() const { return data->sample_block; } /// Different types of keys for maps. #define APPLY_FOR_JOIN_VARIANTS(M) \ @@ -330,7 +323,7 @@ public: std::vector maps; Block sample_block; /// Block as it would appear in the BlockList - BlocksList blocks; + BlocksList blocks; /// Blocks of "right" table. BlockNullmapList blocks_nullmaps; /// Nullmaps for blocks of "right" table (if needed) /// Additional data - strings for string keys and continuation elements of single-linked lists of references to rows. @@ -365,10 +358,6 @@ private: /// This join was created from StorageJoin and it is already filled. bool from_storage_join = false; - /// Names of key columns in right-side table (in the order they appear in ON/USING clause). @note It could contain duplicates. - const NamesVector key_names_right; - const NamesVector key_names_left; - bool nullable_right_side; /// In case of LEFT and FULL joins, if use_nulls, convert right-side columns to Nullable. bool nullable_left_side; /// In case of RIGHT and FULL joins, if use_nulls, convert left-side columns to Nullable. bool any_take_last_row; /// Overwrite existing values when encountering the same key again @@ -383,7 +372,7 @@ private: /// so we must guarantee constantness of hash table during HashJoin lifetime (using method setLock) mutable JoinStuff::JoinUsedFlags used_flags; RightTableDataPtr data; - SizesVector key_sizes; + std::vector key_sizes; /// Block with columns from the right-side table. Block right_sample_block; @@ -396,10 +385,6 @@ private: /// Left table column names that are sources for required_right_keys columns std::vector required_right_keys_sources; - /// Additional conditions for rows to join from JOIN ON section - std::vector condition_mask_column_name_left; - std::vector condition_mask_column_name_right; - Poco::Logger * log; Block totals; @@ -408,50 +393,25 @@ private: /// If set HashJoin instance is not available for modification (addJoinedBlock) std::shared_lock storage_join_lock; - // void init(Type type_); - void init(Type type_, RightTableDataPtr); - void data_map_init(MapsVariant &); - - const Block & savedBlockSample() const { return data->sample_block; } + void dataMapInit(MapsVariant &); /// Modify (structure) right block to save it in block list Block structureRightBlock(const Block & stored_block) const; void initRightBlockStructure(Block & saved_block_sample); - template + template void joinBlockImpl( Block & block, - std::unique_ptr, - size_t existing_columns) const; - - template - std::unique_ptr makeAddedColumns( - Block & block, - const NamesVector & key_names_left, const Block & block_with_columns_to_add, - const std::vector & maps, - bool is_join_get = false) const; - - template - std::unique_ptr makeAddedColumnsV( - Block & block, - const Names & key_names_left, - const Block & block_with_columns_to_add, - const Maps & maps, - const Sizes & key_sizes_, - HashJoin::Type, + const std::vector & maps_, bool is_join_get = false) const; void joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const; - // template - // ColumnWithTypeAndName joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_, HashJoin::Type) const; - static Type chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes); bool empty() const; bool overDictionary() const; }; - } diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 2aced4e72be..f96b5a5bf28 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -507,12 +507,11 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right ErrorCodes::PARAMETER_OUT_OF_BOUND); } - const auto & key_names_left_all = table_join->keyNamesLeft(); - const auto & key_names_right_all = table_join->keyNamesRight(); - if (key_names_left_all.size() != 1 || key_names_right_all.size() != 1) - throw Exception("MergeJoin does not support OR", ErrorCodes::NOT_IMPLEMENTED); + if (!table_join->oneDisjunct()) + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "MergeJoin does not support OR in JOIN ON section"); - std::tie(mask_column_name_left, mask_column_name_right) = table_join->joinConditionColumnNames(0); + const auto & onexpr = table_join->getOnlyClause(); + std::tie(mask_column_name_left, mask_column_name_right) = onexpr.condColumnNames(); /// Add auxiliary joining keys to join only rows where conditions from JOIN ON sections holds /// Input boolean column converted to nullable and only rows with non NULLS value will be joined @@ -524,11 +523,11 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right key_names_right.push_back(deriveTempName(mask_column_name_right)); } - key_names_left.insert(key_names_left.end(), key_names_left_all.front().begin(), key_names_left_all.front().end()); - key_names_right.insert(key_names_right.end(), key_names_right_all.front().begin(), key_names_right_all.front().end()); + key_names_left.insert(key_names_left.end(), onexpr.key_names_left.begin(), onexpr.key_names_left.end()); + key_names_right.insert(key_names_right.end(), onexpr.key_names_right.begin(), onexpr.key_names_right.end()); addConditionJoinColumn(right_sample_block, JoinTableSide::Right); - JoinCommon::splitAdditionalColumns(NamesVector{key_names_right}, right_sample_block, right_table_keys, right_columns_to_add); + JoinCommon::splitAdditionalColumns(key_names_right, right_sample_block, right_table_keys, right_columns_to_add); for (const auto & right_key : key_names_right) { @@ -659,7 +658,7 @@ bool MergeJoin::saveRightBlock(Block && block) Block MergeJoin::modifyRightBlock(const Block & src_block) const { Block block = materializeBlock(src_block); - JoinCommon::removeLowCardinalityInplace(block, table_join->keyNamesRight().front()); + JoinCommon::removeLowCardinalityInplace(block, table_join->getOnlyClause().key_names_right); return block; } @@ -1108,8 +1107,8 @@ std::shared_ptr MergeJoin::getNonJoinedBlocks( { if (table_join->strictness() == ASTTableJoin::Strictness::All && (is_right || is_full)) { - size_t left_columns_count = left_sample_block.columns(); - assert(left_columns_count == result_sample_block.columns() - right_columns_to_add.columns()); + size_t left_columns_count = result_sample_block.columns() - right_columns_to_add.columns(); + assert(left_columns_count == left_sample_block.columns()); auto non_joined = std::make_unique(*this, max_block_size); return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); } diff --git a/src/Interpreters/NullableUtils.cpp b/src/Interpreters/NullableUtils.cpp index ce681b1d569..4c46df54c8d 100644 --- a/src/Interpreters/NullableUtils.cpp +++ b/src/Interpreters/NullableUtils.cpp @@ -1,4 +1,5 @@ #include +#include #include diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index edc03a794e2..b813a1f202b 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -51,6 +51,28 @@ std::string formatTypeMap(const TableJoin::NameToTypeMap & target, const TableJo } + +namespace +{ + +template +bool forAllKeys(OnExpr & expressions, Func callback) +{ + for (auto & expr : expressions) + { + assert(expr.key_names_left.size() == expr.key_names_right.size()); + for (size_t i = 0; i < expr.key_names_left.size(); ++i) + { + bool cont = callback(expr.key_names_left[i], expr.key_names_right[i]); + if (!cont) + return false; + } + } + return true; +} + +} + TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) : size_limits(SizeLimits{settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode}) , default_max_bytes(settings.default_max_bytes_in_join) @@ -62,16 +84,14 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) , partial_merge_join_left_table_buffer_bytes(settings.partial_merge_join_left_table_buffer_bytes) , max_files_to_merge(settings.join_on_disk_max_files_to_merge) , temporary_files_codec(settings.temporary_files_codec) - , left_clauses(1) - , right_clauses(1) + , clauses(1) , tmp_volume(tmp_volume_) { } void TableJoin::resetCollected() { - left_clauses = std::vector(1); - right_clauses = std::vector(1); + clauses = std::vector(1); columns_from_joined_table.clear(); columns_added_by_join.clear(); @@ -87,8 +107,7 @@ void TableJoin::resetCollected() void TableJoin::addUsingKey(const ASTPtr & ast) { - left_clauses.back().addKey(ast->getColumnName(), ast); - right_clauses.back().addKey(renamedRightColumnName(ast->getAliasOrColumnName()), ast); + addKey(ast->getColumnName(), renamedRightColumnName(ast->getAliasOrColumnName()), ast); } /// create new disjunct when see a direct child of a previously discovered OR @@ -98,10 +117,11 @@ void TableJoin::addDisjunct(const ASTPtr & ast) if (std::find_if(disjuncts.begin(), disjuncts.end(), [addr](const ASTPtr & ast_){return ast_.get() == addr;}) != disjuncts.end()) { - if (!left_clauses.back().key_names.empty() || !left_clauses.back().on_filter_conditions.empty() || !right_clauses.back().on_filter_conditions.empty()) + const auto & clause = clauses.back(); + if (!clause.key_names_left.empty() || !clause.on_filter_condition_left || + !clause.key_names_right.empty() || !clause.on_filter_condition_right) { - left_clauses.emplace_back(); - right_clauses.emplace_back(); + clauses.emplace_back(); } } } @@ -114,8 +134,7 @@ void TableJoin::setDisjuncts(Disjuncts&& disjuncts_) void TableJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast) { - left_clauses.back().addKey(left_table_ast->getColumnName(), left_table_ast); - right_clauses.back().addKey(right_table_ast->getAliasOrColumnName(), right_table_ast); + addKey(left_table_ast->getColumnName(), right_table_ast->getAliasOrColumnName(), left_table_ast, right_table_ast); } /// @return how many times right key appears in ON section. @@ -125,8 +144,8 @@ size_t TableJoin::rightKeyInclusion(const String & name) const return 0; size_t count = 0; - for (const auto & clause : right_clauses) - count += std::count(clause.key_names.begin(), clause.key_names.end(), name); + for (const auto & clause : clauses) + count += std::count(clause.key_names_right.begin(), clause.key_names_right.end(), name); return count; } @@ -173,12 +192,12 @@ NamesWithAliases TableJoin::getNamesWithAliases(const NameSet & required_columns ASTPtr TableJoin::leftKeysList() const { ASTPtr keys_list = std::make_shared(); - for (size_t i = 0; i < left_clauses.size(); ++i) + keys_list->children = key_asts_left; + + for (const auto & clause : clauses) { - const auto & clause = left_clauses[i]; - keys_list->children.insert(keys_list->children.end(), clause.key_asts.begin(), clause.key_asts.end()); - if (ASTPtr extra_cond = joinConditionColumn(JoinTableSide::Left, i)) - keys_list->children.push_back(extra_cond); + if (clause.on_filter_condition_left) + keys_list->children.push_back(clause.on_filter_condition_left); } return keys_list; } @@ -186,16 +205,14 @@ ASTPtr TableJoin::leftKeysList() const ASTPtr TableJoin::rightKeysList() const { ASTPtr keys_list = std::make_shared(); - for (size_t i = 0; i < right_clauses.size(); ++i) - { - if (hasOn()) - { - const auto & clause = right_clauses[i]; - keys_list->children.insert(keys_list->children.end(), clause.key_asts.begin(), clause.key_asts.end()); - } - if (ASTPtr extra_cond = joinConditionColumn(JoinTableSide::Right, i)) - keys_list->children.push_back(extra_cond); + if (hasOn()) + keys_list->children = key_asts_right; + + for (const auto & clause : clauses) + { + if (clause.on_filter_condition_right) + keys_list->children.push_back(clause.on_filter_condition_right); } return keys_list; } @@ -203,8 +220,8 @@ ASTPtr TableJoin::rightKeysList() const Names TableJoin::requiredJoinedNames() const { NameSet required_columns_set; - for (const auto & clause : right_clauses) - required_columns_set.insert(clause.key_names.begin(), clause.key_names.end()); + for (const auto & clause : clauses) + required_columns_set.insert(clause.key_names_right.begin(), clause.key_names_right.end()); for (const auto & joined_column : columns_added_by_join) required_columns_set.insert(joined_column.name); @@ -215,19 +232,18 @@ Names TableJoin::requiredJoinedNames() const NameSet TableJoin::requiredRightKeys() const { NameSet required; - for (const auto & clause : right_clauses) + forAllKeys(clauses, [this, &required](const auto &, const auto & name) { - for (const auto & name : clause.key_names) - { - auto rename = renamedRightColumnName(name); - for (const auto & column : columns_added_by_join) - if (rename == column.name) - required.insert(name); - } - } + auto rename = renamedRightColumnName(name); + for (const auto & column : columns_added_by_join) + if (rename == column.name) + required.insert(name); + return true; + }); return required; } + NamesWithAliases TableJoin::getRequiredColumns(const Block & sample, const Names & action_required_columns) const { NameSet required_columns(action_required_columns.begin(), action_required_columns.end()); @@ -243,29 +259,18 @@ NamesWithAliases TableJoin::getRequiredColumns(const Block & sample, const Names Block TableJoin::getRequiredRightKeys(const Block & right_table_keys, std::vector & keys_sources) const { - const auto & left_keys = keyNamesLeft(); - const auto & right_keys = keyNamesRight(); NameSet required_keys = requiredRightKeys(); Block required_right_keys; - - for (size_t p = 0; p < right_keys.size(); ++p) + forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) { - const auto & right_keys_part = right_keys[p]; - - for (size_t i = 0; i < right_keys_part.size(); ++i) + if (required_keys.count(right_key_name) && !required_right_keys.has(right_key_name)) { - const String & right_key_name = right_keys_part[i]; - - if (required_keys.count(right_key_name) && !required_right_keys.has(right_key_name)) - { - const auto & right_key = right_table_keys.getByName(right_key_name); - required_right_keys.insert(right_key); - const auto & left_keys_part = left_keys[p]; - keys_sources.push_back(left_keys_part[i]); - } + const auto & right_key = right_table_keys.getByName(right_key_name); + required_right_keys.insert(right_key); + keys_sources.push_back(left_key_name); } - } - + return true; + }); return required_right_keys; } @@ -347,6 +352,12 @@ bool TableJoin::sameStrictnessAndKind(ASTTableJoin::Strictness strictness_, ASTT return false; } +bool TableJoin::oneDisjunct() const +{ + assert(clauses.size() > 0); + return clauses.size() == 1; +} + bool TableJoin::allowMergeJoin() const { bool is_any = (strictness() == ASTTableJoin::Strictness::Any); @@ -356,9 +367,7 @@ bool TableJoin::allowMergeJoin() const bool all_join = is_all && (isInner(kind()) || isLeft(kind()) || isRight(kind()) || isFull(kind())); bool special_left = isLeft(kind()) && (is_any || is_semi); - bool no_ors = (left_clauses.size() == 1); - - return (all_join || special_left) && no_ors; + return (all_join || special_left) && oneDisjunct(); } bool TableJoin::needStreamWithNonJoinedRows() const @@ -394,12 +403,13 @@ bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context) if (!allowed_inner && !allowed_left) return false; - const Names & right_keys = right_clauses.front().key_names; - if (right_keys.size() != 1) + if (clauses.size() != 1 || clauses[0].key_names_right.size() != 1) return false; + const auto & right_key = clauses.front().key_names_right[0]; + /// TODO: support 'JOIN ... ON expr(dict_key) = table_key' - auto it_key = original_names.find(right_keys[0]); + auto it_key = original_names.find(right_key); if (it_key == original_names.end()) return false; @@ -416,7 +426,7 @@ bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context) NamesAndTypesList dst_columns; for (const auto & col : sample_block) { - if (col.name == right_keys[0]) + if (col.name == right_key) continue; /// do not extract key column auto it = original_names.find(col.name); @@ -437,8 +447,21 @@ TableJoin::createConvertingActions(const ColumnsWithTypeAndName & left_sample_co { inferJoinKeyCommonType(left_sample_columns, right_sample_columns, !isSpecialStorage()); - auto left_converting_actions = applyKeyConvertToTable(left_sample_columns, left_type_map, key_names_left); - auto right_converting_actions = applyKeyConvertToTable(right_sample_columns, right_type_map, key_names_right); + if (need_convert) + { + NameToNameMap left_key_column_rename; + NameToNameMap right_key_column_rename; + left_converting_actions = applyKeyConvertToTable(left_sample_columns, left_type_map, left_key_column_rename); + right_converting_actions = applyKeyConvertToTable(right_sample_columns, right_type_map, right_key_column_rename); + forAllKeys(clauses, [&](auto & left_key, auto & right_key) + { + if (const auto it = left_key_column_rename.find(left_key); it != left_key_column_rename.end()) + left_key = it->second; + if (const auto it = right_key_column_rename.find(left_key); it != right_key_column_rename.end()) + right_key = it->second; + return true; + }); + } return {left_converting_actions, right_converting_actions}; } @@ -457,43 +480,40 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig for (const auto & col : right) right_types[renamedRightColumnName(col.name)] = col.type; - for (size_t d = 0; d < left_clauses.size(); ++d) + + forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) { - auto & key_names_left = left_clauses[d].key_names; - auto & key_names_right = right_clauses[d].key_names; - for (size_t i = 0; i < key_names_left.size(); ++i) + auto ltype = left_types.find(left_key_name); + auto rtype = right_types.find(right_key_name); + if (ltype == left_types.end() || rtype == right_types.end()) { - auto ltype = left_types.find(key_names_left[i]); - auto rtype = right_types.find(key_names_right[i]); - if (ltype == left_types.end() || rtype == right_types.end()) - { - /// Name mismatch, give up - left_type_map.clear(); - right_type_map.clear(); - return false; - } - - if (JoinCommon::typesEqualUpToNullability(ltype->second, rtype->second)) - continue; - - DataTypePtr supertype; - try - { - /// TODO(vdimir): use getMostSubtype if possible - common_type = DB::getLeastSupertype({ltype->second, rtype->second}); - } - catch (DB::Exception & ex) - { - throw Exception( - "Type mismatch of columns to JOIN by: " + - key_names_left[d][i] + ": " + ltype->second->getName() + " at left, " + - key_names_right[d][i] + ": " + rtype->second->getName() + " at right. " + - "Can't get supertype: " + ex.message(), - ErrorCodes::TYPE_MISMATCH); - } - left_type_map[key_names_left[i]] = right_type_map[key_names_right[i]] = supertype; + /// Name mismatch, give up + left_type_map.clear(); + right_type_map.clear(); + return false; /// break; } - } + + if (JoinCommon::typesEqualUpToNullability(ltype->second, rtype->second)) + return true; /// continue; + + DataTypePtr supertype; + try + { + supertype = DB::getLeastSupertype({ltype->second, rtype->second}); + } + catch (DB::Exception & ex) + { + throw Exception( + "Type mismatch of columns to JOIN by: " + + left_key_name + ": " + ltype->second->getName() + " at left, " + + right_key_name + ": " + rtype->second->getName() + " at right. " + + "Can't get supertype: " + ex.message(), + ErrorCodes::TYPE_MISMATCH); + } + left_type_map[left_key_name] = right_type_map[right_key_name] = supertype; + + return true; + }); if (!left_type_map.empty() || !right_type_map.empty()) { @@ -508,7 +528,7 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig } ActionsDAGPtr TableJoin::applyKeyConvertToTable( - const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, std::vector & join_clause) const + const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, NameToNameMap & key_column_rename) const { bool has_some_to_do = false; @@ -525,21 +545,10 @@ ActionsDAGPtr TableJoin::applyKeyConvertToTable( if (!has_some_to_do) return nullptr; - NameToNameMap key_column_rename; /// Returns converting actions for tables that need to be performed before join auto dag = ActionsDAG::makeConvertingActions( cols_src, cols_dst, ActionsDAG::MatchColumnsMode::Name, true, !hasUsing(), &key_column_rename); - for (auto & clause : join_clause) - { - for (auto & name : clause.key_names) - { - const auto it = key_column_rename.find(name); - if (it != key_column_rename.end()) - name = it->second; - } - } - return dag; } @@ -564,12 +573,29 @@ String TableJoin::renamedRightColumnName(const String & name) const return name; } + +static void addJoinConditionWithAnd(ASTPtr & current_cond, const ASTPtr & new_cond) +{ + if (current_cond == nullptr) + { + /// no conditions, set new one + current_cond = new_cond; + } + else if (const auto * func = current_cond->as(); func && func->name == "and") + { + /// already have `and` in condition, just add new argument + func->arguments->children.push_back(new_cond); + } + else + { + /// already have some condition, unite coditions with `and` + current_cond = makeASTFunction("and", current_cond, new_cond); + } +} + void TableJoin::addJoinCondition(const ASTPtr & ast, bool is_left) { - if (is_left) - left_clauses.back().on_filter_conditions.push_back(ast); - else - right_clauses.back().on_filter_conditions.push_back(ast); + addJoinConditionWithAnd(is_left ? clauses.back().on_filter_condition_left : clauses.back().on_filter_condition_right, ast); } void TableJoin::leftToRightKeyRemap( @@ -597,43 +623,24 @@ std::unordered_map TableJoin::leftToRightKeyRemap() const if (hasUsing()) { const auto & required_right_keys = requiredRightKeys(); - for (size_t i = 0; i < left_clauses.size(); ++i) - TableJoin::leftToRightKeyRemap(left_clauses[i].key_names, right_clauses[i].key_names, required_right_keys, left_to_right_key_remap); + forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) + { + if (!required_right_keys.contains(right_key_name)) + left_to_right_key_remap[left_key_name] = right_key_name; + return true; + }); } return left_to_right_key_remap; } -/// Returns all conditions related to one table joined with 'and' function -static ASTPtr buildJoinConditionColumn(const ASTs & on_filter_condition_asts) +Names TableJoin::getAllNames(JoinTableSide side) const { - if (on_filter_condition_asts.empty()) - return nullptr; - - if (on_filter_condition_asts.size() == 1) - return on_filter_condition_asts[0]; - - auto function = std::make_shared(); - function->name = "and"; - function->arguments = std::make_shared(); - function->children.push_back(function->arguments); - function->arguments->children = on_filter_condition_asts; - return function; -} - -ASTPtr TableJoin::joinConditionColumn(JoinTableSide side, size_t disjunct) const -{ - if (side == JoinTableSide::Left) - return buildJoinConditionColumn(left_clauses[disjunct].on_filter_conditions); - return buildJoinConditionColumn(right_clauses[disjunct].on_filter_conditions); -} - -std::pair TableJoin::joinConditionColumnNames(size_t disjunct) const -{ - std::pair res; - if (auto cond_ast = joinConditionColumn(JoinTableSide::Left, disjunct)) - res.first = cond_ast->getColumnName(); - if (auto cond_ast = joinConditionColumn(JoinTableSide::Right, disjunct)) - res.second = cond_ast->getColumnName(); + Names res; + forAllKeys(clauses, [&res, side](const auto & left, const auto & right) + { + res.emplace_back(side == JoinTableSide::Left ? left : right); + return true; + }); return res; } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 77bf6a2215f..23c7f2488fa 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -11,7 +11,11 @@ #include #include #include +#include "Common/Exception.h" +#include "Parsers/IAST_fwd.h" +#include +#include #include #include #include @@ -48,6 +52,34 @@ public: using NameToTypeMap = std::unordered_map; using Disjuncts = ASTs; + /// Corresponds to one disjunct + struct JoinOnClause + { + Names key_names_left; + Names key_names_right; /// Duplicating right key names are qualified. + + ASTPtr on_filter_condition_left; + ASTPtr on_filter_condition_right; + + JoinOnClause() = default; + + std::pair condColumnNames() const + { + std::pair res; + if (on_filter_condition_left) + res.first = on_filter_condition_left->getColumnName(); + if (on_filter_condition_right) + res.second = on_filter_condition_right->getColumnName(); + return res; + } + + size_t keysCount() const + { + assert(key_names_left.size() == key_names_right.size()); + return key_names_right.size(); + } + }; + private: /** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k` * The join is made by column k. @@ -74,31 +106,12 @@ private: const size_t max_files_to_merge = 0; const String temporary_files_codec = "LZ4"; - /// Corresponds to one disjunct - struct JoinOnClause - { - Names key_names; - ASTs key_asts; - - ASTs on_filter_conditions; - - JoinOnClause() = default; - - explicit JoinOnClause(const Names & names) - : key_names(names) - {} - - void addKey(const String & name, const ASTPtr & ast) - { - key_names.emplace_back(name); - key_asts.emplace_back(ast); - } - }; + ASTs key_asts_left; + ASTs key_asts_right; Disjuncts disjuncts; - std::vector left_clauses; - std::vector right_clauses; /// Duplicating key_names are qualified. + std::vector clauses; ASTTableJoin table_join; @@ -131,7 +144,7 @@ private: /// Create converting actions and change key column names if required ActionsDAGPtr applyKeyConvertToTable( - const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, std::vector & join_clause) const; + const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, NameToNameMap & key_column_rename) const; /// Calculates common supertypes for corresponding join key columns. template @@ -144,13 +157,26 @@ private: const NameSet & required_right_keys, std::unordered_map & key_map) const; -public: - TableJoin() - : left_clauses(1) - , right_clauses(1) + void addKey(const String & left_name, const String & right_name, + const ASTPtr & left_ast, const ASTPtr & right_ast = nullptr) { + clauses.back().key_names_left.emplace_back(left_name); + key_asts_left.emplace_back(left_ast); + + clauses.back().key_names_right.emplace_back(right_name); + key_asts_right.emplace_back(right_ast ? right_ast : left_ast); } + void assertHasOneOnExpr() const + { + if (!oneDisjunct()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have only one join clause, got {}", clauses.size()); + } + +public: + TableJoin() : clauses(1) + {} + TableJoin(const Settings & settings, VolumePtr tmp_volume_); /// for StorageJoin @@ -160,9 +186,9 @@ public: , default_max_bytes(0) , join_use_nulls(use_nulls) , join_algorithm(JoinAlgorithm::HASH) - , left_clauses(1) + , clauses(1) { - right_clauses.emplace_back(key_names_right); + getOnlyClause().key_names_right = key_names_right; table_join.kind = kind; table_join.strictness = strictness; } @@ -192,6 +218,16 @@ public: bool enablePartialMergeJoinOptimizations() const { return partial_merge_join_optimizations; } bool needStreamWithNonJoinedRows() const; + bool oneDisjunct() const; + + JoinOnClause & getOnlyClause() { assertHasOneOnExpr(); return clauses[0]; } + const JoinOnClause & getOnlyClause() const { assertHasOneOnExpr(); return clauses[0]; } + + std::vector & getClauses() { return clauses; } + const std::vector & getClauses() const { return clauses; } + + Names getAllNames(JoinTableSide side) const; + void resetCollected(); void addUsingKey(const ASTPtr & ast); void setDisjuncts(Disjuncts &&); @@ -212,8 +248,6 @@ public: * doesn't supported yet, it can be added later. */ void addJoinCondition(const ASTPtr & ast, bool is_left); - ASTPtr joinConditionColumn(JoinTableSide side, size_t disjunct_num) const; - std::pair joinConditionColumnNames(size_t disjunct_num) const; bool hasUsing() const { return table_join.using_expression_list != nullptr; } bool hasOn() const { return table_join.on_expression != nullptr; } @@ -243,26 +277,6 @@ public: ASTPtr leftKeysList() const; ASTPtr rightKeysList() const; /// For ON syntax only - NamesVector keyNamesLeft() const - { - NamesVector key_names; - for (const auto & clause : left_clauses) - { - key_names.push_back(clause.key_names); - } - return key_names; - } - - NamesVector keyNamesRight() const - { - NamesVector key_names; - for (const auto & clause : right_clauses) - { - key_names.push_back(clause.key_names); - } - return key_names; - } - const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; } Names columnsAddedByJoin() const @@ -277,8 +291,9 @@ public: void setRightKeys(const Names & keys) { // assert(right_clauses.size() <= 1); - right_clauses.clear(); - right_clauses.emplace_back(keys); + clauses.clear(); + clauses.emplace_back(); + clauses.back().key_names_right = keys; } Block getRequiredRightKeys(const Block & right_table_keys, std::vector & keys_sources) const; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 2e35acaeb4b..8f11cb9968e 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -703,9 +703,9 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_ CollectJoinOnKeysVisitor::Data data{analyzed_join, tables[0], tables[1], aliases, is_asof}; CollectJoinOnKeysVisitor(data).visit(table_join.on_expression); - for (const auto & key_names_left_part : analyzed_join.keyNamesLeft()) + for (const auto & onexpr : analyzed_join.getClauses()) { - if (key_names_left_part.empty()) + if (onexpr.key_names_left.empty()) { throw Exception("Cannot get JOIN keys from JOIN ON section: " + queryToString(table_join.on_expression), ErrorCodes::INVALID_JOIN_ON_EXPRESSION); diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index d1e9cb09040..bf3c3d7655c 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -1,3 +1,4 @@ +#include #include #include @@ -306,6 +307,21 @@ ColumnRawPtrs materializeColumnsInplace(Block & block, const Names & names) return ptrs; } +ColumnRawPtrMap materializeColumnsInplaceMap(Block & block, const Names & names) +{ + ColumnRawPtrMap ptrs; + ptrs.reserve(names.size()); + + for (const auto & column_name : names) + { + auto & column = block.getByName(column_name).column; + column = recursiveRemoveLowCardinality(column->convertToFullColumnIfConst()); + ptrs[column_name] = column.get(); + } + + return ptrs; +} + ColumnPtr materializeColumn(const Block & block, const String & column_name) { const auto & src_column = block.getByName(column_name).column; @@ -517,21 +533,18 @@ ColumnPtr getColumnAsMask(const Block & block, const String & column_name) } -void splitAdditionalColumns(const NamesVector & key_names, const Block & sample_block, Block & block_keys, Block & block_others) +void splitAdditionalColumns(const Names & key_names, const Block & sample_block, Block & block_keys, Block & block_others) { block_others = materializeBlock(sample_block); - for (const auto & key_names_part : key_names) + for (const String & column_name : key_names) { - for (const String & column_name : key_names_part) + /// Extract right keys with correct keys order. There could be the same key names. + if (!block_keys.has(column_name)) { - /// Extract right keys with correct keys order. There could be the same key names. - if (!block_keys.has(column_name)) - { - auto & col = block_others.getByName(column_name); - block_keys.insert(col); - block_others.erase(column_name); - } + auto & col = block_others.getByName(column_name); + block_keys.insert(col); + block_others.erase(column_name); } } } diff --git a/src/Interpreters/join_common.h b/src/Interpreters/join_common.h index 4b39cee2c37..ec396104673 100644 --- a/src/Interpreters/join_common.h +++ b/src/Interpreters/join_common.h @@ -12,7 +12,9 @@ namespace DB struct ColumnWithTypeAndName; class TableJoin; class IColumn; + using ColumnRawPtrs = std::vector; +using ColumnRawPtrMap = std::unordered_map; using UInt8ColumnDataPtr = const ColumnUInt8::Container *; using UInt8ColumnDataPtrVector = std::vector; @@ -29,6 +31,7 @@ ColumnPtr emptyNotNullableClone(const ColumnPtr & column); ColumnPtr materializeColumn(const Block & block, const String & name); Columns materializeColumns(const Block & block, const Names & names); ColumnRawPtrs materializeColumnsInplace(Block & block, const Names & names); +ColumnRawPtrMap materializeColumnsInplaceMap(Block & block, const Names & names); ColumnRawPtrs getRawPointers(const Columns & columns); void removeLowCardinalityInplace(Block & block); void removeLowCardinalityInplace(Block & block, const Names & names, bool change_type = true); @@ -59,7 +62,7 @@ bool typesEqualUpToNullability(DataTypePtr left_type, DataTypePtr right_type); ColumnPtr getColumnAsMask(const Block & block, const String & column_name); /// Split key and other columns by keys name list -void splitAdditionalColumns(const NamesVector & key_names, const Block & sample_block, Block & block_keys, Block & block_others); +void splitAdditionalColumns(const Names & key_names, const Block & sample_block, Block & block_keys, Block & block_others); void changeLowCardinalityInplace(ColumnWithTypeAndName & column); @@ -102,9 +105,6 @@ private: /// Right block saved in Join Block saved_block_sample; - NamesVector key_names_left; - NamesVector key_names_right; - /// Output of join Block result_sample_block; diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 6f599ddd85c..687946659b6 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -197,13 +197,13 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes /// Push down is for left table only. We need to update JoinStep for push down into right. /// Only inner and left join are supported. Other types may generate default values for left table keys. /// So, if we push down a condition like `key != 0`, not all rows may be filtered. - if (table_join.keyNamesLeft().size() == 1 && (table_join.kind() == ASTTableJoin::Kind::Inner || table_join.kind() == ASTTableJoin::Kind::Left)) + if (table_join.oneDisjunct() && (table_join.kind() == ASTTableJoin::Kind::Inner || table_join.kind() == ASTTableJoin::Kind::Left)) { const auto & left_header = join->getInputStreams().front().header; const auto & res_header = join->getOutputStream().header; Names allowed_keys; - const auto & key_names_left = table_join.keyNamesLeft(); - for (const auto & name : key_names_left.front()) + const auto & key_names_left = table_join.getOnlyClause().key_names_left; + for (const auto & name : key_names_left) { /// Skip key if it is renamed. /// I don't know if it is possible. Just in case. From 0a9a028c6f9fb4e5079c72f6dffea5083f02d0d0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 9 Sep 2021 12:47:08 +0300 Subject: [PATCH 276/317] fix --- src/Interpreters/HashJoin.cpp | 61 +++++++++++++++++-------------- src/Interpreters/HashJoin.h | 4 +- src/Interpreters/MergeJoin.cpp | 4 +- src/Interpreters/TableJoin.cpp | 51 ++++++++++++++++++++++---- src/Interpreters/TableJoin.h | 28 +++++++------- src/Interpreters/TreeRewriter.cpp | 1 + src/Interpreters/join_common.cpp | 6 --- 7 files changed, 95 insertions(+), 60 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index f39abf11c99..fc4ff59e65a 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1,7 +1,5 @@ #include -#include #include -#include #include #include @@ -33,7 +31,8 @@ #include #include #include "Columns/IColumn.h" -#include "Interpreters/IJoin.h" + +#include namespace DB { @@ -235,18 +234,18 @@ static ColumnWithTypeAndName correctNullability(ColumnWithTypeAndName && column, return std::move(column); } -static std::string formatKeysDebug(const std::vector & caluses) -{ - std::vector res; - for (const auto & clause : caluses) - { - std::vector current; - for (size_t i = 0; i < clause.keysCount(); ++i) - current.emplace_back(fmt::format("{} == {}", clause.key_names_left[i], clause.key_names_right[i])); - res.emplace_back(fmt::format("{}", fmt::join(current, ", "))); - } - return fmt::format("{}", fmt::join(res, " | ")); -} +//static std::string formatKeysDebug(const std::vector & onexprs) +//{ +// std::vector res; +// for (const auto & onexpr : onexprs) +// { +// std::vector current; +// for (size_t i = 0; i < onexpr.keysCount(); ++i) +// current.emplace_back(fmt::format("{} == {}", onexpr.key_names_left[i], onexpr.key_names_right[i])); +// res.emplace_back(fmt::format("{}", fmt::join(current, ", "))); +// } +// return fmt::format("{}", fmt::join(res, " | ")); +//} HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_sample_block_, bool any_take_last_row_) : table_join(table_join_) @@ -274,8 +273,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s required_right_keys = table_join->getRequiredRightKeys(right_table_keys, required_right_keys_sources); } - LOG_DEBUG(log, "Join keys: [{}], required right: [{}])", formatKeysDebug(table_join->getClauses()), fmt::join(required_right_keys.getNames(), ", ")); - +// LOG_DEBUG(log, "Join keys: [{}], required right: [{}]", formatKeysDebug(table_join->getClauses()), fmt::join(required_right_keys.getNames(), ", ")); LOG_DEBUG(log, "Columns to add: [{}]", sample_block_with_columns_to_add.dumpStructure()); JoinCommon::removeLowCardinalityInplace(right_table_keys); @@ -290,7 +288,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s size_t disjuncts_num = table_join->getClauses().size(); - data->maps.reserve(disjuncts_num); + data->maps.resize(disjuncts_num); key_sizes.reserve(disjuncts_num); for (const auto & clause : table_join->getClauses()) @@ -332,8 +330,9 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s /// However, this does not depend on the size of the asof join key (as that goes into the BST) /// Therefore, add it back in such that it can be extracted appropriately from the full stored /// key_columns and key_sizes - key_sizes.resize(1); - key_sizes[0].push_back(asof_size); + auto & asof_key_sizes = key_sizes.emplace_back(); + data->type = chooseMethod(key_columns, asof_key_sizes); + asof_key_sizes.push_back(asof_size); } else { @@ -854,13 +853,13 @@ namespace struct JoinOnKeyColumns { - const Names & key_names; + Names key_names; Columns materialized_keys_holder; ColumnRawPtrs key_columns; - ColumnPtr null_map_holder; ConstNullMapPtr null_map; + ColumnPtr null_map_holder; /// Only rows where mask == true can be joined ColumnPtr join_mask_column; @@ -872,6 +871,7 @@ struct JoinOnKeyColumns : key_names(key_names_) , materialized_keys_holder(JoinCommon::materializeColumns(block, key_names)) /// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them. , key_columns(JoinCommon::getRawPointers(materialized_keys_holder)) + , null_map(nullptr) , null_map_holder(extractNestedColumnsAndNullMap(key_columns, null_map)) , join_mask_column(JoinCommon::getColumnAsMask(block, cond_column_name)) , key_sizes(key_sizes_) @@ -938,9 +938,9 @@ public: if (is_asof_join) { assert(join_on_keys.size() == 1); - const auto & right_asof_column = join.savedBlockSample().getByName(join_on_keys[0].key_names.front()); + const ColumnWithTypeAndName & right_asof_column = join.rightAsofKeyColumn(); addColumn(right_asof_column, right_asof_column.name); - left_asof_key = join_on_keys[0].key_columns.front(); + left_asof_key = join_on_keys[0].key_columns.back(); } for (auto & tn : type_name) @@ -1733,10 +1733,10 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) joinBlockImplCross(block, not_processed); return; } - else if (kind == ASTTableJoin::Kind::Right || kind == ASTTableJoin::Kind::Full) + + if (kind == ASTTableJoin::Kind::Right || kind == ASTTableJoin::Kind::Full) { materializeBlockInplace(block); - if (nullable_left_side) JoinCommon::convertColumnsToNullable(block); } @@ -1779,9 +1779,7 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) std::vectormaps[0])> * > maps_vector; for (size_t i = 0; i < table_join->getClauses().size(); ++i) - { maps_vector.push_back(&data->maps[i]); - } if (joinDispatch(kind, strictness, maps_vector, [&](auto kind_, auto strictness_, auto & maps_vector_) { @@ -2027,4 +2025,11 @@ void HashJoin::reuseJoinedData(const HashJoin & join) }); } } + +const ColumnWithTypeAndName & HashJoin::rightAsofKeyColumn() const +{ + /// It should be nullable if nullable_right_side is true + return savedBlockSample().getByName(table_join->getOnlyClause().key_names_right.back()); +} + } diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index d309e9f5deb..07fd6d5b89f 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -191,7 +191,7 @@ public: ASOF::Inequality getAsofInequality() const { return asof_inequality; } bool anyTakeLastRow() const { return any_take_last_row; } - const Block & savedBlockSample() const { return data->sample_block; } + const ColumnWithTypeAndName & rightAsofKeyColumn() const; /// Different types of keys for maps. #define APPLY_FOR_JOIN_VARIANTS(M) \ @@ -395,6 +395,8 @@ private: void dataMapInit(MapsVariant &); + const Block & savedBlockSample() const { return data->sample_block; } + /// Modify (structure) right block to save it in block list Block structureRightBlock(const Block & stored_block) const; void initRightBlockStructure(Block & saved_block_sample); diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index f96b5a5bf28..09f309603c3 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -1107,8 +1107,8 @@ std::shared_ptr MergeJoin::getNonJoinedBlocks( { if (table_join->strictness() == ASTTableJoin::Strictness::All && (is_right || is_full)) { - size_t left_columns_count = result_sample_block.columns() - right_columns_to_add.columns(); - assert(left_columns_count == left_sample_block.columns()); + size_t left_columns_count = left_sample_block.columns(); + assert(left_columns_count == result_sample_block.columns() - right_columns_to_add.columns()); auto non_joined = std::make_unique(*this, max_block_size); return std::make_shared(std::move(non_joined), result_sample_block, left_columns_count, table_join->leftToRightKeyRemap()); } diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index b813a1f202b..50df91b3289 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -55,15 +55,32 @@ std::string formatTypeMap(const TableJoin::NameToTypeMap & target, const TableJo namespace { -template +struct BothSidesTag {}; +struct LeftSideTag {}; +struct RightSideTag {}; + +template bool forAllKeys(OnExpr & expressions, Func callback) { + static_assert(std::is_same_v || + std::is_same_v || + std::is_same_v); + for (auto & expr : expressions) { - assert(expr.key_names_left.size() == expr.key_names_right.size()); + if constexpr (std::is_same_v) + assert(expr.key_names_left.size() == expr.key_names_right.size()); + for (size_t i = 0; i < expr.key_names_left.size(); ++i) { - bool cont = callback(expr.key_names_left[i], expr.key_names_right[i]); + bool cont; + if constexpr (std::is_same_v) + cont = callback(expr.key_names_left[i], expr.key_names_right[i]); + if constexpr (std::is_same_v) + cont = callback(expr.key_names_left[i]); + if constexpr (std::is_same_v) + cont = callback(expr.key_names_right[i]); + if (!cont) return false; } @@ -118,12 +135,14 @@ void TableJoin::addDisjunct(const ASTPtr & ast) if (std::find_if(disjuncts.begin(), disjuncts.end(), [addr](const ASTPtr & ast_){return ast_.get() == addr;}) != disjuncts.end()) { const auto & clause = clauses.back(); - if (!clause.key_names_left.empty() || !clause.on_filter_condition_left || - !clause.key_names_right.empty() || !clause.on_filter_condition_right) + if (!clause.key_names_left.empty() || !clause.key_names_right.empty() || + clause.on_filter_condition_left || clause.on_filter_condition_right) { clauses.emplace_back(); } } + if (joined_storage && clauses.size() > 1) + throw Exception("StorageJoin with ORs is not supported", ErrorCodes::NOT_IMPLEMENTED); } /// remember OR's children @@ -232,7 +251,7 @@ Names TableJoin::requiredJoinedNames() const NameSet TableJoin::requiredRightKeys() const { NameSet required; - forAllKeys(clauses, [this, &required](const auto &, const auto & name) + forAllKeys(clauses, [this, &required](const auto & name) { auto rename = renamedRightColumnName(name); for (const auto & column : columns_added_by_join) @@ -261,6 +280,9 @@ Block TableJoin::getRequiredRightKeys(const Block & right_table_keys, std::vecto { NameSet required_keys = requiredRightKeys(); Block required_right_keys; + if (required_keys.empty()) + return required_right_keys; + forAllKeys(clauses, [&](const auto & left_key_name, const auto & right_key_name) { if (required_keys.count(right_key_name) && !required_right_keys.has(right_key_name)) @@ -354,7 +376,7 @@ bool TableJoin::sameStrictnessAndKind(ASTTableJoin::Strictness strictness_, ASTT bool TableJoin::oneDisjunct() const { - assert(clauses.size() > 0); + assert(!clauses.empty()); return clauses.size() == 1; } @@ -406,7 +428,7 @@ bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context) if (clauses.size() != 1 || clauses[0].key_names_right.size() != 1) return false; - const auto & right_key = clauses.front().key_names_right[0]; + const auto & right_key = getOnlyClause().key_names_right[0]; /// TODO: support 'JOIN ... ON expr(dict_key) = table_key' auto it_key = original_names.find(right_key); @@ -644,4 +666,17 @@ Names TableJoin::getAllNames(JoinTableSide side) const return res; } +void TableJoin::assertHasOneOnExpr() const +{ + if (!oneDisjunct()) + { + std::vector text; + for (const auto & onexpr : clauses) + text.push_back(onexpr.formatDebug()); + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have only one join clause, got {}: [{}], query: '{}'", + clauses.size(), fmt::join(text, " | "), queryToString(table_join)); + + } +} + } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 23c7f2488fa..91f5e363307 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -11,14 +11,15 @@ #include #include #include -#include "Common/Exception.h" -#include "Parsers/IAST_fwd.h" +#include +#include #include #include #include #include -#include +#include +#include namespace DB { @@ -78,6 +79,13 @@ public: assert(key_names_left.size() == key_names_right.size()); return key_names_right.size(); } + + String formatDebug() const + { + return fmt::format("Left keys: [{}] Right keys [{}] Condition columns: '{}', '{}'", + fmt::join(key_names_left, ", "), fmt::join(key_names_right, ", "), + condColumnNames().first, condColumnNames().second); + } }; private: @@ -167,11 +175,7 @@ private: key_asts_right.emplace_back(right_ast ? right_ast : left_ast); } - void assertHasOneOnExpr() const - { - if (!oneDisjunct()) - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have only one join clause, got {}", clauses.size()); - } + void assertHasOneOnExpr() const; public: TableJoin() : clauses(1) @@ -288,13 +292,7 @@ public: } /// StorageJoin overrides key names (cause of different names qualification) - void setRightKeys(const Names & keys) - { - // assert(right_clauses.size() <= 1); - clauses.clear(); - clauses.emplace_back(); - clauses.back().key_names_right = keys; - } + void setRightKeys(const Names & keys) { getOnlyClause().key_names_right = keys; } Block getRequiredRightKeys(const Block & right_table_keys, std::vector & keys_sources) const; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 8f11cb9968e..70bf5cbe7ed 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -35,6 +35,7 @@ #include #include +#include namespace DB { diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index bf3c3d7655c..7bff473903d 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -201,13 +201,7 @@ void convertColumnsToNullable(MutableColumns & mutable_columns, size_t starting_ ColumnPtr column = std::move(mutable_columns[i]); column = makeNullable(column); mutable_columns[i] = IColumn::mutate(std::move(column)); - - // changeNullability(mutable_columns[i]); - // mutable_columns[i]->type = convertTypeToNullable(mutable_columns[i]->type); - - // convertColumnToNullable(*mutable_columns[i]); } - } /// @warning It assumes that every NULL has default value in nested column (or it does not matter) From 300eb5098d7b921bd8b5c074629efe43b6a7545a Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Tue, 14 Sep 2021 21:26:53 +0300 Subject: [PATCH 277/317] fix rebase collisions in ORs in JOIN --- src/Interpreters/HashJoin.cpp | 2 +- src/Interpreters/TableJoin.cpp | 55 ++++++++++++++++++---------------- 2 files changed, 31 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index fc4ff59e65a..fd446a7f44a 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -296,7 +296,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s const auto & key_names_right = clause.key_names_right; ColumnRawPtrs key_columns = JoinCommon::extractKeysForJoin(right_table_keys, key_names_right); - if (table_join->dictionary_reader) + if (table_join->getDictionaryReader()) { assert(disjuncts_num == 1); LOG_DEBUG(log, "Performing join over dict"); diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 50df91b3289..0a7253fe6f7 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -116,10 +116,6 @@ void TableJoin::resetCollected() renames.clear(); left_type_map.clear(); right_type_map.clear(); - key_names_left.resize(1); - key_names_right.resize(1); - on_filter_condition_asts_left.resize(1); - on_filter_condition_asts_right.resize(1); } void TableJoin::addUsingKey(const ASTPtr & ast) @@ -141,7 +137,7 @@ void TableJoin::addDisjunct(const ASTPtr & ast) clauses.emplace_back(); } } - if (joined_storage && clauses.size() > 1) + if (getStorageJoin() && clauses.size() > 1) throw Exception("StorageJoin with ORs is not supported", ErrorCodes::NOT_IMPLEMENTED); } @@ -464,29 +460,31 @@ bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context) return true; } + +static void tryRename(String & name, const NameToNameMap & renames) +{ + if (const auto it = renames.find(name); it != renames.end()) + name = it->second; +} + std::pair TableJoin::createConvertingActions(const ColumnsWithTypeAndName & left_sample_columns, const ColumnsWithTypeAndName & right_sample_columns) { inferJoinKeyCommonType(left_sample_columns, right_sample_columns, !isSpecialStorage()); - if (need_convert) - { - NameToNameMap left_key_column_rename; - NameToNameMap right_key_column_rename; - left_converting_actions = applyKeyConvertToTable(left_sample_columns, left_type_map, left_key_column_rename); - right_converting_actions = applyKeyConvertToTable(right_sample_columns, right_type_map, right_key_column_rename); - forAllKeys(clauses, [&](auto & left_key, auto & right_key) - { - if (const auto it = left_key_column_rename.find(left_key); it != left_key_column_rename.end()) - left_key = it->second; - if (const auto it = right_key_column_rename.find(left_key); it != right_key_column_rename.end()) - right_key = it->second; - return true; - }); - } + NameToNameMap left_key_column_rename; + NameToNameMap right_key_column_rename; + auto left_converting_actions = applyKeyConvertToTable(left_sample_columns, left_type_map, left_key_column_rename); + auto right_converting_actions = applyKeyConvertToTable(right_sample_columns, right_type_map, right_key_column_rename); - return {left_converting_actions, right_converting_actions}; -} + forAllKeys(clauses, [&](auto & left_key, auto & right_key) + { + tryRename(left_key, left_key_column_rename); + tryRename(right_key, right_key_column_rename); + return true; + }); + + return {left_converting_actions, right_converting_actions};} template bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const RightNamesAndTypes & right, bool allow_right) @@ -518,10 +516,11 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig if (JoinCommon::typesEqualUpToNullability(ltype->second, rtype->second)) return true; /// continue; - DataTypePtr supertype; + DataTypePtr common_type; try { - supertype = DB::getLeastSupertype({ltype->second, rtype->second}); + /// TODO(vdimir): use getMostSubtype if possible + common_type = DB::getLeastSupertype({ltype->second, rtype->second}); } catch (DB::Exception & ex) { @@ -532,7 +531,13 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig "Can't get supertype: " + ex.message(), ErrorCodes::TYPE_MISMATCH); } - left_type_map[left_key_name] = right_type_map[right_key_name] = supertype; + if (!allow_right && !common_type->equals(*rtype->second)) + { + throw DB::Exception(ErrorCodes::TYPE_MISMATCH, + "Can't change type for right table: {}: {} -> {}.", + right_key_name, rtype->second->getName(), common_type->getName()); + } + left_type_map[left_key_name] = right_type_map[right_key_name] = common_type; return true; }); From 71b6c9414c53abcf68bd725dd80d10ffd66d35b9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 10 Sep 2021 17:52:44 +0300 Subject: [PATCH 278/317] Minor changes related to JOIN ON ORs --- src/Columns/ColumnNullable.h | 1 - src/Interpreters/HashJoin.cpp | 25 +++++++--- src/Interpreters/IJoin.h | 3 +- src/Interpreters/JoinSwitcher.h | 6 +-- src/Interpreters/NullableUtils.cpp | 1 - src/Interpreters/TableJoin.cpp | 76 +++++++++++------------------- src/Interpreters/TableJoin.h | 34 ++----------- src/Interpreters/TreeRewriter.cpp | 1 - src/Interpreters/joinDispatch.h | 2 +- src/Interpreters/join_common.cpp | 1 - src/Interpreters/join_common.h | 1 - src/Parsers/IAST_fwd.h | 1 - src/Storages/StorageJoin.cpp | 3 ++ 13 files changed, 59 insertions(+), 96 deletions(-) diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 700ad24d27d..9da7b0dac1c 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -13,7 +13,6 @@ namespace DB using NullMap = ColumnUInt8::Container; using ConstNullMapPtr = const NullMap *; -using ConstNullMapPtrVector = std::vector; /// Class that specifies nullable columns. A nullable column represents /// a column, which may have any type, provided with the possibility of diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index fd446a7f44a..d02a2bf3e52 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -260,21 +260,32 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s , log(&Poco::Logger::get("HashJoin")) { LOG_DEBUG(log, "Right sample block: {}", right_sample_block.dumpStructure()); - if (!table_join->oneDisjunct()) + + if (isComma(kind) || isCross(kind)) { - /// required right keys concept does not work well if multiple disjuncts, - /// we need all keys - sample_block_with_columns_to_add = right_table_keys = materializeBlock(right_sample_block); + data->type = Type::CROSS; + sample_block_with_columns_to_add = right_sample_block; } - else + else if (table_join->oneDisjunct()) { const auto & key_names_right = table_join->getOnlyClause().key_names_right; JoinCommon::splitAdditionalColumns(key_names_right, right_sample_block, right_table_keys, sample_block_with_columns_to_add); required_right_keys = table_join->getRequiredRightKeys(right_table_keys, required_right_keys_sources); } + else + { + /// required right keys concept does not work well if multiple disjuncts, we need all keys + sample_block_with_columns_to_add = right_table_keys = materializeBlock(right_sample_block); + } -// LOG_DEBUG(log, "Join keys: [{}], required right: [{}]", formatKeysDebug(table_join->getClauses()), fmt::join(required_right_keys.getNames(), ", ")); - LOG_DEBUG(log, "Columns to add: [{}]", sample_block_with_columns_to_add.dumpStructure()); + LOG_TRACE(log, "Columns to add: [{}], required right [{}]", + sample_block_with_columns_to_add.dumpStructure(), fmt::join(required_right_keys.getNames(), ", ")); + { + std::vector log_text; + for (const auto & clause : table_join->getClauses()) + log_text.push_back(clause.formatDebug()); + LOG_TRACE(log, "Joining on: {}", fmt::join(log_text, " | ")); + } JoinCommon::removeLowCardinalityInplace(right_table_keys); diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 620ec6107c4..ba8367b57e3 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -45,7 +45,8 @@ public: /// Different query plan is used for such joins. virtual bool isFilled() const { return false; } - virtual std::shared_ptr getNonJoinedBlocks(const Block &, const Block &, UInt64) const = 0; + virtual std::shared_ptr + getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const = 0; }; using JoinPtr = std::shared_ptr; diff --git a/src/Interpreters/JoinSwitcher.h b/src/Interpreters/JoinSwitcher.h index 86d8f9d1ba0..aaa7441b8a4 100644 --- a/src/Interpreters/JoinSwitcher.h +++ b/src/Interpreters/JoinSwitcher.h @@ -61,10 +61,10 @@ public: return join->alwaysReturnsEmptySet(); } - std::shared_ptr getNonJoinedBlocks( - const Block & left_sample_block, const Block & result_block, UInt64 max_block_size) const override + std::shared_ptr + getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override { - return join->getNonJoinedBlocks(left_sample_block, result_block, max_block_size); + return join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size); } private: diff --git a/src/Interpreters/NullableUtils.cpp b/src/Interpreters/NullableUtils.cpp index 4c46df54c8d..ce681b1d569 100644 --- a/src/Interpreters/NullableUtils.cpp +++ b/src/Interpreters/NullableUtils.cpp @@ -1,5 +1,4 @@ #include -#include #include diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 0a7253fe6f7..d48d3293642 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -101,7 +101,6 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) , partial_merge_join_left_table_buffer_bytes(settings.partial_merge_join_left_table_buffer_bytes) , max_files_to_merge(settings.join_on_disk_max_files_to_merge) , temporary_files_codec(settings.temporary_files_codec) - , clauses(1) , tmp_volume(tmp_volume_) { } @@ -234,10 +233,8 @@ ASTPtr TableJoin::rightKeysList() const Names TableJoin::requiredJoinedNames() const { - NameSet required_columns_set; - for (const auto & clause : clauses) - required_columns_set.insert(clause.key_names_right.begin(), clause.key_names_right.end()); - + Names key_names_right = getAllNames(JoinTableSide::Right); + NameSet required_columns_set(key_names_right.begin(), key_names_right.end()); for (const auto & joined_column : columns_added_by_join) required_columns_set.insert(joined_column.name); @@ -258,16 +255,13 @@ NameSet TableJoin::requiredRightKeys() const return required; } - NamesWithAliases TableJoin::getRequiredColumns(const Block & sample, const Names & action_required_columns) const { NameSet required_columns(action_required_columns.begin(), action_required_columns.end()); for (auto & column : requiredJoinedNames()) - { if (!sample.has(column)) required_columns.insert(column); - } return getNamesWithAliases(required_columns); } @@ -372,8 +366,9 @@ bool TableJoin::sameStrictnessAndKind(ASTTableJoin::Strictness strictness_, ASTT bool TableJoin::oneDisjunct() const { - assert(!clauses.empty()); - return clauses.size() == 1; + if (!isComma(kind()) && !isCross(kind())) + assert(!clauses.empty()); + return clauses.size() <= 1; } bool TableJoin::allowMergeJoin() const @@ -460,8 +455,7 @@ bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context) return true; } - -static void tryRename(String & name, const NameToNameMap & renames) +static void renameIfNeeded(String & name, const NameToNameMap & renames) { if (const auto it = renames.find(name); it != renames.end()) name = it->second; @@ -479,8 +473,8 @@ TableJoin::createConvertingActions(const ColumnsWithTypeAndName & left_sample_co forAllKeys(clauses, [&](auto & left_key, auto & right_key) { - tryRename(left_key, left_key_column_rename); - tryRename(right_key, right_key_column_rename); + renameIfNeeded(left_key, left_key_column_rename); + renameIfNeeded(right_key, right_key_column_rename); return true; }); @@ -510,11 +504,11 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig /// Name mismatch, give up left_type_map.clear(); right_type_map.clear(); - return false; /// break; + return false; } if (JoinCommon::typesEqualUpToNullability(ltype->second, rtype->second)) - return true; /// continue; + return true; DataTypePtr common_type; try @@ -600,48 +594,34 @@ String TableJoin::renamedRightColumnName(const String & name) const return name; } +void TableJoin::addKey(const String & left_name, const String & right_name, const ASTPtr & left_ast, const ASTPtr & right_ast) +{ + clauses.back().key_names_left.emplace_back(left_name); + key_asts_left.emplace_back(left_ast); + + clauses.back().key_names_right.emplace_back(right_name); + key_asts_right.emplace_back(right_ast ? right_ast : left_ast); +} static void addJoinConditionWithAnd(ASTPtr & current_cond, const ASTPtr & new_cond) { if (current_cond == nullptr) - { /// no conditions, set new one current_cond = new_cond; - } else if (const auto * func = current_cond->as(); func && func->name == "and") - { /// already have `and` in condition, just add new argument func->arguments->children.push_back(new_cond); - } else - { - /// already have some condition, unite coditions with `and` + /// already have some conditions, unite it with `and` current_cond = makeASTFunction("and", current_cond, new_cond); - } } void TableJoin::addJoinCondition(const ASTPtr & ast, bool is_left) { - addJoinConditionWithAnd(is_left ? clauses.back().on_filter_condition_left : clauses.back().on_filter_condition_right, ast); -} - -void TableJoin::leftToRightKeyRemap( - const Names & left_keys, - const Names & right_keys, - const NameSet & required_right_keys, - std::unordered_map & key_map) const -{ - if (hasUsing()) - { - for (size_t i = 0; i < left_keys.size(); ++i) - { - const String & left_key_name = left_keys[i]; - const String & right_key_name = right_keys[i]; - - if (!required_right_keys.contains(right_key_name)) - key_map[left_key_name] = right_key_name; - } - } + auto & cond_ast = is_left ? clauses.back().on_filter_condition_left : clauses.back().on_filter_condition_right; + LOG_TRACE(&Poco::Logger::get("TableJoin"), "Adding join condition for {} table: {} -> {}", + (is_left ? "left" : "right"), ast ? queryToString(ast) : "NULL", cond_ast ? queryToString(cond_ast) : "NULL"); + addJoinConditionWithAnd(cond_ast, ast); } std::unordered_map TableJoin::leftToRightKeyRemap() const @@ -663,11 +643,11 @@ std::unordered_map TableJoin::leftToRightKeyRemap() const Names TableJoin::getAllNames(JoinTableSide side) const { Names res; - forAllKeys(clauses, [&res, side](const auto & left, const auto & right) - { - res.emplace_back(side == JoinTableSide::Left ? left : right); - return true; - }); + auto func = [&res](const auto & name) { res.emplace_back(name); return true; }; + if (side == JoinTableSide::Left) + forAllKeys(clauses, func); + else + forAllKeys(clauses, func); return res; } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 91f5e363307..d83050d9243 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -12,10 +12,7 @@ #include #include #include -#include -#include -#include #include #include #include @@ -51,7 +48,6 @@ class TableJoin public: using NameToTypeMap = std::unordered_map; - using Disjuncts = ASTs; /// Corresponds to one disjunct struct JoinOnClause @@ -114,13 +110,11 @@ private: const size_t max_files_to_merge = 0; const String temporary_files_codec = "LZ4"; + std::vector clauses; + ASTs key_asts_left; ASTs key_asts_right; - Disjuncts disjuncts; - - std::vector clauses; - ASTTableJoin table_join; ASOF::Inequality asof_inequality = ASOF::Inequality::GreaterOrEquals; @@ -154,26 +148,7 @@ private: ActionsDAGPtr applyKeyConvertToTable( const ColumnsWithTypeAndName & cols_src, const NameToTypeMap & type_mapping, NameToNameMap & key_column_rename) const; - /// Calculates common supertypes for corresponding join key columns. - template - bool inferJoinKeyCommonType(const LeftNamesAndTypes & left, const RightNamesAndTypes & right, bool allow_right); - - NamesAndTypesList correctedColumnsAddedByJoin() const; - void leftToRightKeyRemap( - const Names & left_keys, - const Names & right_keys, - const NameSet & required_right_keys, - std::unordered_map & key_map) const; - - void addKey(const String & left_name, const String & right_name, - const ASTPtr & left_ast, const ASTPtr & right_ast = nullptr) - { - clauses.back().key_names_left.emplace_back(left_name); - key_asts_left.emplace_back(left_ast); - - clauses.back().key_names_right.emplace_back(right_name); - key_asts_right.emplace_back(right_ast ? right_ast : left_ast); - } + void addKey(const String & left_name, const String & right_name, const ASTPtr & left_ast, const ASTPtr & right_ast = nullptr); void assertHasOneOnExpr() const; @@ -190,9 +165,8 @@ public: , default_max_bytes(0) , join_use_nulls(use_nulls) , join_algorithm(JoinAlgorithm::HASH) - , clauses(1) { - getOnlyClause().key_names_right = key_names_right; + clauses.emplace_back().key_names_right = key_names_right; table_join.kind = kind; table_join.strictness = strictness; } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 70bf5cbe7ed..8f11cb9968e 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -35,7 +35,6 @@ #include #include -#include namespace DB { diff --git a/src/Interpreters/joinDispatch.h b/src/Interpreters/joinDispatch.h index 6245aaefe88..59b12536e16 100644 --- a/src/Interpreters/joinDispatch.h +++ b/src/Interpreters/joinDispatch.h @@ -113,7 +113,7 @@ inline bool joinDispatch(ASTTableJoin::Kind kind, ASTTableJoin::Strictness stric if (kind == KINDS[i] && strictness == STRICTNESSES[j]) { using MapType = typename MapGetter::Map; - std::vector v; + std::vector v; for (const auto & el : mapsv) { v.push_back(&std::get(*el)); diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index 7bff473903d..dec925d68c1 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -1,4 +1,3 @@ -#include #include #include diff --git a/src/Interpreters/join_common.h b/src/Interpreters/join_common.h index ec396104673..32264d57d33 100644 --- a/src/Interpreters/join_common.h +++ b/src/Interpreters/join_common.h @@ -16,7 +16,6 @@ class IColumn; using ColumnRawPtrs = std::vector; using ColumnRawPtrMap = std::unordered_map; using UInt8ColumnDataPtr = const ColumnUInt8::Container *; -using UInt8ColumnDataPtrVector = std::vector; namespace JoinCommon { diff --git a/src/Parsers/IAST_fwd.h b/src/Parsers/IAST_fwd.h index 437bd50b2e7..18ba79d6618 100644 --- a/src/Parsers/IAST_fwd.h +++ b/src/Parsers/IAST_fwd.h @@ -9,6 +9,5 @@ namespace DB class IAST; using ASTPtr = std::shared_ptr; using ASTs = std::vector; -using ASTsVector = std::vector; } diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index fdc026bb6c2..e45183591f2 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -377,6 +377,9 @@ public: , max_block_size(max_block_size_) , sample_block(std::move(sample_block_)) { + if (!join->getTableJoin().oneDisjunct()) + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "StorageJoin does not support OR for keys in JOIN ON section"); + column_indices.resize(sample_block.columns()); auto & saved_block = join->getJoinedData()->sample_block; From f8e8f6da14bef61f7391c4f85611ecba117686d6 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Sep 2021 12:33:25 +0300 Subject: [PATCH 279/317] Use table_join->getAllNames in HashJoin.cpp --- src/Interpreters/HashJoin.cpp | 8 +------- src/Interpreters/TableJoin.cpp | 3 ++- 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index d02a2bf3e52..6baf7cda4f8 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -758,13 +758,7 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) Block block = materializeBlock(source_block); size_t rows = block.rows(); - ColumnRawPtrMap all_key_columns; - { - Names all_key_names_right; - for (const auto & clause : table_join->getClauses()) - all_key_names_right.insert(all_key_names_right.end(), clause.key_names_right.begin(), clause.key_names_right.end()); - all_key_columns = JoinCommon::materializeColumnsInplaceMap(block, all_key_names_right); - } + ColumnRawPtrMap all_key_columns = JoinCommon::materializeColumnsInplaceMap(block, table_join->getAllNames(JoinTableSide::Right)); Block structured_block = structureRightBlock(block); size_t total_rows = 0; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index d48d3293642..ce6103ec87d 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -71,7 +71,8 @@ bool forAllKeys(OnExpr & expressions, Func callback) if constexpr (std::is_same_v) assert(expr.key_names_left.size() == expr.key_names_right.size()); - for (size_t i = 0; i < expr.key_names_left.size(); ++i) + size_t sz = !std::is_same_v ? expr.key_names_left.size() : expr.key_names_right.size(); + for (size_t i = 0; i < sz; ++i) { bool cont; if constexpr (std::is_same_v) From 212ba1b64f01aa592da3e7af80f9a22a33c913fb Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Sep 2021 12:34:02 +0300 Subject: [PATCH 280/317] Add join_on_or_long.sql --- .../02024_join_on_or_long.reference | 64 +++++++++++++++++++ .../0_stateless/02024_join_on_or_long.sql.j2 | 43 +++++++++++++ 2 files changed, 107 insertions(+) create mode 100644 tests/queries/0_stateless/02024_join_on_or_long.reference create mode 100644 tests/queries/0_stateless/02024_join_on_or_long.sql.j2 diff --git a/tests/queries/0_stateless/02024_join_on_or_long.reference b/tests/queries/0_stateless/02024_join_on_or_long.reference new file mode 100644 index 00000000000..5cd5e5ee5fb --- /dev/null +++ b/tests/queries/0_stateless/02024_join_on_or_long.reference @@ -0,0 +1,64 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02024_join_on_or_long.sql.j2 b/tests/queries/0_stateless/02024_join_on_or_long.sql.j2 new file mode 100644 index 00000000000..58c6e02f426 --- /dev/null +++ b/tests/queries/0_stateless/02024_join_on_or_long.sql.j2 @@ -0,0 +1,43 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (`a` Int64, `b` Int64) ENGINE = MergeTree() ORDER BY a; +CREATE TABLE t2 (`key` Int32, `val` Int64) ENGINE = MergeTree() ORDER BY key; + +INSERT INTO t1 SELECT number + 500 as a, -a as b from numbers(1000); +INSERT INTO t2 SELECT if(number % 2 == 0, toInt64(number), -number) as key, number as val from numbers(1000); + +{% for bs in [99, 100, 101, 127, 128, 129, 256, 2048] -%} + +SET max_block_size = {{ bs }}; + +SET join_use_nulls = 1; + +SELECT count() == 500 AND count(a) == 500 AND sum(a) == 1499 * 500 / 2 FROM ( + SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key); + +SELECT count() == 1000 AND count(val) == 500 AND sum(val) == 1499 * 500 / 2 FROM ( + SELECT a, b, val FROM t1 LEFT JOIN t2 ON t1.a = t2.key OR t1.b = t2.key); + +SELECT count() == 1000 AND count(a) == 500 AND sum(val) == 999 * 1000 / 2 FROM ( + SELECT * FROM t1 RIGHT JOIN t2 ON t1.a = t2.key OR t1.b = t2.key); + +SELECT count() == 1500 AND count(a) == 1000 AND count(val) = 1000 AND sum(val) == 999 * 1000 / 2 AND sum(a) == 1999 * 1000 / 2 FROM ( + SELECT a, b, val FROM t1 FULL JOIN t2 ON t1.a = t2.key OR t1.b = t2.key); + +SET join_use_nulls = 0; + +SELECT count() == 500 AND count(a) == 500 AND sum(a) == 1499 * 500 / 2 FROM ( + SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key); + +SELECT count() == 1000 AND count(val) == 1000 AND sum(val) == 1499 * 500 / 2 FROM ( + SELECT a, b, val FROM t1 LEFT JOIN t2 ON t1.a = t2.key OR t1.b = t2.key); + +SELECT count() == 1000 AND count(a) == 1000 AND sum(val) == 999 * 1000 / 2 FROM ( + SELECT * FROM t1 RIGHT JOIN t2 ON t1.a = t2.key OR t1.b = t2.key); + +SELECT count() == 1500 AND count(a) == 1500 AND count(val) = 1500 AND sum(val) == 999 * 1000 / 2 AND sum(a) == 1999 * 1000 / 2 FROM ( + SELECT a, b, val FROM t1 FULL JOIN t2 ON t1.a = t2.key OR t1.b = t2.key); + + +{% endfor %} From 3b35ab6e8cb5695515f8af7c77cf9aa6225edfc2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 13 Sep 2021 12:34:34 +0300 Subject: [PATCH 281/317] Not implemented for asof and auto join with multiple ORs --- src/Interpreters/TreeRewriter.cpp | 8 ++++++++ tests/queries/0_stateless/00976_asof_join_on.sql | 2 ++ .../0_stateless/01115_join_with_dictionary.reference | 5 +++++ tests/queries/0_stateless/01115_join_with_dictionary.sql | 2 ++ .../queries/0_stateless/01429_join_on_error_messages.sql | 8 ++++++++ 5 files changed, 25 insertions(+) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 8f11cb9968e..0d15819f69f 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -713,7 +713,15 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_ } if (is_asof) + { + if (!analyzed_join.oneDisjunct()) + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "ASOF join doesn't support multiple ORs for keys in JOIN ON section"); data.asofToJoinKeys(); + } + + if (!analyzed_join.oneDisjunct() && !analyzed_join.forceHashJoin()) + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "Only `hash` join supports multiple ORs for keys in JOIN ON section"); + } } diff --git a/tests/queries/0_stateless/00976_asof_join_on.sql b/tests/queries/0_stateless/00976_asof_join_on.sql index ccecc0999c9..8060fb86831 100644 --- a/tests/queries/0_stateless/00976_asof_join_on.sql +++ b/tests/queries/0_stateless/00976_asof_join_on.sql @@ -21,5 +21,7 @@ SELECT A.a, A.t, B.b, B.t FROM A ASOF JOIN B ON A.a == B.b AND A.t < B.t ORDER B SELECT count() FROM A ASOF JOIN B ON A.a == B.b AND A.t == B.t; -- { serverError 403 } SELECT count() FROM A ASOF JOIN B ON A.a == B.b AND A.t != B.t; -- { serverError 403 } +SELECT A.a, A.t, B.b, B.t FROM A ASOF JOIN B ON A.a == B.b AND A.t < B.t OR A.a == B.b + 1 ORDER BY (A.a, A.t); -- { serverError 48 } + DROP TABLE A; DROP TABLE B; diff --git a/tests/queries/0_stateless/01115_join_with_dictionary.reference b/tests/queries/0_stateless/01115_join_with_dictionary.reference index 76337bdfc0b..326f0c5e14b 100644 --- a/tests/queries/0_stateless/01115_join_with_dictionary.reference +++ b/tests/queries/0_stateless/01115_join_with_dictionary.reference @@ -37,6 +37,11 @@ flat: inner on 1 1 1 1 1 2 2 2 2 2 3 3 3 3 3 +flat: inner or +0 0 0 0 0 +1000 1 1 1 1 +2 2 2 2 2 +3000 3 3 3 3 hashed: left on 0 0 0 0 0 1 1 1 1 1 diff --git a/tests/queries/0_stateless/01115_join_with_dictionary.sql b/tests/queries/0_stateless/01115_join_with_dictionary.sql index 8d8589d6085..5fbfe283fea 100644 --- a/tests/queries/0_stateless/01115_join_with_dictionary.sql +++ b/tests/queries/0_stateless/01115_join_with_dictionary.sql @@ -43,6 +43,8 @@ SELECT 'flat: inner'; SELECT * FROM (SELECT number AS key FROM numbers(2)) s1 JOIN dict_flat d USING(key); SELECT 'flat: inner on'; SELECT * FROM (SELECT number AS k FROM numbers(100)) s1 JOIN dict_flat d ON k = key ORDER BY k; +SELECT 'flat: inner or'; -- it's not a join over dictionary, because it doen't suppoert multiple keys, but of falls back to regular join +SELECT * FROM (SELECT if(number % 2 = 0, number, number * 1000) AS k FROM numbers(100)) s1 JOIN dict_flat d ON k = key OR k == 1000 * key ORDER BY key; SET join_use_nulls = 1; diff --git a/tests/queries/0_stateless/01429_join_on_error_messages.sql b/tests/queries/0_stateless/01429_join_on_error_messages.sql index 901e0e16e43..edc6cdcb617 100644 --- a/tests/queries/0_stateless/01429_join_on_error_messages.sql +++ b/tests/queries/0_stateless/01429_join_on_error_messages.sql @@ -8,3 +8,11 @@ SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a > b; -- { server SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a < b; -- { serverError 403 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a >= b; -- { serverError 403 } SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a <= b; -- { serverError 403 } + +SET join_algorithm = 'partial_merge'; +SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b OR a = b; -- { serverError 48 } + +SET join_algorithm = 'auto'; +SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b OR a = b; -- { serverError 48 } + +SET join_algorithm = 'hash'; From 637ff19f795d9655ff7d10cc7268ea7398e4d098 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 15 Sep 2021 14:16:10 +0300 Subject: [PATCH 282/317] optimizeDisjuncts in ORs in JOIN --- src/Interpreters/CollectJoinOnKeysVisitor.cpp | 5 ++ src/Interpreters/CollectJoinOnKeysVisitor.h | 1 + src/Interpreters/TableJoin.cpp | 68 +++++++++++++++++++ src/Interpreters/TableJoin.h | 27 +++++++- src/Interpreters/TreeRewriter.cpp | 3 + 5 files changed, 102 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/src/Interpreters/CollectJoinOnKeysVisitor.cpp index 99f1fbc0082..90f6e420aef 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -85,6 +85,11 @@ void CollectJoinOnKeysMatcher::Data::asofToJoinKeys() addJoinKeys(asof_left_key, asof_right_key, {JoinIdentifierPos::Left, JoinIdentifierPos::Right}); } +void CollectJoinOnKeysMatcher::Data::optimize() +{ + analyzed_join.optimizeDisjuncts(); +} + void CollectJoinOnKeysMatcher::visit(const ASTIdentifier & ident, const ASTPtr & ast, CollectJoinOnKeysMatcher::Data & data) { if (auto expr_from_table = getTableForIdentifiers(ast, false, data); expr_from_table != JoinIdentifierPos::Unknown) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index 61e526b3b4d..c2e1acdde11 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -53,6 +53,7 @@ public: const ASOF::Inequality & asof_inequality); void setDisjuncts(const ASTPtr & or_func_ast); void asofToJoinKeys(); + void optimize(); }; static void visit(const ASTPtr & ast, Data & data) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index ce6103ec87d..94d4555affd 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -141,6 +141,74 @@ void TableJoin::addDisjunct(const ASTPtr & ast) throw Exception("StorageJoin with ORs is not supported", ErrorCodes::NOT_IMPLEMENTED); } +namespace +{ + +bool operator==(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) +{ + return l.key_names_left == r.key_names_left && l.key_names_right == r.key_names_right; +} + +bool operator!=(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) +{ + return !(l == r); +} + +TableJoin::JoinOnClause & operator+=(TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) +{ + for (auto & cp : + {std::pair(&l.on_filter_condition_left, &r.on_filter_condition_left), + std::pair(&l.on_filter_condition_right, &r.on_filter_condition_right) }) + { + if (*cp.first == nullptr) + *cp.first = *cp.second; + else if (const auto * func = (*cp.first)->as(); func && func->name == "or") + /// already have `or` in condition, just add new argument + func->arguments->children.push_back(*cp.second); + else + /// already have some conditions, unite it with `or` + *cp.first = makeASTFunction("or", *cp.first, *cp.second); + } + + return l; +} +} + +void TableJoin::optimizeDisjuncts() +{ + if (clauses.size() > 1) + { + std::sort(clauses.begin(), clauses.end(), [](const JoinOnClause & a, const JoinOnClause & b) { + return a.key_names_left < b.key_names_left || (a.key_names_left == b.key_names_left && a.key_names_left < b.key_names_left); }); + + auto to_it = clauses.begin(); + auto from_it = to_it + 1; + + for (; from_it != clauses.end(); ++from_it) + { + if (*from_it != *to_it) + { + if (++to_it != from_it) + { + *to_it = std::move(*from_it); + } + } + else + { + /// ORing filter conditions + *to_it += *from_it; + } + } + const Clauses::size_type new_size = std::distance(clauses.begin(), to_it) + 1; + if (clauses.size() != new_size) + { + LOG_TRACE( + &Poco::Logger::get("TableJoin"), "optimizeDisjuncts trim clauses, new size is {}", new_size); + clauses.resize(new_size); + } + } +} + /// remember OR's children void TableJoin::setDisjuncts(Disjuncts&& disjuncts_) { diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index d83050d9243..a9c2f1c3134 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -12,6 +12,10 @@ #include #include #include +#include + +#include +#include #include #include @@ -48,6 +52,7 @@ class TableJoin public: using NameToTypeMap = std::unordered_map; + using Disjuncts = ASTs; /// Corresponds to one disjunct struct JoinOnClause @@ -84,6 +89,8 @@ public: } }; + using Clauses = std::vector; + private: /** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k` * The join is made by column k. @@ -110,11 +117,13 @@ private: const size_t max_files_to_merge = 0; const String temporary_files_codec = "LZ4"; - std::vector clauses; - ASTs key_asts_left; ASTs key_asts_right; + Disjuncts disjuncts; + + Clauses clauses; + ASTTableJoin table_join; ASOF::Inequality asof_inequality = ASOF::Inequality::GreaterOrEquals; @@ -152,6 +161,12 @@ private: void assertHasOneOnExpr() const; + /// Calculates common supertypes for corresponding join key columns. + template + bool inferJoinKeyCommonType(const LeftNamesAndTypes & left, const RightNamesAndTypes & right, bool allow_right); + + NamesAndTypesList correctedColumnsAddedByJoin() const; + public: TableJoin() : clauses(1) {} @@ -210,6 +225,14 @@ public: void addUsingKey(const ASTPtr & ast); void setDisjuncts(Disjuncts &&); void addDisjunct(const ASTPtr &); + /// if several disjuncts have exactly the same table columns + /// we can eliminate redundant disjuncts ORing filter conditions + /// This is vital for queries like t1.key = t2.key AND (t1.a = 1 OR t2.bb > 2) + /// to be compartible with merge joins and to create only one hashmap if hashjoin, + /// because after DNFing it is (t1.key = t2.key AND t1.a = 1) OR (t1.key = t2.key AND t2.bb > 2) + /// and we unable to proceed with mergejoin and have to do deal with extra hashmap for hashjoin. + /// Practically we revert DNFing in this case. + void optimizeDisjuncts(); void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast); /* Conditions for left/right table from JOIN ON section. diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 0d15819f69f..76696d8c0a1 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -506,6 +506,8 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul out_table_join = table_join; } +/// Convert to Disjunctive Normal Form https://en.wikipedia.org/wiki/Disjunctive_normal_form +/// based on sample https://github.com/ilejn/ndf class DNF { bool node_added = false; @@ -711,6 +713,7 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_ ErrorCodes::INVALID_JOIN_ON_EXPRESSION); } } + data.optimize(); if (is_asof) { From 4c043a01573d665378f9a26cdd17945a59b04a4d Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 15 Sep 2021 14:17:17 +0300 Subject: [PATCH 283/317] necessary test changes for optimizeDisjuncts in ORs in JOIN --- tests/queries/0_stateless/01429_join_on_error_messages.sql | 6 ++++-- .../queries/0_stateless/01669_join_or_duplicates.reference | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01429_join_on_error_messages.sql b/tests/queries/0_stateless/01429_join_on_error_messages.sql index edc6cdcb617..c6bfcd7af38 100644 --- a/tests/queries/0_stateless/01429_join_on_error_messages.sql +++ b/tests/queries/0_stateless/01429_join_on_error_messages.sql @@ -10,9 +10,11 @@ SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a >= b; -- { serve SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b AND a <= b; -- { serverError 403 } SET join_algorithm = 'partial_merge'; -SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b OR a = b; -- { serverError 48 } +SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError 48 } +-- works for a = b OR a = b because of equivalent disjunct optimization SET join_algorithm = 'auto'; -SELECT 1 FROM (select 1 a) A JOIN (select 1 b) B ON a = b OR a = b; -- { serverError 48 } +SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError 48 } +-- works for a = b OR a = b because of equivalent disjunct optimization SET join_algorithm = 'hash'; diff --git a/tests/queries/0_stateless/01669_join_or_duplicates.reference b/tests/queries/0_stateless/01669_join_or_duplicates.reference index 3837112c486..a2132c25c19 100644 --- a/tests/queries/0_stateless/01669_join_or_duplicates.reference +++ b/tests/queries/0_stateless/01669_join_or_duplicates.reference @@ -107,8 +107,8 @@ 555 a 0 a 0 a 0 a 1 -a 1 a 1 a 1 a 0 +a 1 a 1 a 0 a 0 a 0 a 1 a 1 a 0 From 8057e052a6477b736b302f33e6603ceadeb95903 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 16 Sep 2021 00:51:15 +0300 Subject: [PATCH 284/317] crash fix, style fixes, ASTs moved out of TableJoin in ORs in JOIN --- src/Interpreters/CollectJoinOnKeysVisitor.cpp | 16 ++++-- src/Interpreters/CollectJoinOnKeysVisitor.h | 7 ++- src/Interpreters/HashJoin.cpp | 2 - src/Interpreters/TableJoin.cpp | 52 ++++++++----------- src/Interpreters/TableJoin.h | 9 ++-- ...1881_join_on_conditions_merge.reference.j2 | 1 + .../01881_join_on_conditions_merge.sql.j2 | 2 +- 7 files changed, 46 insertions(+), 43 deletions(-) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/src/Interpreters/CollectJoinOnKeysVisitor.cpp index 90f6e420aef..d558a3c8f0f 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -36,9 +36,17 @@ void CollectJoinOnKeysMatcher::Data::setDisjuncts(const ASTPtr & or_func_ast) { const auto * func = or_func_ast->as(); const auto * func_args = func->arguments->as(); - TableJoin::Disjuncts v = func_args->children; + ASTs v = func_args->children; - analyzed_join.setDisjuncts(std::move(v)); + disjuncts = std::move(v); +} + +void CollectJoinOnKeysMatcher::Data::addDisjunct(const ASTPtr & ast) +{ + const IAST * addr = ast.get(); + + if (std::find_if(disjuncts.begin(), disjuncts.end(), [addr](const ASTPtr & ast_){return ast_.get() == addr;}) != disjuncts.end()) + analyzed_join.newClauseIfPopulated(); } void CollectJoinOnKeysMatcher::Data::addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos) @@ -87,7 +95,7 @@ void CollectJoinOnKeysMatcher::Data::asofToJoinKeys() void CollectJoinOnKeysMatcher::Data::optimize() { - analyzed_join.optimizeDisjuncts(); + analyzed_join.optimizeClauses(); } void CollectJoinOnKeysMatcher::visit(const ASTIdentifier & ident, const ASTPtr & ast, CollectJoinOnKeysMatcher::Data & data) @@ -107,7 +115,7 @@ void CollectJoinOnKeysMatcher::visit(const ASTFunction & func, const ASTPtr & as return; } - data.analyzed_join.addDisjunct(ast); + data.addDisjunct(ast); if (func.name == "and") return; /// go into children diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index c2e1acdde11..04cd341e228 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -47,12 +47,17 @@ public: const bool is_asof{false}; ASTPtr asof_left_key{}; ASTPtr asof_right_key{}; + ASTs disjuncts{}; void addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos); void addAsofJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos, const ASOF::Inequality & asof_inequality); - void setDisjuncts(const ASTPtr & or_func_ast); void asofToJoinKeys(); + + /// remember OR's children + void setDisjuncts(const ASTPtr & or_func_ast); + /// create new disjunct when see a direct child of a previously discovered OR + void addDisjunct(const ASTPtr & ast); void optimize(); }; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 6baf7cda4f8..6f788cdc8bb 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -3,8 +3,6 @@ #include #include -#include - #include #include #include diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 94d4555affd..7b19065fe6c 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -1,6 +1,5 @@ #include - #include #include @@ -23,6 +22,7 @@ #include #include +#include namespace DB @@ -32,6 +32,7 @@ namespace ErrorCodes { extern const int TYPE_MISMATCH; extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; } namespace @@ -123,19 +124,13 @@ void TableJoin::addUsingKey(const ASTPtr & ast) addKey(ast->getColumnName(), renamedRightColumnName(ast->getAliasOrColumnName()), ast); } -/// create new disjunct when see a direct child of a previously discovered OR -void TableJoin::addDisjunct(const ASTPtr & ast) +void TableJoin::newClauseIfPopulated() { - const IAST * addr = ast.get(); - - if (std::find_if(disjuncts.begin(), disjuncts.end(), [addr](const ASTPtr & ast_){return ast_.get() == addr;}) != disjuncts.end()) + const auto & clause = clauses.back(); + if (!clause.key_names_left.empty() || !clause.key_names_right.empty() || + clause.on_filter_condition_left || clause.on_filter_condition_right) { - const auto & clause = clauses.back(); - if (!clause.key_names_left.empty() || !clause.key_names_right.empty() || - clause.on_filter_condition_left || clause.on_filter_condition_right) - { - clauses.emplace_back(); - } + clauses.emplace_back(); } if (getStorageJoin() && clauses.size() > 1) throw Exception("StorageJoin with ORs is not supported", ErrorCodes::NOT_IMPLEMENTED); @@ -149,11 +144,6 @@ bool operator==(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause return l.key_names_left == r.key_names_left && l.key_names_right == r.key_names_right; } -bool operator!=(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) -{ - return !(l == r); -} - TableJoin::JoinOnClause & operator+=(TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) { for (auto & cp : @@ -163,23 +153,33 @@ TableJoin::JoinOnClause & operator+=(TableJoin::JoinOnClause & l, const TableJoi if (*cp.first == nullptr) *cp.first = *cp.second; else if (const auto * func = (*cp.first)->as(); func && func->name == "or") + { /// already have `or` in condition, just add new argument - func->arguments->children.push_back(*cp.second); + if (*cp.second != nullptr) + func->arguments->children.push_back(*cp.second); + } else + { /// already have some conditions, unite it with `or` - *cp.first = makeASTFunction("or", *cp.first, *cp.second); + if (*cp.second != nullptr) + *cp.first = makeASTFunction("or", *cp.first, *cp.second); + } } return l; } } -void TableJoin::optimizeDisjuncts() +bool operator<(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) +{ + return l.key_names_left < r.key_names_left || (l.key_names_left == r.key_names_left && l.key_names_right < r.key_names_right); +} + +void TableJoin::optimizeClauses() { if (clauses.size() > 1) { - std::sort(clauses.begin(), clauses.end(), [](const JoinOnClause & a, const JoinOnClause & b) { - return a.key_names_left < b.key_names_left || (a.key_names_left == b.key_names_left && a.key_names_left < b.key_names_left); }); + std::sort(std::begin(clauses), std::end(clauses)); auto to_it = clauses.begin(); auto from_it = to_it + 1; @@ -203,18 +203,12 @@ void TableJoin::optimizeDisjuncts() if (clauses.size() != new_size) { LOG_TRACE( - &Poco::Logger::get("TableJoin"), "optimizeDisjuncts trim clauses, new size is {}", new_size); + &Poco::Logger::get("TableJoin"), "optimizeClauses trim clauses, size {} => {}", clauses.size(), new_size); clauses.resize(new_size); } } } -/// remember OR's children -void TableJoin::setDisjuncts(Disjuncts&& disjuncts_) -{ - disjuncts = std::move(disjuncts_); -} - void TableJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast) { addKey(left_table_ast->getColumnName(), right_table_ast->getAliasOrColumnName(), left_table_ast, right_table_ast); diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index a9c2f1c3134..655f8a90297 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -52,7 +52,6 @@ class TableJoin public: using NameToTypeMap = std::unordered_map; - using Disjuncts = ASTs; /// Corresponds to one disjunct struct JoinOnClause @@ -120,8 +119,6 @@ private: ASTs key_asts_left; ASTs key_asts_right; - Disjuncts disjuncts; - Clauses clauses; ASTTableJoin table_join; @@ -223,8 +220,8 @@ public: void resetCollected(); void addUsingKey(const ASTPtr & ast); - void setDisjuncts(Disjuncts &&); - void addDisjunct(const ASTPtr &); + void newClauseIfPopulated(); + /// if several disjuncts have exactly the same table columns /// we can eliminate redundant disjuncts ORing filter conditions /// This is vital for queries like t1.key = t2.key AND (t1.a = 1 OR t2.bb > 2) @@ -232,7 +229,7 @@ public: /// because after DNFing it is (t1.key = t2.key AND t1.a = 1) OR (t1.key = t2.key AND t2.bb > 2) /// and we unable to proceed with mergejoin and have to do deal with extra hashmap for hashjoin. /// Practically we revert DNFing in this case. - void optimizeDisjuncts(); + void optimizeClauses(); void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast); /* Conditions for left/right table from JOIN ON section. diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 index fdc9ecfbbef..040b4a059ff 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 @@ -23,6 +23,7 @@ 1 1 1 +1 -- 2 2 diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 index ba0f644504d..ec12833fd99 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 @@ -36,7 +36,7 @@ SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id == 3; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.key2 == 'BBB'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333'; -- { serverError 48 } +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_lc as t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND isNull(t2.key2); SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND t1.key2 like '33%'; From fa6c2a64e4ddf0b11cd0199eac577902bfdb6a84 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Thu, 16 Sep 2021 18:06:30 +0300 Subject: [PATCH 285/317] minor merge mistakes fixed in ORs in JOIN --- src/Interpreters/HashJoin.cpp | 26 ++++---------------------- src/Interpreters/TableJoin.cpp | 20 +++++++++----------- 2 files changed, 13 insertions(+), 33 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 6f788cdc8bb..f7c4108747c 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -3,6 +3,8 @@ #include #include +#include + #include #include #include @@ -24,14 +26,8 @@ #include #include -#include -#include #include #include -#include "Columns/IColumn.h" - -#include - namespace DB { @@ -232,19 +228,6 @@ static ColumnWithTypeAndName correctNullability(ColumnWithTypeAndName && column, return std::move(column); } -//static std::string formatKeysDebug(const std::vector & onexprs) -//{ -// std::vector res; -// for (const auto & onexpr : onexprs) -// { -// std::vector current; -// for (size_t i = 0; i < onexpr.keysCount(); ++i) -// current.emplace_back(fmt::format("{} == {}", onexpr.key_names_left[i], onexpr.key_names_right[i])); -// res.emplace_back(fmt::format("{}", fmt::join(current, ", "))); -// } -// return fmt::format("{}", fmt::join(res, " | ")); -//} - HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_sample_block_, bool any_take_last_row_) : table_join(table_join_) , kind(table_join->kind()) @@ -295,7 +278,6 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s if (nullable_right_side) JoinCommon::convertColumnsToNullable(sample_block_with_columns_to_add); - size_t disjuncts_num = table_join->getClauses().size(); data->maps.resize(disjuncts_num); key_sizes.reserve(disjuncts_num); @@ -1775,7 +1757,7 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) joinBlockImpl(block, sample_block_with_columns_to_add, maps_vector); else - throw throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind); } else { @@ -1792,7 +1774,7 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) /// Joined } else - throw throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind); } } diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 7b19065fe6c..9ce3a676abe 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -52,7 +52,6 @@ std::string formatTypeMap(const TableJoin::NameToTypeMap & target, const TableJo } - namespace { @@ -146,9 +145,8 @@ bool operator==(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause TableJoin::JoinOnClause & operator+=(TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) { - for (auto & cp : - {std::pair(&l.on_filter_condition_left, &r.on_filter_condition_left), - std::pair(&l.on_filter_condition_right, &r.on_filter_condition_right) }) + for (const auto & cp : {std::pair(&l.on_filter_condition_left, &r.on_filter_condition_left), + std::pair(&l.on_filter_condition_right, &r.on_filter_condition_right) }) { if (*cp.first == nullptr) *cp.first = *cp.second; @@ -541,7 +539,8 @@ TableJoin::createConvertingActions(const ColumnsWithTypeAndName & left_sample_co return true; }); - return {left_converting_actions, right_converting_actions};} + return {left_converting_actions, right_converting_actions}; +} template bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const RightNamesAndTypes & right, bool allow_right) @@ -581,12 +580,11 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig } catch (DB::Exception & ex) { - throw Exception( - "Type mismatch of columns to JOIN by: " + - left_key_name + ": " + ltype->second->getName() + " at left, " + - right_key_name + ": " + rtype->second->getName() + " at right. " + - "Can't get supertype: " + ex.message(), - ErrorCodes::TYPE_MISMATCH); + throw DB::Exception(ErrorCodes::TYPE_MISMATCH, + "Can't infer common type for joined columns: {}: {} at left, {}: {} at right. {}", + left_key_name, ltype->second->getName(), + right_key_name, rtype->second->getName(), + ex.message()); } if (!allow_right && !common_type->equals(*rtype->second)) { From 78ad6bf529592b1201b5573238d7c0f0ede54cec Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 17 Sep 2021 16:51:44 +0300 Subject: [PATCH 286/317] MAX_ORS, checkStackSize and beautification per review in ORs in JOIN --- src/Interpreters/CollectJoinOnKeysVisitor.cpp | 8 ++++ src/Interpreters/CollectJoinOnKeysVisitor.h | 5 +++ src/Interpreters/TableJoin.cpp | 44 +++++++++---------- src/Interpreters/TableJoin.h | 2 +- src/Interpreters/TreeRewriter.cpp | 6 +++ .../01429_join_on_error_messages.sql | 5 +++ .../0_stateless/01661_join_complex.sql | 2 + 7 files changed, 49 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/src/Interpreters/CollectJoinOnKeysVisitor.cpp index d558a3c8f0f..15ac43b164c 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -46,7 +46,15 @@ void CollectJoinOnKeysMatcher::Data::addDisjunct(const ASTPtr & ast) const IAST * addr = ast.get(); if (std::find_if(disjuncts.begin(), disjuncts.end(), [addr](const ASTPtr & ast_){return ast_.get() == addr;}) != disjuncts.end()) + { + if (num_of_ors++ > MAX_ORS) + { + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "Maximum allowed number of ORs in JOIN ON section is {}", + MAX_ORS); + } analyzed_join.newClauseIfPopulated(); + } } void CollectJoinOnKeysMatcher::Data::addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index 04cd341e228..56b68552558 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -48,6 +48,11 @@ public: ASTPtr asof_left_key{}; ASTPtr asof_right_key{}; ASTs disjuncts{}; + size_t num_of_ors{0}; + + /// the limit has no technical reasons, it supposed to improve safety + static constexpr std::size_t MAX_ORS = 16; + void addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos); void addAsofJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos, diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 9ce3a676abe..b8ee01cd87f 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -138,33 +138,34 @@ void TableJoin::newClauseIfPopulated() namespace { -bool operator==(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) +bool equalTableColumns(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) { return l.key_names_left == r.key_names_left && l.key_names_right == r.key_names_right; } -TableJoin::JoinOnClause & operator+=(TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) +void joinASTbyOR(ASTPtr & to, const ASTPtr & from) { - for (const auto & cp : {std::pair(&l.on_filter_condition_left, &r.on_filter_condition_left), - std::pair(&l.on_filter_condition_right, &r.on_filter_condition_right) }) + if (from == nullptr) + return; + if (to == nullptr) + to = from; + else if (const auto * func = (to)->as(); func && func->name == "or") { - if (*cp.first == nullptr) - *cp.first = *cp.second; - else if (const auto * func = (*cp.first)->as(); func && func->name == "or") - { - /// already have `or` in condition, just add new argument - if (*cp.second != nullptr) - func->arguments->children.push_back(*cp.second); - } - else - { - /// already have some conditions, unite it with `or` - if (*cp.second != nullptr) - *cp.first = makeASTFunction("or", *cp.first, *cp.second); - } + /// already have `or` in condition, just add new argument + func->arguments->children.push_back(from); } + else + { + /// already have some conditions, unite it with `or` + to = makeASTFunction("or", to, from); + } +} - return l; +/// from's conditions added to to's ones +void addConditionsToClause(TableJoin::JoinOnClause & to, const TableJoin::JoinOnClause & from) +{ + joinASTbyOR(to.on_filter_condition_left, from.on_filter_condition_left); + joinASTbyOR(to.on_filter_condition_right, from.on_filter_condition_right); } } @@ -184,7 +185,7 @@ void TableJoin::optimizeClauses() for (; from_it != clauses.end(); ++from_it) { - if (*from_it != *to_it) + if (! equalTableColumns(*from_it,*to_it)) { if (++to_it != from_it) { @@ -193,8 +194,7 @@ void TableJoin::optimizeClauses() } else { - /// ORing filter conditions - *to_it += *from_it; + addConditionsToClause(*to_it, *from_it); } } const Clauses::size_type new_size = std::distance(clauses.begin(), to_it) + 1; diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 655f8a90297..1bd0fa812a2 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -227,7 +227,7 @@ public: /// This is vital for queries like t1.key = t2.key AND (t1.a = 1 OR t2.bb > 2) /// to be compartible with merge joins and to create only one hashmap if hashjoin, /// because after DNFing it is (t1.key = t2.key AND t1.a = 1) OR (t1.key = t2.key AND t2.bb > 2) - /// and we unable to proceed with mergejoin and have to do deal with extra hashmap for hashjoin. + /// and we unable to proceed with mergejoin and have to deal with extra hashmap for hashjoin. /// Practically we revert DNFing in this case. void optimizeClauses(); void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 76696d8c0a1..9a86fa1dcd2 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1,6 +1,8 @@ #include #include +#include + #include #include #include @@ -626,6 +628,7 @@ class DNF makeASTFunction("or", lst) : lst[0]; + checkStackSize(); node_added = true; return ret; @@ -886,6 +889,9 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (required.count(name)) { /// Optimisation: do not add columns needed only in JOIN ON section. + /// Does not work well if AST was altered, the problem here is DNFing applied to table_join.on_query, + /// not to original query, so calculation of 'how many times a column occurred in a query' is not affected by DNFing, + /// but calculation of 'how many times it occurred in join' is affected. if (columns_context.converted_to_dnf || columns_context.nameInclusion(name) > analyzed_join->rightKeyInclusion(name)) analyzed_join->addJoinedColumn(joined_column); diff --git a/tests/queries/0_stateless/01429_join_on_error_messages.sql b/tests/queries/0_stateless/01429_join_on_error_messages.sql index c6bfcd7af38..44c321ca741 100644 --- a/tests/queries/0_stateless/01429_join_on_error_messages.sql +++ b/tests/queries/0_stateless/01429_join_on_error_messages.sql @@ -17,4 +17,9 @@ SET join_algorithm = 'auto'; SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError 48 } -- works for a = b OR a = b because of equivalent disjunct optimization +SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, 1 bbbb, 1 bbbbb) B ON a = b OR a = bb OR a = bbb OR a = bbbb OR aa = b OR aa = bb OR aa = bbb OR aa = bbbb OR aaa = b OR aaa = bb OR aaa = bbb OR aaa = bbbb OR aaaa = b OR aaaa = bb OR aaaa = bbb OR aaaa = bbbb OR a = bbbbb; -- { serverError 48 } + + SET join_algorithm = 'hash'; + +SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, 1 bbbb, 1 bbbbb) B ON a = b OR a = bb OR a = bbb OR a = bbbb OR aa = b OR aa = bb OR aa = bbb OR aa = bbbb OR aaa = b OR aaa = bb OR aaa = bbb OR aaa = bbbb OR aaaa = b OR aaaa = bb OR aaaa = bbb OR aaaa = bbbb OR a = bbbbb OR aa = bbbbb; -- { serverError 403 } diff --git a/tests/queries/0_stateless/01661_join_complex.sql b/tests/queries/0_stateless/01661_join_complex.sql index 309f8fe41e7..e76e77d7e09 100644 --- a/tests/queries/0_stateless/01661_join_complex.sql +++ b/tests/queries/0_stateless/01661_join_complex.sql @@ -4,6 +4,8 @@ select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 right join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or t1.e = t2.e; select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) and (t1.e = t2.e or t1.f=t2.f); +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b and t1.e = t2.e or t1.d = t2.b and t1.f=t2.f or t1.c = t2.b and t1.e = t2.e or t1.c = t2.b and t1.f=t2.f); +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (or(t1.d = t2.b and t1.e = t2.e, t1.d = t2.b and t1.f=t2.f, t1.c = t2.b and t1.e = t2.e, t1.c = t2.b and t1.f=t2.f)); SET joined_subquery_requires_alias = 0; From 6daef66364aeddabe508fe14d239cd3f46791596 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 20 Sep 2021 00:41:11 +0300 Subject: [PATCH 287/317] MAX_DISJUNCTS instead of MAX_ORS in ORs in JOIN --- src/Interpreters/CollectJoinOnKeysVisitor.cpp | 8 -------- src/Interpreters/CollectJoinOnKeysVisitor.h | 4 ---- src/Interpreters/TableJoin.cpp | 8 ++++++++ src/Interpreters/TableJoin.h | 3 +++ .../0_stateless/00854_multiple_join_asterisks.reference | 1 + .../queries/0_stateless/00854_multiple_join_asterisks.sql | 1 + .../queries/0_stateless/01429_join_on_error_messages.sql | 2 +- .../01881_join_on_conditions_hash.reference.j2 | 1 + .../0_stateless/01881_join_on_conditions_hash.sql.j2 | 2 +- 9 files changed, 16 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/src/Interpreters/CollectJoinOnKeysVisitor.cpp index 15ac43b164c..d558a3c8f0f 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -46,15 +46,7 @@ void CollectJoinOnKeysMatcher::Data::addDisjunct(const ASTPtr & ast) const IAST * addr = ast.get(); if (std::find_if(disjuncts.begin(), disjuncts.end(), [addr](const ASTPtr & ast_){return ast_.get() == addr;}) != disjuncts.end()) - { - if (num_of_ors++ > MAX_ORS) - { - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "Maximum allowed number of ORs in JOIN ON section is {}", - MAX_ORS); - } analyzed_join.newClauseIfPopulated(); - } } void CollectJoinOnKeysMatcher::Data::addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index 56b68552558..6eb7f7b5cd8 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -50,10 +50,6 @@ public: ASTs disjuncts{}; size_t num_of_ors{0}; - /// the limit has no technical reasons, it supposed to improve safety - static constexpr std::size_t MAX_ORS = 16; - - void addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos); void addAsofJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos, const ASOF::Inequality & asof_inequality); diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index b8ee01cd87f..0dfe4bd29f8 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -30,6 +30,7 @@ namespace DB namespace ErrorCodes { + extern const int INVALID_JOIN_ON_EXPRESSION; extern const int TYPE_MISMATCH; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; @@ -204,6 +205,13 @@ void TableJoin::optimizeClauses() &Poco::Logger::get("TableJoin"), "optimizeClauses trim clauses, size {} => {}", clauses.size(), new_size); clauses.resize(new_size); } + + if (clauses.size() > MAX_DISJUNCTS) + { + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "Maximum number of keys that join tables via OR is {} (after normalization), consider reducing", + MAX_DISJUNCTS); + } } } diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 1bd0fa812a2..72842e654c8 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -116,6 +116,9 @@ private: const size_t max_files_to_merge = 0; const String temporary_files_codec = "LZ4"; + /// the limit has no technical reasons, it supposed to improve safety + const size_t MAX_DISJUNCTS = 16; + ASTs key_asts_left; ASTs key_asts_right; diff --git a/tests/queries/0_stateless/00854_multiple_join_asterisks.reference b/tests/queries/0_stateless/00854_multiple_join_asterisks.reference index 4500bdba755..99243e57760 100644 --- a/tests/queries/0_stateless/00854_multiple_join_asterisks.reference +++ b/tests/queries/0_stateless/00854_multiple_join_asterisks.reference @@ -3,4 +3,5 @@ 0 0 0 0 0 0 +0 0 0 0 0 0 0 diff --git a/tests/queries/0_stateless/00854_multiple_join_asterisks.sql b/tests/queries/0_stateless/00854_multiple_join_asterisks.sql index c01ea721000..6794f178ff9 100644 --- a/tests/queries/0_stateless/00854_multiple_join_asterisks.sql +++ b/tests/queries/0_stateless/00854_multiple_join_asterisks.sql @@ -3,6 +3,7 @@ select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy join syste select t1.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; select t2.*, t3.* from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; select t1.dummy, t2.*, t3.dummy from system.one t1 join system.one t2 on t1.dummy = t2.dummy join system.one t3 ON t1.dummy = t3.dummy; +select t1.dummy, t2.*, t3.dummy from system.one t1 join (select dummy,0 as another_one from system.one) t2 on t1.dummy = t2.dummy OR t1.dummy = t2.another_one join system.one t3 ON t1.dummy = t3.dummy; select t1.dummy, t2.*, t3.dummy from (select * from system.one) t1 join system.one t2 on t1.dummy = t2.dummy diff --git a/tests/queries/0_stateless/01429_join_on_error_messages.sql b/tests/queries/0_stateless/01429_join_on_error_messages.sql index 44c321ca741..7ea2c21885e 100644 --- a/tests/queries/0_stateless/01429_join_on_error_messages.sql +++ b/tests/queries/0_stateless/01429_join_on_error_messages.sql @@ -17,7 +17,7 @@ SET join_algorithm = 'auto'; SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError 48 } -- works for a = b OR a = b because of equivalent disjunct optimization -SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, 1 bbbb, 1 bbbbb) B ON a = b OR a = bb OR a = bbb OR a = bbbb OR aa = b OR aa = bb OR aa = bbb OR aa = bbbb OR aaa = b OR aaa = bb OR aaa = bbb OR aaa = bbbb OR aaaa = b OR aaaa = bb OR aaaa = bbb OR aaaa = bbbb OR a = bbbbb; -- { serverError 48 } +SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, 1 bbbb, 1 bbbbb) B ON a = b OR a = bb OR a = bbb OR a = bbbb OR aa = b OR aa = bb OR aa = bbb OR aa = bbbb OR aaa = b OR aaa = bb OR aaa = bbb OR aaa = bbbb OR aaaa = b OR aaaa = bb OR aaaa = bbb OR aaaa = bbbb; -- { serverError 48 } SET join_algorithm = 'hash'; diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 index 94b3e19483f..14622b95fa4 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 @@ -70,4 +70,5 @@ t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA {% endfor -%} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index 9230db75c03..bcc8e6f8b7c 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -86,7 +86,7 @@ SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2); SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; - +SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); {% endfor -%} DROP TABLE IF EXISTS t1; From aa4751a63213da6221e3f5e7257407d47ceb3098 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 20 Sep 2021 14:36:37 +0300 Subject: [PATCH 288/317] checkStackSize moved to the top of DNF::distributed in ORs in JOIN --- src/Interpreters/TreeRewriter.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 9a86fa1dcd2..c426d39ccfc 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -555,6 +555,8 @@ class DNF ASTPtr distribute(ASTPtr node) { + checkStackSize(); + const auto * function = node->as(); if (function && function->children.size() == 1) @@ -628,7 +630,6 @@ class DNF makeASTFunction("or", lst) : lst[0]; - checkStackSize(); node_added = true; return ret; From 29b911f86b793d23d9c805ebf77df7d9a9ed43a4 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 20 Sep 2021 22:30:34 +0300 Subject: [PATCH 289/317] rebase collisions fixed in ORs in JOIN --- src/Interpreters/HashJoin.cpp | 5 ++++- src/Interpreters/MergeJoin.cpp | 3 ++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index f7c4108747c..07872df8ce5 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1700,7 +1700,10 @@ ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block void HashJoin::checkTypesOfKeys(const Block & block) const { - JoinCommon::checkTypesOfKeys(block, table_join->keyNamesLeft(), right_table_keys, key_names_right); + for (const auto & onexpr : table_join->getClauses()) + { + JoinCommon::checkTypesOfKeys(block, onexpr.key_names_left, right_table_keys, onexpr.key_names_right); + } } void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 09f309603c3..9cfc43b92c5 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -674,7 +674,8 @@ bool MergeJoin::addJoinedBlock(const Block & src_block, bool) void MergeJoin::checkTypesOfKeys(const Block & block) const { /// Do not check auxailary column for extra conditions, use original key names - JoinCommon::checkTypesOfKeys(block, table_join->keyNamesLeft(), right_table_keys, table_join->keyNamesRight()); + const auto & onexpr = table_join->getOnlyClause(); + JoinCommon::checkTypesOfKeys(block, onexpr.key_names_left, right_table_keys, onexpr.key_names_right); } void MergeJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) From 17e6cfbefb802a0dc08074cf896dad9d1927a6bd Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Wed, 22 Sep 2021 23:46:28 +0300 Subject: [PATCH 290/317] DNF bugfix in ORs in JOIN --- src/Interpreters/TreeRewriter.cpp | 9 ++++++++- .../01881_join_on_conditions_hash.reference.j2 | 1 + .../0_stateless/01881_join_on_conditions_hash.sql.j2 | 1 + .../01881_join_on_conditions_merge.reference.j2 | 1 + .../0_stateless/01881_join_on_conditions_merge.sql.j2 | 1 + 5 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index c426d39ccfc..39502398cd1 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -563,12 +563,19 @@ class DNF { if (function->name == "and") { - const auto * func_args = function->arguments->as(); + auto * func_args = function->arguments->as(); if (!func_args) { return node; } + ASTs distr_lst; + for (const auto & arg : func_args->children) + { + distr_lst.push_back(distribute(arg)); + } + func_args->children = distr_lst; + auto or_child = std::find_if(func_args->children.begin(), func_args->children.end(), [](ASTPtr arg) { const auto * f = arg->as(); diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 index 14622b95fa4..170a1c57900 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 @@ -70,5 +70,6 @@ t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA +t22 3 333 333 3 100 BBB BBB t22 1 111 111 2 1 AAA AAA {% endfor -%} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index bcc8e6f8b7c..2433d3a20ef 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -86,6 +86,7 @@ SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2); SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; +SELECT 't22', * FROM t1 INNER JOIN t22 ON t1.id == t22.id AND (toUInt8(t1.key) > 222 OR toUInt8(t1.key2) >2) AND (t22.idd>3 OR t22.key2='BBB'); SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); {% endfor -%} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 index 040b4a059ff..c69133893ca 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 @@ -57,4 +57,5 @@ -- 2 222 2 2 AAA a 2 222 222 2 AAA AAA +t22 3 333 333 3 100 BBB BBB {% endfor -%} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 index ec12833fd99..5b37edf201a 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 @@ -80,6 +80,7 @@ SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t SELECT '--'; -- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; +SELECT 't22', * FROM t1 INNER JOIN t22 ON t1.id == t22.id AND (toUInt8(t1.key) > 222 OR toUInt8(t1.key2) >2) AND (t22.idd>3 OR t22.key2='BBB'); {% endfor -%} From 760a92c490fab387ecfed72a7ac23aa3715c54d1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 22 Sep 2021 14:19:01 +0300 Subject: [PATCH 291/317] Do not allow in optimizeClauses conditions for different table joined via OR --- src/Interpreters/TableJoin.cpp | 30 ++++++++++++++++--- .../01429_join_on_error_messages.sql | 4 +++ 2 files changed, 30 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 0dfe4bd29f8..95eebab5ed4 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -1,5 +1,7 @@ #include +#include +#include #include #include @@ -148,9 +150,12 @@ void joinASTbyOR(ASTPtr & to, const ASTPtr & from) { if (from == nullptr) return; + if (to == nullptr) + { to = from; - else if (const auto * func = (to)->as(); func && func->name == "or") + } + else if (const auto * func = to->as(); func && func->name == "or") { /// already have `or` in condition, just add new argument func->arguments->children.push_back(from); @@ -165,9 +170,27 @@ void joinASTbyOR(ASTPtr & to, const ASTPtr & from) /// from's conditions added to to's ones void addConditionsToClause(TableJoin::JoinOnClause & to, const TableJoin::JoinOnClause & from) { + bool has_left = to.on_filter_condition_left || from.on_filter_condition_left; + bool has_right = to.on_filter_condition_right || from.on_filter_condition_right; + /// Cannot join conditions for left and right table via OR. + /// Currently all rows that don't hold condition from left(/right) table are filtered + if (has_left && has_right) + { + /// Format for debug + const auto & all_conds = {to.on_filter_condition_left, from.on_filter_condition_left, + to.on_filter_condition_right, from.on_filter_condition_right}; + std::vector conditions_str; + for (const auto & cond : all_conds) + { + if (cond != nullptr) + conditions_str.push_back(queryToString(cond)); + } + throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported JOIN ON conditions: '{}'", fmt::join(conditions_str, " OR ")); + } joinASTbyOR(to.on_filter_condition_left, from.on_filter_condition_left); joinASTbyOR(to.on_filter_condition_right, from.on_filter_condition_right); } + } bool operator<(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) @@ -182,11 +205,10 @@ void TableJoin::optimizeClauses() std::sort(std::begin(clauses), std::end(clauses)); auto to_it = clauses.begin(); - auto from_it = to_it + 1; - for (; from_it != clauses.end(); ++from_it) + for (auto from_it = to_it + 1; from_it != clauses.end(); ++from_it) { - if (! equalTableColumns(*from_it,*to_it)) + if (!equalTableColumns(*from_it, *to_it)) { if (++to_it != from_it) { diff --git a/tests/queries/0_stateless/01429_join_on_error_messages.sql b/tests/queries/0_stateless/01429_join_on_error_messages.sql index 7ea2c21885e..1aa61034db5 100644 --- a/tests/queries/0_stateless/01429_join_on_error_messages.sql +++ b/tests/queries/0_stateless/01429_join_on_error_messages.sql @@ -23,3 +23,7 @@ SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, SET join_algorithm = 'hash'; SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, 1 bbbb, 1 bbbbb) B ON a = b OR a = bb OR a = bbb OR a = bbbb OR aa = b OR aa = bb OR aa = bbb OR aa = bbbb OR aaa = b OR aaa = bb OR aaa = bbb OR aaa = bbbb OR aaaa = b OR aaaa = bb OR aaaa = bbb OR aaaa = bbbb OR a = bbbbb OR aa = bbbbb; -- { serverError 403 } + +-- conditions for different table joined via OR +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); -- { serverError NOT_IMPLEMENTED } +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND t1.b > 0 OR t1.a = t2.a AND t2.b > 0; -- { serverError NOT_IMPLEMENTED } From 336b2a4c683289ca9285ca8425ae63f858d52358 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 24 Sep 2021 19:21:05 +0300 Subject: [PATCH 292/317] bypass filer conditions in DNF in ORs in JOIN (part 1) --- src/Interpreters/CollectJoinOnKeysVisitor.cpp | 25 ------ src/Interpreters/CollectJoinOnKeysVisitor.h | 1 + src/Interpreters/TableJoin.cpp | 12 ++- src/Interpreters/TableJoin.h | 4 +- src/Interpreters/TreeRewriter.cpp | 84 ++++++++++++++++--- .../01881_join_on_conditions_merge.sql.j2 | 4 +- 6 files changed, 90 insertions(+), 40 deletions(-) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/src/Interpreters/CollectJoinOnKeysVisitor.cpp index d558a3c8f0f..af8ce3538e9 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -32,23 +32,6 @@ bool isRightIdentifier(JoinIdentifierPos pos) } -void CollectJoinOnKeysMatcher::Data::setDisjuncts(const ASTPtr & or_func_ast) -{ - const auto * func = or_func_ast->as(); - const auto * func_args = func->arguments->as(); - ASTs v = func_args->children; - - disjuncts = std::move(v); -} - -void CollectJoinOnKeysMatcher::Data::addDisjunct(const ASTPtr & ast) -{ - const IAST * addr = ast.get(); - - if (std::find_if(disjuncts.begin(), disjuncts.end(), [addr](const ASTPtr & ast_){return ast_.get() == addr;}) != disjuncts.end()) - analyzed_join.newClauseIfPopulated(); -} - void CollectJoinOnKeysMatcher::Data::addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos) { ASTPtr left = left_ast->clone(); @@ -109,14 +92,6 @@ void CollectJoinOnKeysMatcher::visit(const ASTIdentifier & ident, const ASTPtr & void CollectJoinOnKeysMatcher::visit(const ASTFunction & func, const ASTPtr & ast, Data & data) { - if (func.name == "or") - { - data.setDisjuncts(ast); - return; - } - - data.addDisjunct(ast); - if (func.name == "and") return; /// go into children diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index 6eb7f7b5cd8..d4078d1803a 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -92,6 +92,7 @@ private: static void getIdentifiers(const ASTPtr & ast, std::vector & out); static JoinIdentifierPosPair getTableNumbers(const ASTPtr & left_ast, const ASTPtr & right_ast, Data & data); static const ASTIdentifier * unrollAliases(const ASTIdentifier * identifier, const Aliases & aliases); +public: static JoinIdentifierPos getTableForIdentifiers(const ASTPtr & ast, bool throw_on_table_mix, const Data & data); }; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 95eebab5ed4..389f3e11c72 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -111,8 +111,7 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) void TableJoin::resetCollected() { - clauses = std::vector(1); - + clauses.clear(); columns_from_joined_table.clear(); columns_added_by_join.clear(); original_names.clear(); @@ -237,6 +236,14 @@ void TableJoin::optimizeClauses() } } +void TableJoin::addDisjunct() +{ + clauses.emplace_back(); + + if (getStorageJoin() && clauses.size() > 1) + throw Exception("StorageJoin with ORs is not supported", ErrorCodes::NOT_IMPLEMENTED); +} + void TableJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast) { addKey(left_table_ast->getColumnName(), right_table_ast->getAliasOrColumnName(), left_table_ast, right_table_ast); @@ -616,6 +623,7 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig right_key_name, rtype->second->getName(), ex.message()); } + if (!allow_right && !common_type->equals(*rtype->second)) { throw DB::Exception(ErrorCodes::TYPE_MISMATCH, diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 72842e654c8..5262ab87c9d 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -168,8 +168,7 @@ private: NamesAndTypesList correctedColumnsAddedByJoin() const; public: - TableJoin() : clauses(1) - {} + TableJoin() = default; TableJoin(const Settings & settings, VolumePtr tmp_volume_); @@ -233,6 +232,7 @@ public: /// and we unable to proceed with mergejoin and have to deal with extra hashmap for hashjoin. /// Practically we revert DNFing in this case. void optimizeClauses(); + void addDisjunct(); void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast); /* Conditions for left/right table from JOIN ON section. diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 39502398cd1..024b30fe89d 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -508,11 +508,14 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul out_table_join = table_join; } + +#if 0 /// Convert to Disjunctive Normal Form https://en.wikipedia.org/wiki/Disjunctive_normal_form /// based on sample https://github.com/ilejn/ndf class DNF { bool node_added = false; + const CollectJoinOnKeysVisitor::Data & data; void normTree(ASTPtr node) { @@ -576,10 +579,33 @@ class DNF } func_args->children = distr_lst; - auto or_child = std::find_if(func_args->children.begin(), func_args->children.end(), [](ASTPtr arg) + auto or_child = std::find_if(func_args->children.begin(), func_args->children.end(), [this](ASTPtr arg) { const auto * f = arg->as(); - return f && f->name == "or" && f->children.size() == 1; + if (f && f->name == "or" && f->children.size() == 1) + { + + auto pos = CollectJoinOnKeysMatcher::getTableForIdentifiers(std::make_shared(*f), false /* throw_on_table_mix */, data)/* == JoinIdentifierPos::Unknown */; + switch (pos) + { + case JoinIdentifierPos::Unknown: + LOG_TRACE(&Poco::Logger::get("distribute"), "Unknown"); + break; + case JoinIdentifierPos::Left: + LOG_TRACE(&Poco::Logger::get("distribute"), "Left"); + break; + case JoinIdentifierPos::Right: + LOG_TRACE(&Poco::Logger::get("distribute"), "Right"); + break; + case JoinIdentifierPos::NotApplicable: + LOG_TRACE(&Poco::Logger::get("distribute"), "NotApplicable"); + break; + } + + return pos != JoinIdentifierPos::Left && pos != JoinIdentifierPos::Right; + } + return false; + }); if (or_child == func_args->children.end()) { @@ -668,6 +694,10 @@ class DNF public: + DNF(const CollectJoinOnKeysVisitor::Data & data_) + : data(data_) + { + } void process(const ASTSelectQuery & select_query, const TablesWithColumns & tables) { @@ -696,7 +726,7 @@ public: table_join.converted_to_dnf = node_added; } }; - +#endif /// Find the columns that are obtained by JOIN. void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_join, @@ -707,6 +737,8 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_ if (table_join.using_expression_list) { const auto & keys = table_join.using_expression_list->as(); + + analyzed_join.addDisjunct(); for (const auto & key : keys.children) analyzed_join.addUsingKey(key); } @@ -715,14 +747,33 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_ bool is_asof = (table_join.strictness == ASTTableJoin::Strictness::Asof); CollectJoinOnKeysVisitor::Data data{analyzed_join, tables[0], tables[1], aliases, is_asof}; - CollectJoinOnKeysVisitor(data).visit(table_join.on_expression); + if (auto * or_func = table_join.on_expression->as(); or_func && or_func->name == "or") + { + for (auto & disjunct : or_func->arguments->children) + { + analyzed_join.addDisjunct(); + CollectJoinOnKeysVisitor(data).visit(disjunct); + } + assert(analyzed_join.getClauses().size() == or_func->arguments->children.size()); + } + else + { + analyzed_join.addDisjunct(); + CollectJoinOnKeysVisitor(data).visit(table_join.on_expression); + assert(analyzed_join.oneDisjunct()); + } + + if (analyzed_join.getClauses().empty()) + throw DB::Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "Cannot get JOIN keys from JOIN ON section: '{}'", + queryToString(table_join.on_expression)); + for (const auto & onexpr : analyzed_join.getClauses()) { if (onexpr.key_names_left.empty()) - { - throw Exception("Cannot get JOIN keys from JOIN ON section: " + queryToString(table_join.on_expression), - ErrorCodes::INVALID_JOIN_ON_EXPRESSION); - } + throw DB::Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "Cannot get JOIN keys from JOIN ON section: '{}'", + queryToString(table_join.on_expression)); } data.optimize(); @@ -1138,12 +1189,25 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( { for (const auto & [name, _] : table_join->columns_from_joined_table) all_source_columns_set.insert(name); - DNF().process(*select_query, tables_with_columns); + // DNF().process(*select_query, tables_with_columns); } normalize(query, result.aliases, all_source_columns_set, select_options.ignore_alias, settings, /* allow_self_aliases = */ true); - /// Remove unneeded columns according to 'required_result_columns'. +#if 0 + if (table_join) + { + auto * table_join_ast = select_query->join() ? select_query->join()->table_join->as() : nullptr; + if (table_join_ast) + { + CollectJoinOnKeysVisitor::Data data{*result.analyzed_join, tables_with_columns[0], tables_with_columns[1], result.aliases, table_join_ast->strictness == ASTTableJoin::Strictness::Asof}; + DNF(data).process(*select_query, tables_with_columns); + } + } +#endif + + +/// Remove unneeded columns according to 'required_result_columns'. /// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside. /// Must be after 'normalizeTree' (after expanding aliases, for aliases not get lost) /// and before 'executeScalarSubqueries', 'analyzeAggregation', etc. to avoid excessive calculations. diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 index 5b37edf201a..b347ba2981e 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 @@ -36,7 +36,6 @@ SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id == 3; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.key2 == 'BBB'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_lc as t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND isNull(t2.key2); SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND t1.key2 like '33%'; @@ -77,6 +76,9 @@ SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333'; -- { serverError 48 }} + + SELECT '--'; -- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; From bbd548e81d132d4844b011a816e63a8dbf63e00b Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Sat, 25 Sep 2021 01:43:13 +0300 Subject: [PATCH 293/317] bypass filer conditions in DNF in ORs in JOIN (part 2) --- src/Interpreters/CollectJoinOnKeysVisitor.cpp | 5 --- src/Interpreters/CollectJoinOnKeysVisitor.h | 14 ++------ src/Interpreters/TableJoin.h | 8 ++--- src/Interpreters/TreeRewriter.cpp | 36 +++++-------------- ...01881_join_on_conditions_hash.reference.j2 | 9 +++++ .../01881_join_on_conditions_hash.sql.j2 | 9 +++++ ...1881_join_on_conditions_merge.reference.j2 | 3 ++ .../01881_join_on_conditions_merge.sql.j2 | 6 ++-- 8 files changed, 39 insertions(+), 51 deletions(-) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.cpp b/src/Interpreters/CollectJoinOnKeysVisitor.cpp index af8ce3538e9..9715af01a0a 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.cpp +++ b/src/Interpreters/CollectJoinOnKeysVisitor.cpp @@ -76,11 +76,6 @@ void CollectJoinOnKeysMatcher::Data::asofToJoinKeys() addJoinKeys(asof_left_key, asof_right_key, {JoinIdentifierPos::Left, JoinIdentifierPos::Right}); } -void CollectJoinOnKeysMatcher::Data::optimize() -{ - analyzed_join.optimizeClauses(); -} - void CollectJoinOnKeysMatcher::visit(const ASTIdentifier & ident, const ASTPtr & ast, CollectJoinOnKeysMatcher::Data & data) { if (auto expr_from_table = getTableForIdentifiers(ast, false, data); expr_from_table != JoinIdentifierPos::Unknown) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index d4078d1803a..956f4e35d7e 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -47,19 +47,11 @@ public: const bool is_asof{false}; ASTPtr asof_left_key{}; ASTPtr asof_right_key{}; - ASTs disjuncts{}; - size_t num_of_ors{0}; void addJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos); void addAsofJoinKeys(const ASTPtr & left_ast, const ASTPtr & right_ast, JoinIdentifierPosPair table_pos, const ASOF::Inequality & asof_inequality); void asofToJoinKeys(); - - /// remember OR's children - void setDisjuncts(const ASTPtr & or_func_ast); - /// create new disjunct when see a direct child of a previously discovered OR - void addDisjunct(const ASTPtr & ast); - void optimize(); }; static void visit(const ASTPtr & ast, Data & data) @@ -81,10 +73,11 @@ public: static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { if (auto * func = node->as()) - return func->name == "and" || func->name == "or"; + return func->name == "and"; return true; } + static JoinIdentifierPos getTableForIdentifiers(const ASTPtr & ast, bool throw_on_table_mix, const Data & data); private: static void visit(const ASTFunction & func, const ASTPtr & ast, Data & data); static void visit(const ASTIdentifier & ident, const ASTPtr & ast, Data & data); @@ -92,8 +85,7 @@ private: static void getIdentifiers(const ASTPtr & ast, std::vector & out); static JoinIdentifierPosPair getTableNumbers(const ASTPtr & left_ast, const ASTPtr & right_ast, Data & data); static const ASTIdentifier * unrollAliases(const ASTIdentifier * identifier, const Aliases & aliases); -public: - static JoinIdentifierPos getTableForIdentifiers(const ASTPtr & ast, bool throw_on_table_mix, const Data & data); + }; /// Parse JOIN ON expression and collect ASTs for joined columns. diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 5262ab87c9d..498ae71cf3b 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -226,11 +226,9 @@ public: /// if several disjuncts have exactly the same table columns /// we can eliminate redundant disjuncts ORing filter conditions - /// This is vital for queries like t1.key = t2.key AND (t1.a = 1 OR t2.bb > 2) - /// to be compartible with merge joins and to create only one hashmap if hashjoin, - /// because after DNFing it is (t1.key = t2.key AND t1.a = 1) OR (t1.key = t2.key AND t2.bb > 2) - /// and we unable to proceed with mergejoin and have to deal with extra hashmap for hashjoin. - /// Practically we revert DNFing in this case. + /// This is needed for queries like + /// SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.a = t2.a AND t1.c > 0) OR (t1.a = t2.a AND t1.b > 0); + /// to be compartible with merge joins and to create only one hashmap if hashjoin. void optimizeClauses(); void addDisjunct(); void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 024b30fe89d..b163b83ef92 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -508,8 +508,6 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul out_table_join = table_join; } - -#if 0 /// Convert to Disjunctive Normal Form https://en.wikipedia.org/wiki/Disjunctive_normal_form /// based on sample https://github.com/ilejn/ndf class DNF @@ -584,24 +582,9 @@ class DNF const auto * f = arg->as(); if (f && f->name == "or" && f->children.size() == 1) { - - auto pos = CollectJoinOnKeysMatcher::getTableForIdentifiers(std::make_shared(*f), false /* throw_on_table_mix */, data)/* == JoinIdentifierPos::Unknown */; - switch (pos) - { - case JoinIdentifierPos::Unknown: - LOG_TRACE(&Poco::Logger::get("distribute"), "Unknown"); - break; - case JoinIdentifierPos::Left: - LOG_TRACE(&Poco::Logger::get("distribute"), "Left"); - break; - case JoinIdentifierPos::Right: - LOG_TRACE(&Poco::Logger::get("distribute"), "Right"); - break; - case JoinIdentifierPos::NotApplicable: - LOG_TRACE(&Poco::Logger::get("distribute"), "NotApplicable"); - break; - } - + auto pos = CollectJoinOnKeysMatcher::getTableForIdentifiers(std::make_shared(*f), + false /* throw_on_table_mix */, + data); return pos != JoinIdentifierPos::Left && pos != JoinIdentifierPos::Right; } return false; @@ -726,7 +709,6 @@ public: table_join.converted_to_dnf = node_added; } }; -#endif /// Find the columns that are obtained by JOIN. void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_join, @@ -775,7 +757,7 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_ "Cannot get JOIN keys from JOIN ON section: '{}'", queryToString(table_join.on_expression)); } - data.optimize(); + analyzed_join.optimizeClauses(); if (is_asof) { @@ -1189,23 +1171,23 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( { for (const auto & [name, _] : table_join->columns_from_joined_table) all_source_columns_set.insert(name); - // DNF().process(*select_query, tables_with_columns); } normalize(query, result.aliases, all_source_columns_set, select_options.ignore_alias, settings, /* allow_self_aliases = */ true); -#if 0 if (table_join) { auto * table_join_ast = select_query->join() ? select_query->join()->table_join->as() : nullptr; if (table_join_ast) { - CollectJoinOnKeysVisitor::Data data{*result.analyzed_join, tables_with_columns[0], tables_with_columns[1], result.aliases, table_join_ast->strictness == ASTTableJoin::Strictness::Asof}; + CollectJoinOnKeysVisitor::Data data{*result.analyzed_join, + tables_with_columns[0], + tables_with_columns[1], + result.aliases, + table_join_ast->strictness == ASTTableJoin::Strictness::Asof}; DNF(data).process(*select_query, tables_with_columns); } } -#endif - /// Remove unneeded columns according to 'required_result_columns'. /// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside. diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 index 170a1c57900..3e46b097f4e 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 @@ -72,4 +72,13 @@ t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA t22 3 333 333 3 100 BBB BBB t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA +t22 1 111 111 2 1 AAA AAA {% endfor -%} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index 2433d3a20ef..b3d1671bef7 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -88,6 +88,15 @@ SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; SELECT 't22', * FROM t1 INNER JOIN t22 ON t1.id == t22.id AND (toUInt8(t1.key) > 222 OR toUInt8(t1.key2) >2) AND (t22.idd>3 OR t22.key2='BBB'); SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id); +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2); +SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; +SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; +SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2; +SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd; +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd; {% endfor -%} DROP TABLE IF EXISTS t1; diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 index c69133893ca..0f3c63ca237 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 @@ -54,6 +54,9 @@ 2 222 2 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 2 222 222 2 AAA AAA 3 333 333 3 BBB BBB +2 222 222 2 AAA AAA +2 222 222 2 AAA a +3 333 333 3 BBB BBB -- 2 222 2 2 AAA a 2 222 222 2 AAA AAA diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 index b347ba2981e..9debc9a6a9c 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 @@ -35,6 +35,7 @@ SELECT '--'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id > 2; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id == 3; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.key2 == 'BBB'; +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_lc as t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND isNull(t2.key2); @@ -62,6 +63,7 @@ SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id = SELECT '--'; SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id NULLS FIRST, t2.id NULLS FIRST; +SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND t1.key2 == '333'); SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 } @@ -72,13 +74,11 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverErro SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 48 } +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; -- { serverError 48 } -- non-equi condition containing columns from different tables doesn't supported yet SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333'; -- { serverError 48 }} - - SELECT '--'; -- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; From 626bfdf23ce8a2ce802b5475682c7614dcf9a955 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 27 Sep 2021 00:21:25 +0300 Subject: [PATCH 294/317] compatible filter conditions, fixes and new tests in ORs in JOIN --- src/Interpreters/TableJoin.cpp | 34 ++++++++----------- src/Interpreters/TableJoin.h | 2 +- src/Interpreters/TreeRewriter.cpp | 12 ++++--- .../01429_join_on_error_messages.sql | 4 --- .../0_stateless/01660_join_or_inner.sql | 6 ---- ...01881_join_on_conditions_hash.reference.j2 | 17 ++++++++++ .../01881_join_on_conditions_hash.sql.j2 | 11 ++++++ ...1881_join_on_conditions_merge.reference.j2 | 1 + .../01881_join_on_conditions_merge.sql.j2 | 7 ++++ 9 files changed, 60 insertions(+), 34 deletions(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 389f3e11c72..932bea4c8bf 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -140,9 +140,18 @@ void TableJoin::newClauseIfPopulated() namespace { +bool compatibleFilerConditions(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) +{ + bool has_left = l.on_filter_condition_left || r.on_filter_condition_left; + bool has_right = l.on_filter_condition_right || r.on_filter_condition_right; + return !(has_left && has_right); +} + bool equalTableColumns(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) { - return l.key_names_left == r.key_names_left && l.key_names_right == r.key_names_right; + return l.key_names_left == r.key_names_left && + l.key_names_right == r.key_names_right && + compatibleFilerConditions(l, r); } void joinASTbyOR(ASTPtr & to, const ASTPtr & from) @@ -169,23 +178,8 @@ void joinASTbyOR(ASTPtr & to, const ASTPtr & from) /// from's conditions added to to's ones void addConditionsToClause(TableJoin::JoinOnClause & to, const TableJoin::JoinOnClause & from) { - bool has_left = to.on_filter_condition_left || from.on_filter_condition_left; - bool has_right = to.on_filter_condition_right || from.on_filter_condition_right; - /// Cannot join conditions for left and right table via OR. - /// Currently all rows that don't hold condition from left(/right) table are filtered - if (has_left && has_right) - { - /// Format for debug - const auto & all_conds = {to.on_filter_condition_left, from.on_filter_condition_left, - to.on_filter_condition_right, from.on_filter_condition_right}; - std::vector conditions_str; - for (const auto & cond : all_conds) - { - if (cond != nullptr) - conditions_str.push_back(queryToString(cond)); - } - throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported JOIN ON conditions: '{}'", fmt::join(conditions_str, " OR ")); - } + assert(compatibleFilerConditions(to, from)); + joinASTbyOR(to.on_filter_condition_left, from.on_filter_condition_left); joinASTbyOR(to.on_filter_condition_right, from.on_filter_condition_right); } @@ -194,7 +188,9 @@ void addConditionsToClause(TableJoin::JoinOnClause & to, const TableJoin::JoinOn bool operator<(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) { - return l.key_names_left < r.key_names_left || (l.key_names_left == r.key_names_left && l.key_names_right < r.key_names_right); + return l.key_names_left < r.key_names_left || + (l.key_names_left == r.key_names_left && l.key_names_right < r.key_names_right) || + (l.key_names_left == r.key_names_left && l.key_names_left == r.key_names_left && l.on_filter_condition_left && !r.on_filter_condition_left); } void TableJoin::optimizeClauses() diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 498ae71cf3b..c3e7eb4a520 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -228,7 +228,7 @@ public: /// we can eliminate redundant disjuncts ORing filter conditions /// This is needed for queries like /// SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.a = t2.a AND t1.c > 0) OR (t1.a = t2.a AND t1.b > 0); - /// to be compartible with merge joins and to create only one hashmap if hashjoin. + /// to be compatible with merge joins and to reduce number of hashmaps if hashjoin. void optimizeClauses(); void addDisjunct(); void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index b163b83ef92..f4c2c8c5d59 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -510,6 +510,7 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul /// Convert to Disjunctive Normal Form https://en.wikipedia.org/wiki/Disjunctive_normal_form /// based on sample https://github.com/ilejn/ndf +/// Keep join conditions as is. class DNF { bool node_added = false; @@ -582,9 +583,10 @@ class DNF const auto * f = arg->as(); if (f && f->name == "or" && f->children.size() == 1) { + const bool throw_on_table_mix = false; auto pos = CollectJoinOnKeysMatcher::getTableForIdentifiers(std::make_shared(*f), - false /* throw_on_table_mix */, - data); + throw_on_table_mix, + data); return pos != JoinIdentifierPos::Left && pos != JoinIdentifierPos::Right; } return false; @@ -601,7 +603,8 @@ class DNF { // LOG_DEBUG(&Poco::Logger::get("toDNF"), "IDs {} vs. {}", arg->getTreeHash(), (*or_child)->getTreeHash()); - if (arg->getTreeHash() != (*or_child)->getTreeHash()) + // if (arg->getTreeHash() != (*or_child)->getTreeHash()) + if (arg.get() != (*or_child).get()) { rest_children.push_back(arg); } @@ -677,13 +680,14 @@ class DNF public: - DNF(const CollectJoinOnKeysVisitor::Data & data_) + explicit DNF(const CollectJoinOnKeysVisitor::Data & data_) : data(data_) { } void process(const ASTSelectQuery & select_query, const TablesWithColumns & tables) { + const ASTTablesInSelectQueryElement * node = select_query.join(); if (!node || tables.size() < 2) { diff --git a/tests/queries/0_stateless/01429_join_on_error_messages.sql b/tests/queries/0_stateless/01429_join_on_error_messages.sql index 1aa61034db5..7ea2c21885e 100644 --- a/tests/queries/0_stateless/01429_join_on_error_messages.sql +++ b/tests/queries/0_stateless/01429_join_on_error_messages.sql @@ -23,7 +23,3 @@ SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, SET join_algorithm = 'hash'; SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, 1 bbbb, 1 bbbbb) B ON a = b OR a = bb OR a = bbb OR a = bbbb OR aa = b OR aa = bb OR aa = bbb OR aa = bbbb OR aaa = b OR aaa = bb OR aaa = bbb OR aaa = bbbb OR aaaa = b OR aaaa = bb OR aaaa = bbb OR aaaa = bbbb OR a = bbbbb OR aa = bbbbb; -- { serverError 403 } - --- conditions for different table joined via OR -SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); -- { serverError NOT_IMPLEMENTED } -SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND t1.b > 0 OR t1.a = t2.a AND t2.b > 0; -- { serverError NOT_IMPLEMENTED } diff --git a/tests/queries/0_stateless/01660_join_or_inner.sql b/tests/queries/0_stateless/01660_join_or_inner.sql index 9b93940631d..60c04033e5f 100644 --- a/tests/queries/0_stateless/01660_join_or_inner.sql +++ b/tests/queries/0_stateless/01660_join_or_inner.sql @@ -1,15 +1,11 @@ SET joined_subquery_requires_alias = 0; -drop table if exists tab1; drop table if exists tab2; drop table if exists tab3; -create table tab1 (a1 Int32, b1 Int32) engine = MergeTree order by a1; create table tab2 (a2 Int32, b2 Int32) engine = MergeTree order by a2; create table tab3 (a3 Int32, b3 Int32) engine = MergeTree order by a3; -insert into tab1 values (1, 2); - insert into tab2 values (2, 3); insert into tab2 values (6, 4); insert into tab2 values (998, 999); @@ -33,7 +29,5 @@ select tab2.*, tab3.* from tab2 any join tab3 on a2 = a3 or b2 = b3; select '=='; select tab2.*, tab3.* from tab2 any join tab3 on b2 = b3 or a2 = a3; - -drop table tab1; drop table tab2; drop table tab3; diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 index 3e46b097f4e..186c28de0c2 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 @@ -81,4 +81,21 @@ t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA +1 1 1 1 1 1 +1 1 1 1 1 1 +2 222 2 2 AAA AAA +2 222 222 2 AAA AAA +2 222 222 2 AAA a +3 333 333 3 BBB BBB +2 222 2 2 AAA AAA +2 222 222 2 AAA AAA +2 222 222 2 AAA a +2 222 222 2 AAA AAA +2 222 222 2 AAA a +3 333 333 3 BBB BBB +2 222 2 2 AAA AAA +2 222 2 2 AAA a +2 222 222 2 AAA AAA +2 222 222 2 AAA a +3 333 333 3 BBB BBB {% endfor -%} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index b3d1671bef7..40321cf33fc 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -97,6 +97,17 @@ SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2; SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd; SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd; + +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND t1.b > 0 OR t1.a = t2.a AND t2.b > 0; +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); + +SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND (t2.key2='AAA' OR t2.key2='BBB')); + +SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND (t2.key2='AAA' OR t1.key2='BBB')); + +select t1.*, t2.* from t1 inner join t2 on t1.id = t2.id AND (((toInt32(t1.key) > 222 OR toInt32(t1.key2) > 5) AND (length(t2.key) > 0 OR length(t2.key2) > 0))); + +select t1.*, t2.* from t1 inner join t2 on t1.id = t2.id AND (((toInt32(t1.key) > 222 OR toInt32(t1.key2) > 5) AND (length(t2.key) > 0 OR length(t2.key2) > 0)) OR t1.key = '222'); {% endfor -%} DROP TABLE IF EXISTS t1; diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 index 0f3c63ca237..1e8aa6af213 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 @@ -61,4 +61,5 @@ 2 222 2 2 AAA a 2 222 222 2 AAA AAA t22 3 333 333 3 100 BBB BBB +1 1 1 1 1 1 {% endfor -%} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 index 9debc9a6a9c..b16f1a79f6d 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 @@ -84,7 +84,14 @@ SELECT '--'; SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; SELECT 't22', * FROM t1 INNER JOIN t22 ON t1.id == t22.id AND (toUInt8(t1.key) > 222 OR toUInt8(t1.key2) >2) AND (t22.idd>3 OR t22.key2='BBB'); +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND t1.b > 0 OR t1.a = t2.a AND t2.b > 0; -- { serverError 48 } +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); -- { serverError 48 } +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON (t1.a = t2.a AND t1.b > 0) OR (t1.a = t2.a AND t1.c > 0); + +SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND (t2.key2='AAA' OR t2.key2='BBB')); -- { serverError 48 } + +SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND (t2.key2='AAA' OR t1.key2='BBB')); -- { serverError 48 } {% endfor -%} DROP TABLE IF EXISTS t1; From 1dc7fc5962caa298c0c731abafa587ac6ed5e712 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 27 Sep 2021 10:17:41 +0300 Subject: [PATCH 295/317] fix bug found by PVS in ORs in JOIN --- src/Interpreters/TableJoin.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 932bea4c8bf..b4f232326fa 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -190,7 +190,7 @@ bool operator<(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause { return l.key_names_left < r.key_names_left || (l.key_names_left == r.key_names_left && l.key_names_right < r.key_names_right) || - (l.key_names_left == r.key_names_left && l.key_names_left == r.key_names_left && l.on_filter_condition_left && !r.on_filter_condition_left); + (l.key_names_left == r.key_names_left && l.key_names_right == r.key_names_right && l.on_filter_condition_left && !r.on_filter_condition_left); } void TableJoin::optimizeClauses() From 7ebc16c1b3d6428e38861d9348f770a76e4f95c5 Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 27 Sep 2021 15:57:26 +0300 Subject: [PATCH 296/317] get rid of DNF and related features in ORs in JOIN --- src/Interpreters/CollectJoinOnKeysVisitor.h | 3 +- src/Interpreters/RequiredSourceColumnsData.h | 1 - .../RequiredSourceColumnsVisitor.cpp | 8 +- src/Interpreters/TableJoin.cpp | 108 --------- src/Interpreters/TableJoin.h | 8 +- src/Interpreters/TreeRewriter.cpp | 228 +----------------- src/Parsers/ASTTablesInSelectQuery.h | 2 - .../01429_join_on_error_messages.sql | 6 +- .../0_stateless/01660_join_or_inner.sql | 2 + .../0_stateless/01661_join_complex.sql | 19 +- .../0_stateless/01669_join_or_duplicates.sql | 4 +- ...01881_join_on_conditions_hash.reference.j2 | 28 --- .../01881_join_on_conditions_hash.sql.j2 | 29 +-- ...1881_join_on_conditions_merge.reference.j2 | 5 - .../01881_join_on_conditions_merge.sql.j2 | 13 +- 15 files changed, 32 insertions(+), 432 deletions(-) diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index 956f4e35d7e..0647f58f79b 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -77,7 +77,6 @@ public: return true; } - static JoinIdentifierPos getTableForIdentifiers(const ASTPtr & ast, bool throw_on_table_mix, const Data & data); private: static void visit(const ASTFunction & func, const ASTPtr & ast, Data & data); static void visit(const ASTIdentifier & ident, const ASTPtr & ast, Data & data); @@ -85,7 +84,7 @@ private: static void getIdentifiers(const ASTPtr & ast, std::vector & out); static JoinIdentifierPosPair getTableNumbers(const ASTPtr & left_ast, const ASTPtr & right_ast, Data & data); static const ASTIdentifier * unrollAliases(const ASTIdentifier * identifier, const Aliases & aliases); - + static JoinIdentifierPos getTableForIdentifiers(const ASTPtr & ast, bool throw_on_table_mix, const Data & data); }; /// Parse JOIN ON expression and collect ASTs for joined columns. diff --git a/src/Interpreters/RequiredSourceColumnsData.h b/src/Interpreters/RequiredSourceColumnsData.h index 6b2e514ff94..d425989393e 100644 --- a/src/Interpreters/RequiredSourceColumnsData.h +++ b/src/Interpreters/RequiredSourceColumnsData.h @@ -36,7 +36,6 @@ struct RequiredSourceColumnsData bool has_table_join = false; bool has_array_join = false; - bool converted_to_dnf = false; bool addColumnAliasIfAny(const IAST & ast); void addColumnIdentifier(const ASTIdentifier & node); diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index 85bd8d1b70e..21ec94a6917 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -178,14 +178,8 @@ void RequiredSourceColumnsMatcher::visit(const ASTFunction & node, const ASTPtr void RequiredSourceColumnsMatcher::visit(const ASTTablesInSelectQueryElement & node, const ASTPtr &, Data & data) { for (const auto & child : node.children) - { - const auto * join_node = child->as(); - if (join_node) - { + if (child->as()) data.has_table_join = true; - data.converted_to_dnf = join_node->converted_to_dnf; - } - } } /// ASTIdentifiers here are tables. Do not visit them as generic ones. diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index b4f232326fa..154a344410c 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -125,113 +125,6 @@ void TableJoin::addUsingKey(const ASTPtr & ast) addKey(ast->getColumnName(), renamedRightColumnName(ast->getAliasOrColumnName()), ast); } -void TableJoin::newClauseIfPopulated() -{ - const auto & clause = clauses.back(); - if (!clause.key_names_left.empty() || !clause.key_names_right.empty() || - clause.on_filter_condition_left || clause.on_filter_condition_right) - { - clauses.emplace_back(); - } - if (getStorageJoin() && clauses.size() > 1) - throw Exception("StorageJoin with ORs is not supported", ErrorCodes::NOT_IMPLEMENTED); -} - -namespace -{ - -bool compatibleFilerConditions(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) -{ - bool has_left = l.on_filter_condition_left || r.on_filter_condition_left; - bool has_right = l.on_filter_condition_right || r.on_filter_condition_right; - return !(has_left && has_right); -} - -bool equalTableColumns(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) -{ - return l.key_names_left == r.key_names_left && - l.key_names_right == r.key_names_right && - compatibleFilerConditions(l, r); -} - -void joinASTbyOR(ASTPtr & to, const ASTPtr & from) -{ - if (from == nullptr) - return; - - if (to == nullptr) - { - to = from; - } - else if (const auto * func = to->as(); func && func->name == "or") - { - /// already have `or` in condition, just add new argument - func->arguments->children.push_back(from); - } - else - { - /// already have some conditions, unite it with `or` - to = makeASTFunction("or", to, from); - } -} - -/// from's conditions added to to's ones -void addConditionsToClause(TableJoin::JoinOnClause & to, const TableJoin::JoinOnClause & from) -{ - assert(compatibleFilerConditions(to, from)); - - joinASTbyOR(to.on_filter_condition_left, from.on_filter_condition_left); - joinASTbyOR(to.on_filter_condition_right, from.on_filter_condition_right); -} - -} - -bool operator<(const TableJoin::JoinOnClause & l, const TableJoin::JoinOnClause & r) -{ - return l.key_names_left < r.key_names_left || - (l.key_names_left == r.key_names_left && l.key_names_right < r.key_names_right) || - (l.key_names_left == r.key_names_left && l.key_names_right == r.key_names_right && l.on_filter_condition_left && !r.on_filter_condition_left); -} - -void TableJoin::optimizeClauses() -{ - if (clauses.size() > 1) - { - std::sort(std::begin(clauses), std::end(clauses)); - - auto to_it = clauses.begin(); - - for (auto from_it = to_it + 1; from_it != clauses.end(); ++from_it) - { - if (!equalTableColumns(*from_it, *to_it)) - { - if (++to_it != from_it) - { - *to_it = std::move(*from_it); - } - } - else - { - addConditionsToClause(*to_it, *from_it); - } - } - const Clauses::size_type new_size = std::distance(clauses.begin(), to_it) + 1; - if (clauses.size() != new_size) - { - LOG_TRACE( - &Poco::Logger::get("TableJoin"), "optimizeClauses trim clauses, size {} => {}", clauses.size(), new_size); - clauses.resize(new_size); - } - - if (clauses.size() > MAX_DISJUNCTS) - { - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "Maximum number of keys that join tables via OR is {} (after normalization), consider reducing", - MAX_DISJUNCTS); - } - } -} - void TableJoin::addDisjunct() { clauses.emplace_back(); @@ -619,7 +512,6 @@ bool TableJoin::inferJoinKeyCommonType(const LeftNamesAndTypes & left, const Rig right_key_name, rtype->second->getName(), ex.message()); } - if (!allow_right && !common_type->equals(*rtype->second)) { throw DB::Exception(ErrorCodes::TYPE_MISMATCH, diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index c3e7eb4a520..a207c519439 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -222,15 +222,9 @@ public: void resetCollected(); void addUsingKey(const ASTPtr & ast); - void newClauseIfPopulated(); - /// if several disjuncts have exactly the same table columns - /// we can eliminate redundant disjuncts ORing filter conditions - /// This is needed for queries like - /// SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.a = t2.a AND t1.c > 0) OR (t1.a = t2.a AND t1.b > 0); - /// to be compatible with merge joins and to reduce number of hashmaps if hashjoin. - void optimizeClauses(); void addDisjunct(); + void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast); /* Conditions for left/right table from JOIN ON section. diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index f4c2c8c5d59..7f7603e9a25 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -508,212 +508,6 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul out_table_join = table_join; } -/// Convert to Disjunctive Normal Form https://en.wikipedia.org/wiki/Disjunctive_normal_form -/// based on sample https://github.com/ilejn/ndf -/// Keep join conditions as is. -class DNF -{ - bool node_added = false; - const CollectJoinOnKeysVisitor::Data & data; - - void normTree(ASTPtr node) - { - auto * func = node->as(); - if (func && func->children.size() == 1) - { - for (bool touched = true; touched;) - { - touched = false; - - ASTs new_children; - const auto * func_args = func->arguments->as(); - for (const auto & child : func_args->children) - { - auto * child_func = child->as(); - if (child_func && func->children.size() == 1 - && ((func->name == "or" && child_func->name == "or") || (func->name == "and" && child_func->name == "and"))) - { - std::copy(child_func->arguments->children.begin(), - child_func->arguments->children.end(), - std::back_inserter(new_children)); - touched = true; - } - else - { - new_children.push_back(child); - } - } - - func->arguments->children = std::move(new_children); - } - - for (auto & child : func->arguments->children) - { - normTree(child); - } - } - // LOG_TRACE(&Poco::Logger::get("TreeRewrite"), "bottom of normTree: {}", node->dumpTree()); - } - - ASTPtr distribute(ASTPtr node) - { - checkStackSize(); - - const auto * function = node->as(); - - if (function && function->children.size() == 1) - { - if (function->name == "and") - { - auto * func_args = function->arguments->as(); - if (!func_args) - { - return node; - } - - ASTs distr_lst; - for (const auto & arg : func_args->children) - { - distr_lst.push_back(distribute(arg)); - } - func_args->children = distr_lst; - - auto or_child = std::find_if(func_args->children.begin(), func_args->children.end(), [this](ASTPtr arg) - { - const auto * f = arg->as(); - if (f && f->name == "or" && f->children.size() == 1) - { - const bool throw_on_table_mix = false; - auto pos = CollectJoinOnKeysMatcher::getTableForIdentifiers(std::make_shared(*f), - throw_on_table_mix, - data); - return pos != JoinIdentifierPos::Left && pos != JoinIdentifierPos::Right; - } - return false; - - }); - if (or_child == func_args->children.end()) - { - return node; - } - - ASTs rest_children; - - for (const auto & arg : func_args->children) - { - // LOG_DEBUG(&Poco::Logger::get("toDNF"), "IDs {} vs. {}", arg->getTreeHash(), (*or_child)->getTreeHash()); - - // if (arg->getTreeHash() != (*or_child)->getTreeHash()) - if (arg.get() != (*or_child).get()) - { - rest_children.push_back(arg); - } - } - if (rest_children.empty()) - { - return node; - } - - const auto * or_child_function = (*or_child)->as(); - if (!or_child_function) - { - return node; - } - - auto rest = rest_children.size() > 1 ? - makeASTFunction("and", rest_children): - rest_children[0]; - - const auto * or_child_expression_list = or_child_function->children[0]->as(); - assert(or_child_expression_list); - - if (or_child_expression_list) - { - - ASTs lst; - for (const auto & arg : or_child_expression_list->children) - { - ASTs arg_rest_lst; - arg_rest_lst.push_back(arg); - arg_rest_lst.push_back(rest); - - auto and_node = makeASTFunction("and", arg_rest_lst); - lst.push_back(distribute(and_node)); - } - if (lst.empty()) - { - return node; - } - - auto ret = lst.size()>1 ? - makeASTFunction("or", lst) : - lst[0]; - - node_added = true; - - return ret; - } - } - else if (function->name == "or") - { - const auto * expression_list = function->children[0]->as(); - if (!expression_list) - { - return node; - } - - ASTs lst; - for (const auto & arg : expression_list->children) - { - lst.push_back(distribute(arg)); - } - - auto ret = lst.size() > 1 - ? makeASTFunction("or", lst) - : lst[0]; - return ret; - } - } - - return node; - } - - -public: - explicit DNF(const CollectJoinOnKeysVisitor::Data & data_) - : data(data_) - { - } - - void process(const ASTSelectQuery & select_query, const TablesWithColumns & tables) - { - - const ASTTablesInSelectQueryElement * node = select_query.join(); - if (!node || tables.size() < 2) - { - return; - } - - auto & table_join = node->table_join->as(); - if (!table_join.on_expression || table_join.strictness == ASTTableJoin::Strictness::Asof) - { - return; - } - - normTree(table_join.on_expression); - - auto distributed_expression = distribute(table_join.on_expression); - - normTree(distributed_expression); - LOG_TRACE(&Poco::Logger::get("TreeRewrite"), "bottom of toDNF: {}, node_added {}", - distributed_expression->dumpTree(), node_added); - - table_join.on_expression = distributed_expression; - - table_join.converted_to_dnf = node_added; - } -}; - /// Find the columns that are obtained by JOIN. void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_join, const TablesWithColumns & tables, const Aliases & aliases) @@ -761,7 +555,6 @@ void collectJoinedColumns(TableJoin & analyzed_join, const ASTTableJoin & table_ "Cannot get JOIN keys from JOIN ON section: '{}'", queryToString(table_join.on_expression)); } - analyzed_join.optimizeClauses(); if (is_asof) { @@ -934,10 +727,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (required.count(name)) { /// Optimisation: do not add columns needed only in JOIN ON section. - /// Does not work well if AST was altered, the problem here is DNFing applied to table_join.on_query, - /// not to original query, so calculation of 'how many times a column occurred in a query' is not affected by DNFing, - /// but calculation of 'how many times it occurred in join' is affected. - if (columns_context.converted_to_dnf || columns_context.nameInclusion(name) > analyzed_join->rightKeyInclusion(name)) + if (columns_context.nameInclusion(name) > analyzed_join->rightKeyInclusion(name)) analyzed_join->addJoinedColumn(joined_column); required.erase(name); @@ -1179,21 +969,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( normalize(query, result.aliases, all_source_columns_set, select_options.ignore_alias, settings, /* allow_self_aliases = */ true); - if (table_join) - { - auto * table_join_ast = select_query->join() ? select_query->join()->table_join->as() : nullptr; - if (table_join_ast) - { - CollectJoinOnKeysVisitor::Data data{*result.analyzed_join, - tables_with_columns[0], - tables_with_columns[1], - result.aliases, - table_join_ast->strictness == ASTTableJoin::Strictness::Asof}; - DNF(data).process(*select_query, tables_with_columns); - } - } - -/// Remove unneeded columns according to 'required_result_columns'. + /// Remove unneeded columns according to 'required_result_columns'. /// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside. /// Must be after 'normalizeTree' (after expanding aliases, for aliases not get lost) /// and before 'executeScalarSubqueries', 'analyzeAggregation', etc. to avoid excessive calculations. diff --git a/src/Parsers/ASTTablesInSelectQuery.h b/src/Parsers/ASTTablesInSelectQuery.h index 74666e141b1..9d31ca6f653 100644 --- a/src/Parsers/ASTTablesInSelectQuery.h +++ b/src/Parsers/ASTTablesInSelectQuery.h @@ -99,8 +99,6 @@ struct ASTTableJoin : public IAST Strictness strictness = Strictness::Unspecified; Kind kind = Kind::Inner; - bool converted_to_dnf = false; - /// Condition. One of fields is non-nullptr. ASTPtr using_expression_list; ASTPtr on_expression; diff --git a/tests/queries/0_stateless/01429_join_on_error_messages.sql b/tests/queries/0_stateless/01429_join_on_error_messages.sql index 7ea2c21885e..cf9aac0e4da 100644 --- a/tests/queries/0_stateless/01429_join_on_error_messages.sql +++ b/tests/queries/0_stateless/01429_join_on_error_messages.sql @@ -17,9 +17,7 @@ SET join_algorithm = 'auto'; SELECT 1 FROM (select 1 a) A JOIN (select 1 b, 1 c) B ON a = b OR a = c; -- { serverError 48 } -- works for a = b OR a = b because of equivalent disjunct optimization -SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, 1 bbbb, 1 bbbbb) B ON a = b OR a = bb OR a = bbb OR a = bbbb OR aa = b OR aa = bb OR aa = bbb OR aa = bbbb OR aaa = b OR aaa = bb OR aaa = bbb OR aaa = bbbb OR aaaa = b OR aaaa = bb OR aaaa = bbb OR aaaa = bbbb; -- { serverError 48 } - - SET join_algorithm = 'hash'; -SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, 1 bbbb, 1 bbbbb) B ON a = b OR a = bb OR a = bbb OR a = bbbb OR aa = b OR aa = bb OR aa = bbb OR aa = bbbb OR aaa = b OR aaa = bb OR aaa = bbb OR aaa = bbbb OR aaaa = b OR aaaa = bb OR aaaa = bbb OR aaaa = bbbb OR a = bbbbb OR aa = bbbbb; -- { serverError 403 } +-- conditions for different table joined via OR +SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); -- { serverError 403 } diff --git a/tests/queries/0_stateless/01660_join_or_inner.sql b/tests/queries/0_stateless/01660_join_or_inner.sql index 60c04033e5f..b7b15b1a071 100644 --- a/tests/queries/0_stateless/01660_join_or_inner.sql +++ b/tests/queries/0_stateless/01660_join_or_inner.sql @@ -29,5 +29,7 @@ select tab2.*, tab3.* from tab2 any join tab3 on a2 = a3 or b2 = b3; select '=='; select tab2.*, tab3.* from tab2 any join tab3 on b2 = b3 or a2 = a3; +SELECT 1 FROM (select 1 a, 1 aa, 1 aaa, 1 aaaa) A JOIN (select 1 b, 1 bb, 1 bbb, 1 bbbb, 1 bbbbb) B ON a = b OR a = bb OR a = bbb OR a = bbbb OR aa = b OR aa = bb OR aa = bbb OR aa = bbbb OR aaa = b OR aaa = bb OR aaa = bbb OR aaa = bbbb OR aaaa = b OR aaaa = bb OR aaaa = bbb OR aaaa = bbbb OR a = bbbbb OR aa = bbbbb; + drop table tab2; drop table tab3; diff --git a/tests/queries/0_stateless/01661_join_complex.sql b/tests/queries/0_stateless/01661_join_complex.sql index e76e77d7e09..7ce426dee22 100644 --- a/tests/queries/0_stateless/01661_join_complex.sql +++ b/tests/queries/0_stateless/01661_join_complex.sql @@ -1,12 +1,17 @@ select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.b = t2.b and t1.c = t2.b and t1.d = t2.b or t1.e = t2.e; select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b or t1.e = t2.e and t1.a=t2.a and t2.f=t1.f; -select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 left join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); -select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 right join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or t1.e = t2.e; -select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); -select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) and (t1.e = t2.e or t1.f=t2.f); -select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b and t1.e = t2.e or t1.d = t2.b and t1.f=t2.f or t1.c = t2.b and t1.e = t2.e or t1.c = t2.b and t1.f=t2.f); -select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (or(t1.d = t2.b and t1.e = t2.e, t1.d = t2.b and t1.f=t2.f, t1.c = t2.b and t1.e = t2.e, t1.c = t2.b and t1.f=t2.f)); +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 left join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 right join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b or t1.c = t2.b or t1.e = t2.e; +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b or t1.c = t2.b or t1.e = t2.e and t1.a=t2.a and t2.f=t1.f; +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on t1.d = t2.b AND t1.e = t2.e OR t1.c = t2.b AND t1.e = t2.e OR t1.d = t2.b AND t1.f=t2.f OR t1.c = t2.b AND t1.f=t2.f; +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (or(t1.d = t2.b and t1.e = t2.e, t1.d = t2.b and t1.f=t2.f, t1.c = t2.b and t1.e = t2.e, t1.c = t2.b and t1.f=t2.f)); +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b and t1.e = t2.e or t1.d = t2.b and t1.f=t2.f or t1.c = t2.b and t1.e = t2.e or t1.c = t2.b and t1.f=t2.f); + +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 left join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b or t1.d = t2.b and t1.d = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); -- { serverError INVALID_JOIN_ON_EXPRESSION } +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 right join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or t1.e = t2.e; -- { serverError INVALID_JOIN_ON_EXPRESSION } +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) or (t1.e = t2.e and t1.a=t2.a and t2.f=t1.f); -- { serverError INVALID_JOIN_ON_EXPRESSION } +select * from (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t1 inner join (select 'a' as a, 'b' as b, 'c' as c, 'd' as d, 'e' as e, 'f' as f) as t2 on (t1.d = t2.b or t1.c = t2.b) and (t1.e = t2.e or t1.f=t2.f); -- { serverError INVALID_JOIN_ON_EXPRESSION } SET joined_subquery_requires_alias = 0; SET max_threads = 1; @@ -24,7 +29,7 @@ insert into tab3 values (5, 4); insert into tab3 values (100, 4); select 'join on OR/AND chain'; -select a2, b2, a3, b3 from tab2 any left join tab3 on (a2=a3 or b2=b3) and a2 +1 = b3 + 0; +select a2, b2, a3, b3 from tab2 any left join tab3 on a2=a3 and a2 +1 = b3 + 0 or b2=b3 and a2 +1 = b3 + 0 ; drop table tab2; drop table tab3; diff --git a/tests/queries/0_stateless/01669_join_or_duplicates.sql b/tests/queries/0_stateless/01669_join_or_duplicates.sql index 96e592dd497..7495ecd1dec 100644 --- a/tests/queries/0_stateless/01669_join_or_duplicates.sql +++ b/tests/queries/0_stateless/01669_join_or_duplicates.sql @@ -18,6 +18,6 @@ select '17 full', * from (select 1 as x, 2 as y) t1 full join (select 1 as xx, 2 select count(1) from (select * from (select 1 as x, 2 as y) t1 left join (select 1 as xx, 2 as yy from numbers(555)) t2 on x = xx or y = yy); -select * from (select 'a' as a, number as c from numbers(2)) as t1 join (select 'a' as a, number as c from numbers(2)) as t2 on t1.c=t2.c or t1.a = t2.a; +select * from (select 'a' as a, number as c from numbers(2)) as t1 join (select 'a' as a, number as c from numbers(2)) as t2 on t1.c = t2.c or t1.a = t2.a order by t1.c, t2.c; -select * from (select 'a' as a, number as c from numbers(2)) as t1 join (select 'a' as a, number as c from numbers(2)) as t2 on t1.a = t2.a or t1.c=t2.c; +select * from (select 'a' as a, number as c from numbers(2)) as t1 join (select 'a' as a, number as c from numbers(2)) as t2 on t1.a = t2.a or t1.c = t2.c order by t1.c, t2.c; diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 index 186c28de0c2..94b3e19483f 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.reference.j2 @@ -70,32 +70,4 @@ t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA t22 1 111 111 2 1 AAA AAA -t22 3 333 333 3 100 BBB BBB -t22 1 111 111 2 1 AAA AAA -t22 1 111 111 2 1 AAA AAA -t22 1 111 111 2 1 AAA AAA -t22 1 111 111 2 1 AAA AAA -t22 1 111 111 2 1 AAA AAA -t22 1 111 111 2 1 AAA AAA -t22 1 111 111 2 1 AAA AAA -t22 1 111 111 2 1 AAA AAA -t22 1 111 111 2 1 AAA AAA -t22 1 111 111 2 1 AAA AAA -1 1 1 1 1 1 -1 1 1 1 1 1 -2 222 2 2 AAA AAA -2 222 222 2 AAA AAA -2 222 222 2 AAA a -3 333 333 3 BBB BBB -2 222 2 2 AAA AAA -2 222 222 2 AAA AAA -2 222 222 2 AAA a -2 222 222 2 AAA AAA -2 222 222 2 AAA a -3 333 333 3 BBB BBB -2 222 2 2 AAA AAA -2 222 2 2 AAA a -2 222 222 2 AAA AAA -2 222 222 2 AAA a -3 333 333 3 BBB BBB {% endfor -%} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 index 40321cf33fc..0d6bef7fadb 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_hash.sql.j2 @@ -73,7 +73,8 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverErro SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; +SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } +SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; -- non-equi condition containing columns from different tables doesn't supported yet SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.id >= length(t2.key); -- { serverError 403 } @@ -82,32 +83,16 @@ SELECT '--'; -- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id); -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2); -SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; -SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -SELECT 't22', * FROM t1 INNER JOIN t22 ON t1.id == t22.id AND (toUInt8(t1.key) > 222 OR toUInt8(t1.key2) >2) AND (t22.idd>3 OR t22.key2='BBB'); -SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id); -SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2); -SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; -SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t22.key == t22.key2 OR t1.id == t22.id); -- { serverError 403 } +SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and (t1.id == t22.id OR t22.key == t22.key2); -- { serverError 403 } +SELECT 't22', * FROM t1 JOIN t22 ON (t22.key == t22.key2 OR t1.id == t22.id) and t1.id == t22.idd; -- { serverError 403 } +SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and t1.id == t22.idd; -- { serverError 403 } +SELECT 't22', * FROM t1 JOIN t22 ON (t1.id == t22.id OR t22.key == t22.key2) and (t1.id == t22.idd AND (t1.key2 = 'a1' OR t1.key2 = 'a2' OR t1.key2 = 'a3' OR t1.key2 = 'a4' OR t1.key2 = 'a5' OR t1.key2 = 'a6' OR t1.key2 = 'a7' OR t1.key2 = 'a8' OR t1.key2 = 'a9' OR t1.key2 = 'a10' OR t1.key2 = 'a11' OR t1.key2 = 'a12' OR t1.key2 = 'a13' OR t1.key2 = 'a14' OR t1.key2 = 'a15' OR t1.key2 = 'a16' OR t1.key2 = 'a17' OR t1.key2 = 'a18' OR t1.key2 = 'a19' OR t1.key2 = '111')); -- { serverError 403 } SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t22.key == t22.key2 OR t1.id == t22.idd and t1.id == t22.id; SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.idd and t1.id == t22.id OR t1.id == t22.idd and t22.key == t22.key2; SELECT 't22', * FROM t1 JOIN t22 ON t22.key == t22.key2 and t1.id == t22.idd OR t1.id == t22.id and t1.id == t22.idd; SELECT 't22', * FROM t1 JOIN t22 ON t1.id == t22.id and t1.id == t22.idd OR t22.key == t22.key2 and t1.id == t22.idd; -SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND t1.b > 0 OR t1.a = t2.a AND t2.b > 0; -SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); - -SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND (t2.key2='AAA' OR t2.key2='BBB')); - -SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND (t2.key2='AAA' OR t1.key2='BBB')); - -select t1.*, t2.* from t1 inner join t2 on t1.id = t2.id AND (((toInt32(t1.key) > 222 OR toInt32(t1.key2) > 5) AND (length(t2.key) > 0 OR length(t2.key2) > 0))); - -select t1.*, t2.* from t1 inner join t2 on t1.id = t2.id AND (((toInt32(t1.key) > 222 OR toInt32(t1.key2) > 5) AND (length(t2.key) > 0 OR length(t2.key2) > 0)) OR t1.key = '222'); {% endfor -%} DROP TABLE IF EXISTS t1; diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 index 1e8aa6af213..040b4a059ff 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.reference.j2 @@ -54,12 +54,7 @@ 2 222 2 {{ jnull(jn, 0) }} {{ jnull(jn) }} {{ jnull(jn) }} 2 222 222 2 AAA AAA 3 333 333 3 BBB BBB -2 222 222 2 AAA AAA -2 222 222 2 AAA a -3 333 333 3 BBB BBB -- 2 222 2 2 AAA a 2 222 222 2 AAA AAA -t22 3 333 333 3 100 BBB BBB -1 1 1 1 1 1 {% endfor -%} diff --git a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 index b16f1a79f6d..a51f4c856f3 100644 --- a/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 +++ b/tests/queries/0_stateless/01881_join_on_conditions_merge.sql.j2 @@ -35,8 +35,8 @@ SELECT '--'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id > 2; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.id == 3; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t2.key2 == 'BBB'; -SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; +SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND (t2.key == t2.key2 OR isNull(t2.key2)) AND t1.key == t1.key2 AND t1.key2 == '333'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_lc as t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key2 == '333'; SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND isNull(t2.key2); SELECT '333' = t1.key FROM t1 INNER ANY JOIN t2_nullable as t2 ON t1.id == t2.id AND t1.key2 like '33%'; @@ -63,7 +63,6 @@ SELECT DISTINCT t1.key, toUInt8(t1.id) as e FROM t1 INNER ANY JOIN t2 ON t1.id = SELECT '--'; SELECT t1.id, t1.key, t1.key2, t2.id, t2.key, t2.key2 FROM t1 FULL JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t1.key == t1.key2 ORDER BY t1.id NULLS FIRST, t2.id NULLS FIRST; -SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND t1.key2 == '333'); SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id; -- { serverError 403 } SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.id; -- { serverError 403 } @@ -73,7 +72,7 @@ SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.key; -- { serverErro SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t2.key; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND (t1.id == t2.id OR isNull(t2.key2)); -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 OR t1.id == t2.id; -- { serverError 403 } -SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 48 } +SELECT * FROM t1 JOIN t2 ON (t2.key == t2.key2 AND (t1.key == t1.key2 AND t1.key != 'XXX' OR t1.id == t2.id)) AND t1.id == t2.id; -- { serverError 403 } SELECT * FROM t1 JOIN t2 ON t2.key == t2.key2 AND t1.key == t1.key2 AND t1.key != 'XXX' AND t1.id == t2.id OR t2.key == t2.key2 AND t1.id == t2.id AND t1.id == t2.id; -- { serverError 48 } -- non-equi condition containing columns from different tables doesn't supported yet SELECT * FROM t1 INNER ALL JOIN t2 ON t1.id == t2.id AND t1.id >= t2.id; -- { serverError 403 } @@ -82,16 +81,8 @@ SELECT * FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND t2.key == t2.key2 AND t SELECT '--'; -- length(t1.key2) == length(t2.key2) is expression for columns from both tables, it works because it part of joining key SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON t1.id == t2.id AND length(t1.key2) == length(t2.key2) AND t1.key != '333'; -SELECT 't22', * FROM t1 INNER JOIN t22 ON t1.id == t22.id AND (toUInt8(t1.key) > 222 OR toUInt8(t1.key2) >2) AND (t22.idd>3 OR t22.key2='BBB'); -SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND t1.b > 0 OR t1.a = t2.a AND t2.b > 0; -- { serverError 48 } -SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0); -- { serverError 48 } -SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON (t1.a = t2.a AND t1.b > 0) OR (t1.a = t2.a AND t1.c > 0); - -SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND (t2.key2='AAA' OR t2.key2='BBB')); -- { serverError 48 } - -SELECT * FROM t1 INNER ALL JOIN t2 ON (t1.id = t2.id AND t1.key2 == '222') OR (t1.id = t2.id AND (t2.key2='AAA' OR t1.key2='BBB')); -- { serverError 48 } {% endfor -%} DROP TABLE IF EXISTS t1; From d67bc0bef446c376b016239c9b0f5c4d844367be Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Mon, 27 Sep 2021 21:35:06 +0300 Subject: [PATCH 297/317] minor fixes in ORs in JOIN --- src/Interpreters/TableJoin.cpp | 1 - tests/queries/0_stateless/01660_join_or_inner.reference | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 154a344410c..db65e074eba 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -32,7 +32,6 @@ namespace DB namespace ErrorCodes { - extern const int INVALID_JOIN_ON_EXPRESSION; extern const int TYPE_MISMATCH; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; diff --git a/tests/queries/0_stateless/01660_join_or_inner.reference b/tests/queries/0_stateless/01660_join_or_inner.reference index eb12265358f..c5aaf095fd9 100644 --- a/tests/queries/0_stateless/01660_join_or_inner.reference +++ b/tests/queries/0_stateless/01660_join_or_inner.reference @@ -10,3 +10,4 @@ any_join_distinct_right_table_keys = 1 == 2 3 2 3 6 4 5 4 +1 From 19d389297b7724e8ff179a849fd324a43fe1ecbd Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 28 Sep 2021 15:07:20 +0300 Subject: [PATCH 298/317] [docs] switch to a new repo for website content (tied to content.clickhouse.com) --- docs/tools/release.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/tools/release.sh b/docs/tools/release.sh index 035581534f1..582d8571388 100755 --- a/docs/tools/release.sh +++ b/docs/tools/release.sh @@ -4,8 +4,8 @@ set -ex BASE_DIR=$(dirname $(readlink -f $0)) BUILD_DIR="${BASE_DIR}/../build" PUBLISH_DIR="${BASE_DIR}/../publish" -BASE_DOMAIN="${BASE_DOMAIN:-content.clickhouse.tech}" -GIT_TEST_URI="${GIT_TEST_URI:-git@github.com:ClickHouse/clickhouse-website-content.git}" +BASE_DOMAIN="${BASE_DOMAIN:-content.clickhouse.com}" +GIT_TEST_URI="${GIT_TEST_URI:-git@github.com:ClickHouse/clickhouse-com-content.git}" GIT_PROD_URI="git@github.com:ClickHouse/clickhouse-website-content.git" EXTRA_BUILD_ARGS="${EXTRA_BUILD_ARGS:---minify --verbose}" From ebe4cb0bbb86024e37db833dff1da1be9b351a6e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 28 Sep 2021 15:32:43 +0300 Subject: [PATCH 299/317] Fix tests. --- src/Processors/Transforms/TotalsHavingTransform.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index 8497b4b0069..c475b87e08f 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -66,9 +66,6 @@ TotalsHavingTransform::TotalsHavingTransform( , auto_include_threshold(auto_include_threshold_) , final(final_) { - if (!filter_column_name.empty()) - filter_column_pos = outputs.front().getHeader().getPositionByName(filter_column_name); - finalized_header = getInputPort().getHeader(); finalizeBlock(finalized_header); @@ -78,12 +75,17 @@ TotalsHavingTransform::TotalsHavingTransform( auto totals_header = finalized_header; size_t num_rows = totals_header.rows(); expression->execute(totals_header, num_rows); + filter_column_pos = totals_header.getPositionByName(filter_column_name); if (remove_filter) totals_header.erase(filter_column_name); outputs.emplace_back(totals_header, this); } else + { + if (!filter_column_name.empty()) + filter_column_pos = finalized_header.getPositionByName(filter_column_name); outputs.emplace_back(finalized_header, this); + } /// Initialize current totals with initial state. current_totals.reserve(header.columns()); @@ -180,11 +182,11 @@ void TotalsHavingTransform::transform(Chunk & chunk) } expression->execute(finalized_block, num_rows); + ColumnPtr filter_column_ptr = finalized_block.getByPosition(filter_column_pos).column; if (remove_filter) finalized_block.erase(filter_column_name); auto columns = finalized_block.getColumns(); - ColumnPtr filter_column_ptr = columns[filter_column_pos]; ConstantFilterDescription const_filter_description(*filter_column_ptr); if (const_filter_description.always_true) From 2c56352b7eecf4d4d7639709f83ef61add9379d7 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 28 Sep 2021 20:16:33 +0300 Subject: [PATCH 300/317] Remove old CLA --- CONTRIBUTING.md | 34 +--------------------------------- 1 file changed, 1 insertion(+), 33 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index e16957b759d..8628d7a01fd 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -6,38 +6,6 @@ Thank you. ## Technical Info -We have a [developer's guide](https://clickhouse.yandex/docs/en/development/developer_instruction/) for writing code for ClickHouse. Besides this guide, you can find [Overview of ClickHouse Architecture](https://clickhouse.yandex/docs/en/development/architecture/) and instructions on how to build ClickHouse in different environments. +We have a [developer's guide](https://clickhouse.com/docs/en/development/developer_instruction/) for writing code for ClickHouse. Besides this guide, you can find [Overview of ClickHouse Architecture](https://clickhouse.com/docs/en/development/architecture/) and instructions on how to build ClickHouse in different environments. If you want to contribute to documentation, read the [Contributing to ClickHouse Documentation](docs/README.md) guide. - -## Legal Info - -In order for us (YANDEX LLC) to accept patches and other contributions from you, you may adopt our Yandex Contributor License Agreement (the "**CLA**"). The current version of the CLA you may find here: -1) https://yandex.ru/legal/cla/?lang=en (in English) and -2) https://yandex.ru/legal/cla/?lang=ru (in Russian). - -By adopting the CLA, you state the following: - -* You obviously wish and are willingly licensing your contributions to us for our open source projects under the terms of the CLA, -* You have read the terms and conditions of the CLA and agree with them in full, -* You are legally able to provide and license your contributions as stated, -* We may use your contributions for our open source projects and for any other our project too, -* We rely on your assurances concerning the rights of third parties in relation to your contributions. - -If you agree with these principles, please read and adopt our CLA. By providing us your contributions, you hereby declare that you have already read and adopt our CLA, and we may freely merge your contributions with our corresponding open source project and use it in further in accordance with terms and conditions of the CLA. - -If you have already adopted terms and conditions of the CLA, you are able to provide your contributes. When you submit your pull request, please add the following information into it: - -``` -I hereby agree to the terms of the CLA available at: [link]. -``` - -Replace the bracketed text as follows: -* [link] is the link at the current version of the CLA (you may add here a link https://yandex.ru/legal/cla/?lang=en (in English) or a link https://yandex.ru/legal/cla/?lang=ru (in Russian). - -It is enough to provide us such notification once. - -As an alternative, you can provide DCO instead of CLA. You can find the text of DCO here: https://developercertificate.org/ -It is enough to read and copy it verbatim to your pull request. - -If you don't agree with the CLA and don't want to provide DCO, you still can open a pull request to provide your contributions. From ee2c12b4abac02fe1b78d9f33d0c9e2e3634925c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 28 Sep 2021 20:47:20 +0300 Subject: [PATCH 301/317] Fix pissble hang in PushingAsyncPipelineExecutor. --- src/Processors/Executors/PushingAsyncPipelineExecutor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp index 08ff6d4d96c..c2b68719202 100644 --- a/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PushingAsyncPipelineExecutor.cpp @@ -41,6 +41,7 @@ public: void finish() { + std::unique_lock lock(mutex); is_finished = true; condvar.notify_all(); } @@ -64,7 +65,7 @@ protected: private: Chunk data; bool has_data = false; - std::atomic_bool is_finished = false; + bool is_finished = false; std::mutex mutex; std::condition_variable condvar; }; From 0e602d85f0e418d741b2cc5e78015f0fd6e78819 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 28 Sep 2021 20:16:32 +0000 Subject: [PATCH 302/317] Done --- src/Parsers/fuzzers/create_parser_fuzzer.cpp | 2 +- src/Parsers/fuzzers/select_parser_fuzzer.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/fuzzers/create_parser_fuzzer.cpp b/src/Parsers/fuzzers/create_parser_fuzzer.cpp index b0adec7d634..032d9ca3ffe 100644 --- a/src/Parsers/fuzzers/create_parser_fuzzer.cpp +++ b/src/Parsers/fuzzers/create_parser_fuzzer.cpp @@ -13,7 +13,7 @@ try std::string input = std::string(reinterpret_cast(data), size); DB::ParserCreateQuery parser; - DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000); DB::WriteBufferFromOwnString wb; DB::formatAST(*ast, wb); diff --git a/src/Parsers/fuzzers/select_parser_fuzzer.cpp b/src/Parsers/fuzzers/select_parser_fuzzer.cpp index bdba552dfa7..caa6c586cd6 100644 --- a/src/Parsers/fuzzers/select_parser_fuzzer.cpp +++ b/src/Parsers/fuzzers/select_parser_fuzzer.cpp @@ -12,7 +12,7 @@ try std::string input = std::string(reinterpret_cast(data), size); DB::ParserQueryWithOutput parser(input.data() + input.size()); - DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0); + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000); DB::WriteBufferFromOwnString wb; DB::formatAST(*ast, wb); From 7a150c6916eaaee54474c59ba79a3ff9840b89fa Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 28 Sep 2021 17:40:50 -0300 Subject: [PATCH 303/317] Update datetime64.md range of precision datetime64 --- docs/en/sql-reference/data-types/datetime64.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/datetime64.md b/docs/en/sql-reference/data-types/datetime64.md index 112461535f7..bac1a080a80 100644 --- a/docs/en/sql-reference/data-types/datetime64.md +++ b/docs/en/sql-reference/data-types/datetime64.md @@ -7,7 +7,8 @@ toc_title: DateTime64 Allows to store an instant in time, that can be expressed as a calendar date and a time of a day, with defined sub-second precision -Tick size (precision): 10-precision seconds +Tick size (precision): 10-precision seconds. Valid range: [ 0 : 9 ]. +Typically are used - 3 (milliseconds), 6 (microseconds), 9 (nanoseconds). **Syntax:** From 7ac7105765696786a08b346257541cd2ace1f705 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 28 Sep 2021 17:43:12 -0300 Subject: [PATCH 304/317] Update datetime64.md --- docs/ru/sql-reference/data-types/datetime64.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/datetime64.md b/docs/ru/sql-reference/data-types/datetime64.md index 73daada3af3..869543dbbaf 100644 --- a/docs/ru/sql-reference/data-types/datetime64.md +++ b/docs/ru/sql-reference/data-types/datetime64.md @@ -7,7 +7,8 @@ toc_title: DateTime64 Позволяет хранить момент времени, который может быть представлен как календарная дата и время, с заданной суб-секундной точностью. -Размер тика (точность, precision): 10-precision секунд, где precision - целочисленный параметр. +Размер тика (точность, precision): 10-precision секунд, где precision - целочисленный параметр. Возможные значения: [ 0 : 9 ]. +Обычно используются - 3 (миллисекунды), 6 (микросекунды), 9 (наносекунды). **Синтаксис:** From 2805c28e6573fab09128b43150c4a7cb7ad21cc1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 29 Sep 2021 02:57:45 +0300 Subject: [PATCH 305/317] Update version_date.tsv after release 21.8.8.29 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 739e0b8edd2..539aa6f1b19 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,6 +1,7 @@ v21.9.4.35-stable 2021-09-22 v21.9.3.30-stable 2021-09-16 v21.9.2.17-stable 2021-09-09 +v21.8.8.29-lts 2021-09-28 v21.8.7.22-lts 2021-09-22 v21.8.6.15-lts 2021-09-16 v21.8.5.7-lts 2021-09-02 From 764b170319ccfa8fb60147b29eef0d91f67a49e7 Mon Sep 17 00:00:00 2001 From: lehasm Date: Wed, 29 Sep 2021 08:22:06 +0300 Subject: [PATCH 306/317] Update docs/ru/sql-reference/functions/index.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/index.md b/docs/ru/sql-reference/functions/index.md index 49af0e14dd2..3352b32764c 100644 --- a/docs/ru/sql-reference/functions/index.md +++ b/docs/ru/sql-reference/functions/index.md @@ -63,7 +63,7 @@ str -> str != Referer Функции можно создавать из лямбда выражений с помощью [CREATE FUNCTION](../statements/create/function.md). Для удаления таких функций используется выражение [DROP FUNCTION](../statements/drop.md#drop-function). ## Исполняемые пользовательские функции {#executable-user-defined-functions} -ClickHouse может вызывать внешнюю программу или скрипт для обработки данных. Такие функции описываются в [конфигурационном файле](../../operations/configuration-files.md). А путь к нему должен быть указан в настройке `user_defined_executable_functions_config` в основной конфигурации. В пути можно использовать символ подстановки `*`, тогда будут загружены все файлы, соответствующие шаблону. Пример: +ClickHouse может вызывать внешнюю программу или скрипт для обработки данных. Такие функции описываются в [конфигурационном файле](../../operations/configuration-files.md). Путь к нему должен быть указан в настройке `user_defined_executable_functions_config` в основной конфигурации. В пути можно использовать символ подстановки `*`, тогда будут загружены все файлы, соответствующие шаблону. Пример: ``` xml *_function.xml ``` From 30eed6138ecfbe52a5f370f7d16af1aab4d3edc1 Mon Sep 17 00:00:00 2001 From: lehasm Date: Wed, 29 Sep 2021 08:24:04 +0300 Subject: [PATCH 307/317] Update docs/ru/sql-reference/functions/index.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/index.md b/docs/ru/sql-reference/functions/index.md index 3352b32764c..016b613e8fb 100644 --- a/docs/ru/sql-reference/functions/index.md +++ b/docs/ru/sql-reference/functions/index.md @@ -77,7 +77,7 @@ ClickHouse может вызывать внешнюю программу или - `format` - [формат](../../interfaces/formats.md), в котором аргументы передаются команде. - `return_type` - тип возвращаемого значения. - `type` - вариант запуска команды. Если задан вариант `executable`, то запускается одна команда. При указании `executable_pool` создается пул команд. -- `max_command_execution_time` - максимальное время в секундах, отводимое на обработку блока данных. Эта настройка применима только для команд с вариантом запуска `executable_pool`. Необязательная настройка. Значение по умолчанию `10`. +- `max_command_execution_time` - максимальное время в секундах, которое отводится на обработку блока данных. Эта настройка применима только для команд с вариантом запуска `executable_pool`. Необязательная настройка. Значение по умолчанию `10`. - `command_termination_timeout` - максимальное время завершения команды в секундах после закрытия конвейера. Если команда не завершается, то процессу отправляется сигнал SIGTERM. Эта настройка применима только для команд с вариантом запуска `executable_pool`. Необязательная настройка. Значение по умолчанию `10`. - `pool_size` - размер пула команд. Необязательная настройка. Значение по умолчанию `16`. - `lifetime` - интервал перезагрузки функций в секундах. Если задан `0`, то функция не перезагружается. From de5d2271e5a63111d7c64796e636544a49aea0bd Mon Sep 17 00:00:00 2001 From: lehasm Date: Wed, 29 Sep 2021 08:24:30 +0300 Subject: [PATCH 308/317] Update docs/ru/sql-reference/functions/index.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/index.md b/docs/ru/sql-reference/functions/index.md index 016b613e8fb..62e7c9a6e21 100644 --- a/docs/ru/sql-reference/functions/index.md +++ b/docs/ru/sql-reference/functions/index.md @@ -87,7 +87,7 @@ ClickHouse может вызывать внешнюю программу или **Пример** -XML конфигурация описывает функцию `test_function`. +XML конфигурация, описывающая функцию `test_function`: ``` From 85826bb014dab432b65a0c2b46022081257ecbd2 Mon Sep 17 00:00:00 2001 From: lehasm Date: Wed, 29 Sep 2021 08:24:56 +0300 Subject: [PATCH 309/317] Update docs/en/sql-reference/functions/index.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/index.md b/docs/en/sql-reference/functions/index.md index 0749fd9e2d7..019218941f3 100644 --- a/docs/en/sql-reference/functions/index.md +++ b/docs/en/sql-reference/functions/index.md @@ -87,7 +87,7 @@ A function configuration contains the following settings: The command must read arguments from STDIN and must output the result to STDOUT. The command must process arguments iteratively. That is after processing a chunk of arguments it must wait for the next chunk. **Example** -The following example creates `test_function` using XML configuration. +Creating `test_function` using XML configuration: ``` From 1d98170f029ff084fefcc11c5aa6f6d51363e638 Mon Sep 17 00:00:00 2001 From: lehasm Date: Wed, 29 Sep 2021 08:25:25 +0300 Subject: [PATCH 310/317] Update docs/ru/sql-reference/functions/index.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/index.md b/docs/ru/sql-reference/functions/index.md index 62e7c9a6e21..dd8224368d3 100644 --- a/docs/ru/sql-reference/functions/index.md +++ b/docs/ru/sql-reference/functions/index.md @@ -83,7 +83,7 @@ ClickHouse может вызывать внешнюю программу или - `lifetime` - интервал перезагрузки функций в секундах. Если задан `0`, то функция не перезагружается. - `send_chunk_header` - управляет отправкой количества строк перед отправкой блока данных для обработки. Необязательная настройка. Значение по умолчанию `false`. -Команд должна читать аргументы из STDIN и выводить результат в STDOUT. Обработка должна выполняться в цикле. То есть после обработки группы аргументов команда должна ожидать следующую группу. +Команда должна читать аргументы из `STDIN` и выводить результат в `STDOUT`. Обработка должна выполняться в цикле. То есть после обработки группы аргументов команда должна ожидать следующую группу. **Пример** From 900dad6d6fe3ce79b1da04e9600eb8867ef15320 Mon Sep 17 00:00:00 2001 From: lehasm Date: Wed, 29 Sep 2021 08:25:44 +0300 Subject: [PATCH 311/317] Update docs/en/sql-reference/functions/index.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/index.md b/docs/en/sql-reference/functions/index.md index 019218941f3..a5a9dd88e2a 100644 --- a/docs/en/sql-reference/functions/index.md +++ b/docs/en/sql-reference/functions/index.md @@ -84,7 +84,7 @@ A function configuration contains the following settings: - `lifetime` - the reload interval of a function in seconds. If it is set to `0` then the function is not reloaded. - `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. -The command must read arguments from STDIN and must output the result to STDOUT. The command must process arguments iteratively. That is after processing a chunk of arguments it must wait for the next chunk. +The command must read arguments from `STDIN` and must output the result to `STDOUT`. The command must process arguments iteratively. That is after processing a chunk of arguments it must wait for the next chunk. **Example** Creating `test_function` using XML configuration: From c8a78fc2b7d5f46bb62f2e312efaa32ccdd9d2e3 Mon Sep 17 00:00:00 2001 From: lehasm Date: Wed, 29 Sep 2021 08:26:12 +0300 Subject: [PATCH 312/317] Update docs/ru/sql-reference/functions/index.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/index.md b/docs/ru/sql-reference/functions/index.md index dd8224368d3..4bdad7e3cac 100644 --- a/docs/ru/sql-reference/functions/index.md +++ b/docs/ru/sql-reference/functions/index.md @@ -78,7 +78,7 @@ ClickHouse может вызывать внешнюю программу или - `return_type` - тип возвращаемого значения. - `type` - вариант запуска команды. Если задан вариант `executable`, то запускается одна команда. При указании `executable_pool` создается пул команд. - `max_command_execution_time` - максимальное время в секундах, которое отводится на обработку блока данных. Эта настройка применима только для команд с вариантом запуска `executable_pool`. Необязательная настройка. Значение по умолчанию `10`. -- `command_termination_timeout` - максимальное время завершения команды в секундах после закрытия конвейера. Если команда не завершается, то процессу отправляется сигнал SIGTERM. Эта настройка применима только для команд с вариантом запуска `executable_pool`. Необязательная настройка. Значение по умолчанию `10`. +- `command_termination_timeout` - максимальное время завершения команды в секундах после закрытия конвейера. Если команда не завершается, то процессу отправляется сигнал `SIGTERM`. Эта настройка применима только для команд с вариантом запуска `executable_pool`. Необязательная настройка. Значение по умолчанию `10`. - `pool_size` - размер пула команд. Необязательная настройка. Значение по умолчанию `16`. - `lifetime` - интервал перезагрузки функций в секундах. Если задан `0`, то функция не перезагружается. - `send_chunk_header` - управляет отправкой количества строк перед отправкой блока данных для обработки. Необязательная настройка. Значение по умолчанию `false`. From ca203c943ec9621d92c00dbfbb8ba0be6f660cd8 Mon Sep 17 00:00:00 2001 From: lehasm Date: Wed, 29 Sep 2021 08:26:32 +0300 Subject: [PATCH 313/317] Update docs/en/sql-reference/functions/index.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/index.md b/docs/en/sql-reference/functions/index.md index a5a9dd88e2a..e86e6b37998 100644 --- a/docs/en/sql-reference/functions/index.md +++ b/docs/en/sql-reference/functions/index.md @@ -79,7 +79,7 @@ A function configuration contains the following settings: - `return_type` - the type of a returned value. - `type` - an executable type. If `type` is set to `executable` then single command is started. If it is set to `executable_pool` then a pool of commands is created. - `max_command_execution_time` - maximum execution time in seconds for processing block of data. This setting is valid for `executable_pool` commands only. Optional. Default value is `10`. -- `command_termination_timeout` - time in seconds during which a command should finish after its pipe is closed. After that time SIGTERM is sent to the process executing the command. This setting is valid for `executable_pool` commands only. Optional. Default value is `10`. +- `command_termination_timeout` - time in seconds during which a command should finish after its pipe is closed. After that time `SIGTERM` is sent to the process executing the command. This setting is valid for `executable_pool` commands only. Optional. Default value is `10`. - `pool_size` - the size of a command pool. Optional. Default value is `16`. - `lifetime` - the reload interval of a function in seconds. If it is set to `0` then the function is not reloaded. - `send_chunk_header` - controls whether to send row count before sending a chunk of data to process. Optional. Default value is `false`. From ff8140b6be0e5c3abda9ba16d40d9fb7c899ce6d Mon Sep 17 00:00:00 2001 From: Alexey Date: Wed, 29 Sep 2021 05:33:16 +0000 Subject: [PATCH 314/317] minor update --- docs/ru/sql-reference/functions/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/index.md b/docs/ru/sql-reference/functions/index.md index 4bdad7e3cac..a63c76d8833 100644 --- a/docs/ru/sql-reference/functions/index.md +++ b/docs/ru/sql-reference/functions/index.md @@ -74,7 +74,7 @@ ClickHouse может вызывать внешнюю программу или - `name` - имя функции. - `command` - исполняемая команда или скрипт. - `argument` - описание аргумента, содержащее его тип во вложенной настройке `type`. Каждый аргумент описывается отдельно. -- `format` - [формат](../../interfaces/formats.md), в котором аргументы передаются команде. +- `format` - [формат](../../interfaces/formats.md) передачи аргументов. - `return_type` - тип возвращаемого значения. - `type` - вариант запуска команды. Если задан вариант `executable`, то запускается одна команда. При указании `executable_pool` создается пул команд. - `max_command_execution_time` - максимальное время в секундах, которое отводится на обработку блока данных. Эта настройка применима только для команд с вариантом запуска `executable_pool`. Необязательная настройка. Значение по умолчанию `10`. From 723ff795a8aaf492ff8b7235bd131a5e72f2ce9a Mon Sep 17 00:00:00 2001 From: MaxWk <610379995@qq.com> Date: Wed, 29 Sep 2021 14:11:28 +0800 Subject: [PATCH 315/317] support obs storage --- src/IO/S3Common.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index bb73a116ea4..51dbf44a765 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -603,7 +603,7 @@ namespace S3 /// Case when bucket name represented in domain name of S3 URL. /// E.g. (https://bucket-name.s3.Region.amazonaws.com/key) /// https://docs.aws.amazon.com/AmazonS3/latest/dev/VirtualHosting.html#virtual-hosted-style-access - static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3|cos)([.\-][a-z0-9\-.:]+))"); + static const RE2 virtual_hosted_style_pattern(R"((.+)\.(s3|cos|obs)([.\-][a-z0-9\-.:]+))"); /// Case when bucket name and key represented in path of S3 URL. /// E.g. (https://s3.Region.amazonaws.com/bucket-name/key) @@ -613,6 +613,8 @@ namespace S3 static constexpr auto S3 = "S3"; static constexpr auto COSN = "COSN"; static constexpr auto COS = "COS"; + static constexpr auto OBS = "OBS"; + uri = uri_; storage_name = S3; @@ -636,7 +638,7 @@ namespace S3 } boost::to_upper(name); - if (name != S3 && name != COS) + if (name != S3 && name != COS && name != OBS) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Object storage system name is unrecognized in virtual hosted style S3 URI: {}", quoteString(name)); } @@ -644,6 +646,10 @@ namespace S3 { storage_name = name; } + else if (name == OBS) + { + storage_name = OBS; + } else { storage_name = COSN; From d64587ef019cdf794a15d75a0311cd8b1c237641 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 29 Sep 2021 10:02:47 +0300 Subject: [PATCH 316/317] Bump cmake dependency to 3.14 libxz uses if(DEFINED CACHE{}) which has been added only in 3.14. --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 9c8903d853c..3f553c5c26d 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -1,4 +1,4 @@ -cmake_minimum_required(VERSION 3.3) +cmake_minimum_required(VERSION 3.14) foreach(policy CMP0023 From f1161d4752a6c3fb98fd1e8f09dda7a0b00d1e37 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 29 Sep 2021 15:54:23 +0300 Subject: [PATCH 317/317] Changelog heredoc added backward incompatible change --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6e209293e67..3e7f3fd4665 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,7 @@ * Under clickhouse-local, always treat local addresses with a port as remote. [#26736](https://github.com/ClickHouse/ClickHouse/pull/26736) ([Raúl Marín](https://github.com/Algunenano)). * Fix the issue that in case of some sophisticated query with column aliases identical to the names of expressions, bad cast may happen. This fixes [#25447](https://github.com/ClickHouse/ClickHouse/issues/25447). This fixes [#26914](https://github.com/ClickHouse/ClickHouse/issues/26914). This fix may introduce backward incompatibility: if there are different expressions with identical names, exception will be thrown. It may break some rare cases when `enable_optimize_predicate_expression` is set. [#26639](https://github.com/ClickHouse/ClickHouse/pull/26639) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Now, scalar subquery always returns `Nullable` result if it's type can be `Nullable`. It is needed because in case of empty subquery it's result should be `Null`. Previously, it was possible to get error about incompatible types (type deduction does not execute scalar subquery, and it could use not-nullable type). Scalar subquery with empty result which can't be converted to `Nullable` (like `Array` or `Tuple`) now throws error. Fixes [#25411](https://github.com/ClickHouse/ClickHouse/issues/25411). [#26423](https://github.com/ClickHouse/ClickHouse/pull/26423) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Introduce syntax for here documents. Example `SELECT $doc$ VALUE $doc$`. [#26671](https://github.com/ClickHouse/ClickHouse/pull/26671) ([Maksim Kita](https://github.com/kitaisreal)). This change is backward incompatible if in query there are identifiers that contain `$` [#28768](https://github.com/ClickHouse/ClickHouse/issues/28768). #### New Feature @@ -17,7 +18,6 @@ * Added integration with S2 geometry library. [#24980](https://github.com/ClickHouse/ClickHouse/pull/24980) ([Andr0901](https://github.com/Andr0901)). ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Add SQLite table engine, table function, database engine. [#24194](https://github.com/ClickHouse/ClickHouse/pull/24194) ([Arslan Gumerov](https://github.com/g-arslan)). ([Kseniia Sumarokova](https://github.com/kssenii)). * Added support for custom query for `MySQL`, `PostgreSQL`, `ClickHouse`, `JDBC`, `Cassandra` dictionary source. Closes [#1270](https://github.com/ClickHouse/ClickHouse/issues/1270). [#26995](https://github.com/ClickHouse/ClickHouse/pull/26995) ([Maksim Kita](https://github.com/kitaisreal)). -* Introduce syntax for here documents. Example `SELECT $doc$ VALUE $doc$`. [#26671](https://github.com/ClickHouse/ClickHouse/pull/26671) ([Maksim Kita](https://github.com/kitaisreal)). * Add shared (replicated) storage of user, roles, row policies, quotas and settings profiles through ZooKeeper. [#27426](https://github.com/ClickHouse/ClickHouse/pull/27426) ([Kevin Michel](https://github.com/kmichel-aiven)). * Add compression for `INTO OUTFILE` that automatically choose compression algorithm. Closes [#3473](https://github.com/ClickHouse/ClickHouse/issues/3473). [#27134](https://github.com/ClickHouse/ClickHouse/pull/27134) ([Filatenkov Artur](https://github.com/FArthur-cmd)). * Add `INSERT ... FROM INFILE` similarly to `SELECT ... INTO OUTFILE`. [#27655](https://github.com/ClickHouse/ClickHouse/pull/27655) ([Filatenkov Artur](https://github.com/FArthur-cmd)).