From 6afd9c176b2bc12a0c5fed93a3d7599dc1333ee6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 24 Aug 2022 21:05:00 +0000 Subject: [PATCH 01/39] inode based singleton --- .../decompressor.cpp | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 8d8d137a2ac..14e6e4171c7 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -13,6 +13,8 @@ #include #include #include +#include +#include #if (defined(OS_DARWIN) || defined(OS_FREEBSD)) && defined(__GNUC__) # include @@ -359,6 +361,27 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress #endif +uint32_t get_inode(const char * self) +{ + std::ifstream maps("/proc/self/maps"); + if (maps.fail()) + { + perror("open maps"); + return 0; + } + + for (std::string line; std::getline(maps, line); ) + { + std::stringstream ss(line); + std::string addr, mode, offset, id, path; + uint32_t inode = 0; + if (ss >> addr >> mode >> offset >> id >> inode >> path && path == self) + return inode; + } + + return 0; +} + int main(int/* argc*/, char* argv[]) { char self[4096] = {0}; @@ -382,6 +405,52 @@ int main(int/* argc*/, char* argv[]) else name = file_path; + uint32_t inode = get_inode(self); + if (inode == 0) + { + std::cerr << "Unable to obtain inode." << std::endl; + return 1; + } + + std::stringstream lock_path; + lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; + int lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); + if (lock < 0) + { + perror("lock open"); + return 1; + } + + if (lockf(lock, F_LOCK, 0)) + { + perror("lockf"); + return 1; + } + + struct stat input_info; + if (0 != stat(self, &input_info)) + { + perror("stat"); + return 1; + } + + /// if decompression was performed by another process + if (input_info.st_ino != inode) + { + struct stat lock_info; + if (0 != fstat(lock, &lock_info)) + { + perror("fstat lock"); + return 1; + } + + if (lock_info.st_size == 1) + execv(self, argv); + + printf("No target executable - decompression only was performed.\n"); + return 0; + } + int input_fd = open(self, O_RDONLY); if (input_fd == -1) { @@ -443,6 +512,8 @@ int main(int/* argc*/, char* argv[]) if (has_exec) { + write(lock, "1", 1); + execv(self, argv); /// This part of code will be reached only if error happened From f043d0305381742d5456b2d01e72b9d630143aa0 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 24 Aug 2022 21:28:52 +0000 Subject: [PATCH 02/39] fix style --- utils/self-extracting-executable/decompressor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 14e6e4171c7..bf5be5f15cf 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -372,7 +372,7 @@ uint32_t get_inode(const char * self) for (std::string line; std::getline(maps, line); ) { - std::stringstream ss(line); + std::stringstream ss(line); // STYLE_CHECK_ALLOW_STD_STRING_STREAM std::string addr, mode, offset, id, path; uint32_t inode = 0; if (ss >> addr >> mode >> offset >> id >> inode >> path && path == self) @@ -412,7 +412,7 @@ int main(int/* argc*/, char* argv[]) return 1; } - std::stringstream lock_path; + std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; int lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); if (lock < 0) From 6c1a7205e34f766da376311de474286ca066991c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 24 Aug 2022 21:47:37 +0000 Subject: [PATCH 03/39] fix style --- utils/self-extracting-executable/decompressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index bf5be5f15cf..e687717dacd 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -370,7 +370,7 @@ uint32_t get_inode(const char * self) return 0; } - for (std::string line; std::getline(maps, line); ) + for (std::string line; std::getline(maps, line);) { std::stringstream ss(line); // STYLE_CHECK_ALLOW_STD_STRING_STREAM std::string addr, mode, offset, id, path; From ae82c687256e69486723872c70fe1a0cb83afda9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 25 Aug 2022 10:52:56 +0000 Subject: [PATCH 04/39] truncate lock --- utils/self-extracting-executable/decompressor.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index e687717dacd..73d34f1fe8e 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -521,6 +521,8 @@ int main(int/* argc*/, char* argv[]) return 1; } + ftruncate(lock, 0); + printf("No target executable - decompression only was performed.\n"); } } From c5bc7b0a0c3a29bb906bc23d7c01d8b52ae5d67e Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 19 Jul 2022 14:07:44 +0000 Subject: [PATCH 05/39] Resize pipeline after full sort join --- src/Processors/QueryPlan/JoinStep.cpp | 6 +++++- src/QueryPipeline/QueryPipelineBuilder.cpp | 6 ++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/JoinStep.cpp b/src/Processors/QueryPlan/JoinStep.cpp index 909933fbed2..6e212a53bc6 100644 --- a/src/Processors/QueryPlan/JoinStep.cpp +++ b/src/Processors/QueryPlan/JoinStep.cpp @@ -34,8 +34,12 @@ QueryPipelineBuilderPtr JoinStep::updatePipeline(QueryPipelineBuilders pipelines throw Exception(ErrorCodes::LOGICAL_ERROR, "JoinStep expect two input steps"); if (join->pipelineType() == JoinPipelineType::YShaped) - return QueryPipelineBuilder::joinPipelinesYShaped( + { + auto joined_pipeline = QueryPipelineBuilder::joinPipelinesYShaped( std::move(pipelines[0]), std::move(pipelines[1]), join, output_stream->header, max_block_size, &processors); + joined_pipeline->resize(max_streams); + return joined_pipeline; + } return QueryPipelineBuilder::joinPipelinesRightLeft( std::move(pipelines[0]), diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 82907d883bc..a7db7442cba 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -348,8 +348,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesYShaped left->pipe.dropExtremes(); right->pipe.dropExtremes(); - - if (left->pipe.output_ports.size() != 1 || right->pipe.output_ports.size() != 1) + if (left->getNumStreams() != 1 || right->getNumStreams() != 1) throw Exception("Join is supported only for pipelines with one output port", ErrorCodes::LOGICAL_ERROR); if (left->hasTotals() || right->hasTotals()) @@ -359,8 +358,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesYShaped auto joining = std::make_shared(join, inputs, out_header, max_block_size); - auto result = mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors); - return result; + return mergePipelines(std::move(left), std::move(right), std::move(joining), collected_processors); } std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLeft( From 031aaf3a4545ba491e02bec1af0001df06add20b Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 19 Jul 2022 17:43:18 +0000 Subject: [PATCH 06/39] Add Creating/FilterBySetsOnTheFlyStep for full sorting join --- src/Core/Settings.h | 2 + src/Interpreters/InterpreterSelectQuery.cpp | 26 ++++ src/Interpreters/Set.cpp | 16 ++- src/Interpreters/Set.h | 8 +- .../QueryPlan/CreatingSetOnTheFlyStep.cpp | 134 +++++++++++++++++ .../QueryPlan/CreatingSetOnTheFlyStep.h | 58 ++++++++ .../CreatingSetsOnTheFlyTransform.cpp | 135 ++++++++++++++++++ .../CreatingSetsOnTheFlyTransform.h | 71 +++++++++ 8 files changed, 445 insertions(+), 5 deletions(-) create mode 100644 src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp create mode 100644 src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h create mode 100644 src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp create mode 100644 src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h diff --git a/src/Core/Settings.h b/src/Core/Settings.h index af32c15a867..60f588f1618 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -366,6 +366,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \ + M(UInt64, max_bytes_in_set_to_optimize_join, 1_GiB, "Maximal size of the set to filter joined tables by each other row sets before joining.", 0) \ + \ M(Bool, compatibility_ignore_collation_in_create_table, true, "Compatibility ignore collation in create table", 0) \ \ M(String, temporary_files_codec, "LZ4", "Set compression codec for temporary files (sort and join on disk). I.e. LZ4, NONE.", 0) \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 76f199d8e41..cb33f3657b2 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #include #include #include @@ -1445,9 +1446,34 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

SetPtr + { + SizeLimits size_limits(0, settings.max_bytes_in_set_to_optimize_join, OverflowMode::BREAK); + auto creating_set_step = std::make_unique(plan.getCurrentDataStream(), key_names, size_limits); + SetPtr set = creating_set_step->getSet(); + plan.addStep(std::move(creating_set_step)); + return set; + }; + + auto add_filter_by_set = [](QueryPlan & plan, const Names & key_names, SetPtr set) + { + auto filter_by_set_step = std::make_unique(plan.getCurrentDataStream(), key_names, set); + plan.addStep(std::move(filter_by_set_step)); + }; + if (expressions.join->pipelineType() == JoinPipelineType::YShaped) { const auto & join_clause = expressions.join->getTableJoin().getOnlyClause(); + + if (settings.max_bytes_in_set_to_optimize_join > 0) + { + SetPtr left_set = add_create_set(query_plan, join_clause.key_names_left); + SetPtr right_set = add_create_set(*joined_plan, join_clause.key_names_right); + + add_filter_by_set(query_plan, join_clause.key_names_left, right_set); + add_filter_by_set(*joined_plan, join_clause.key_names_right, left_set); + } + add_sorting(query_plan, join_clause.key_names_left, false); add_sorting(*joined_plan, join_clause.key_names_right, true); } diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index af05b33c1f6..7d279750f4c 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -22,6 +22,8 @@ #include #include +#include + #include #include @@ -162,8 +164,16 @@ void Set::setHeader(const ColumnsWithTypeAndName & header) data.init(data.chooseMethod(key_columns, key_sizes)); } - bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns) +{ + Columns cols; + cols.reserve(columns.size()); + for (const auto & column : columns) + cols.emplace_back(column.column); + return insertFromBlock(cols); +} + +bool Set::insertFromBlock(const Columns & columns) { std::lock_guard lock(rwlock); @@ -179,11 +189,11 @@ bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns) /// Remember the columns we will work with for (size_t i = 0; i < keys_size; ++i) { - materialized_columns.emplace_back(columns.at(i).column->convertToFullIfNeeded()); + materialized_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); key_columns.emplace_back(materialized_columns.back().get()); } - size_t rows = columns.at(0).column->size(); + size_t rows = columns.at(0)->size(); /// We will insert to the Set only keys, where all components are not NULL. ConstNullMapPtr null_map{}; diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 6a3b28407ee..44f543ce222 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -20,6 +20,7 @@ class Context; class IFunctionBase; using FunctionBasePtr = std::shared_ptr; +class Chunk; /** Data structure for implementation of IN expression. */ @@ -45,11 +46,14 @@ public: void setHeader(const ColumnsWithTypeAndName & header); /// Returns false, if some limit was exceeded and no need to insert more data. + bool insertFromBlock(const Columns & columns); bool insertFromBlock(const ColumnsWithTypeAndName & columns); + /// Call after all blocks were inserted. To get the information that set is already created. void finishInsert() { is_created = true; } - bool isCreated() const { return is_created; } + /// finishInsert and isCreated are thread-safe + bool isCreated() const { return is_created.load(); } /** For columns of 'block', check belonging of corresponding rows to the set. * Return UInt8 column with the result. @@ -111,7 +115,7 @@ private: bool transform_null_in; /// Check if set contains all the data. - bool is_created = false; + std::atomic is_created = false; /// If in the left part columns contains the same types as the elements of the set. void executeOrdinary( diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp new file mode 100644 index 00000000000..7a718a17ad7 --- /dev/null +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp @@ -0,0 +1,134 @@ +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +static ITransformingStep::Traits getTraits(bool is_filter) +{ + return ITransformingStep::Traits + { + { + .preserves_distinct_columns = true, + .returns_single_stream = true, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = !is_filter, + } + }; +} + +CreatingSetOnTheFlyStep::CreatingSetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, const SizeLimits & size_limits) + : ITransformingStep(input_stream_, input_stream_.header, getTraits(false)) + , column_names(column_names_) +{ + if (input_streams.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); + + set = std::make_shared(size_limits, false, true); + + { + ColumnsWithTypeAndName header; + for (const auto & name : column_names) + { + ColumnWithTypeAndName column = input_streams[0].header.getByName(name); + header.emplace_back(column); + } + set->setHeader(header); + } +} + +void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) +{ + UNUSED(settings); + size_t num_streams = pipeline.getNumStreams(); + + pipeline.resize(1); + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + return std::make_shared(header, column_names, set); + }); + pipeline.resize(num_streams); +} + +void CreatingSetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const +{ + map.add(getName(), true); +} + +void CreatingSetOnTheFlyStep::describeActions(FormatSettings & settings) const +{ + String prefix(settings.offset, ' '); + settings.out << prefix << getName(); + + settings.out << '\n'; +} + +void CreatingSetOnTheFlyStep::updateOutputStream() +{ + if (input_streams.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); + + output_stream = input_streams[0]; +} + + +FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, SetPtr set_) + : ITransformingStep(input_stream_, input_stream_.header, getTraits(true)) + , column_names(column_names_) + , set(set_) +{ + if (input_streams.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); +} + +void FilterBySetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) +{ + UNUSED(settings); + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipelineBuilder::StreamType::Main) + return nullptr; + return std::make_shared(header, column_names, set); + }); +} + +void FilterBySetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const +{ + map.add(getName(), true); +} + +void FilterBySetOnTheFlyStep::describeActions(FormatSettings & settings) const +{ + String prefix(settings.offset, ' '); + settings.out << prefix << getName(); + + settings.out << '\n'; +} + +void FilterBySetOnTheFlyStep::updateOutputStream() +{ + if (input_streams.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); + + output_stream = input_streams[0]; +} + + +} diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h new file mode 100644 index 00000000000..9482428534e --- /dev/null +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h @@ -0,0 +1,58 @@ +#pragma once +#include +#include + +#include + + +namespace DB +{ + +using SetPtr = std::shared_ptr; + +class CreatingSetOnTheFlyStep : public ITransformingStep +{ +public: + explicit CreatingSetOnTheFlyStep( + const DataStream & input_stream_, + const Names & column_names_, + const SizeLimits & size_limits = {}); + + String getName() const override { return "CreatingSetsOnTheFly"; } + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; + + void describeActions(JSONBuilder::JSONMap & map) const override; + void describeActions(FormatSettings & settings) const override; + + SetPtr getSet() const { return set; } + +private: + void updateOutputStream() override; + + Names column_names; + SetPtr set; +}; + + +class FilterBySetOnTheFlyStep : public ITransformingStep +{ +public: + FilterBySetOnTheFlyStep( + const DataStream & input_stream_, + const Names & column_names_, + SetPtr set_); + + String getName() const override { return "FilterBySetOnTheFly"; } + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; + + void describeActions(JSONBuilder::JSONMap & map) const override; + void describeActions(FormatSettings & settings) const override; + +private: + void updateOutputStream() override; + + Names column_names; + SetPtr set; +}; + +} diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp new file mode 100644 index 00000000000..10b8513ee2b --- /dev/null +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp @@ -0,0 +1,135 @@ +#include +#include +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +std::vector getColumnIndices(const Block & block, const Names & column_names) +{ + std::vector indices; + for (const auto & name : column_names) + indices.push_back(block.getPositionByName(name)); + return indices; +} + +Columns getColumnsByIndices(const Chunk & chunk, const std::vector & indices) +{ + Columns columns; + const Columns & all_cols = chunk.getColumns(); + for (const auto & index : indices) + columns.push_back(all_cols.at(index)); + return columns; +} + +ColumnsWithTypeAndName getColumnsByIndices(const Block & sample_block, const Chunk & chunk, const std::vector & indices) +{ + Block block = sample_block.cloneEmpty(); + block.setColumns(getColumnsByIndices(chunk, indices)); + return block.getColumnsWithTypeAndName(); +} + +} + + +CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform(const Block & header_, const Names & column_names, SetPtr set_) + : ISimpleTransform(header_, header_, true) + , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) + , set(set_) + , log(&Poco::Logger::get(getName())) +{ +} + +IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() +{ + auto status = ISimpleTransform::prepare(); + return status; +} + +void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) +{ + if (!set) + return; + + if (chunk.getNumRows()) + { + Columns key_cols = getColumnsByIndices(chunk, key_column_indices); + bool limit_exceeded = !set->insertFromBlock(key_cols); + if (limit_exceeded) + { + LOG_DEBUG(log, "Set limit exceeded, give up building set, after using {} KB", set->getTotalByteCount() / 1024); + // set->clear(); + // LOG_DEBUG(log, "Set limit exceeded, give up building set, after using {} KB", set->getTotalByteCount() / 1024); + set.reset(); + } + } + + if (input.isFinished()) + { + set->finishInsert(); + LOG_DEBUG(log, "Finish building set with {} rows, set size is {} MB", set->getTotalRowCount(), set->getTotalByteCount() / 1024 / 1024); + + /// Release pointer to make it possible destroy it by consumer + set.reset(); + } +} + +FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_, const Names & column_names, SetPtr set_) + : ISimpleTransform(header_, header_, true) + , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) + , set(set_) + , log(&Poco::Logger::get(getName())) +{ + const auto & header = inputs.front().getHeader(); + for (size_t idx : key_column_indices) + key_sample_block.insert(header.getByPosition(idx)); +} + +IProcessor::Status FilterBySetOnTheFlyTransform::prepare() +{ + auto status = ISimpleTransform::prepare(); + return status; +} + +void FilterBySetOnTheFlyTransform::transform(Chunk & chunk) +{ + + if (!set) + return; + + if (!set->isCreated()) + return; + + if (chunk.getNumRows()) + { + auto key_columns = getColumnsByIndices(key_sample_block, chunk, key_column_indices); + ColumnPtr mask_col = set->execute(key_columns, false); + const auto & mask = assert_cast(mask_col.get())->getData(); + + Columns columns = chunk.detachColumns(); + size_t num_rows = 0; + for (auto & col : columns) + { + col = col->filter(mask, 0); + num_rows = col->size(); + } + chunk.setColumns(std::move(columns), num_rows); + } + + if (input.isFinished()) + { + /// Release set to free memory + set.reset(); + } +} + +} diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h new file mode 100644 index 00000000000..e36e850030f --- /dev/null +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h @@ -0,0 +1,71 @@ +#pragma once +#include +#include +#include + + +namespace DB +{ + +class Set; +using SetPtr = std::shared_ptr; + +/* + * Create a set on the fly. + * The set is created from the key columns of the input block. + * Data is not changed and returned as is. + * Can be executed only in one stream. + */ +class CreatingSetsOnTheFlyTransform : public ISimpleTransform +{ +public: + explicit CreatingSetsOnTheFlyTransform( + const Block & header_, const Names & column_names_, SetPtr set_); + + String getName() const override { return "CreatingSetsOnTheFlyTransform"; } + + Status prepare() override; + + void transform(Chunk & chunk) override; + +private: + std::vector key_column_indices; + + /// Set to fill + SetPtr set; + + Poco::Logger * log; +}; + + +/* + * Filter the input chunk by the set. + * When set building is not comleted, just return the source data. + */ +class FilterBySetOnTheFlyTransform : public ISimpleTransform +{ +public: + explicit FilterBySetOnTheFlyTransform( + const Block & header_, const Names & column_names_, SetPtr set_); + + String getName() const override { return "FilterBySetOnTheFlyTransform"; } + + Status prepare() override; + + void transform(Chunk & chunk) override; + +private: + /// Set::execute requires ColumnsWithTypesAndNames, so we need to convert Chunk to that format + Block key_sample_block; + + std::vector key_column_indices; + + /// Filter by this set when it's created + SetPtr set; + + Poco::Logger * log; +}; + + + +} From 3292566603df6511dd65365e9fd1f8b53e76e4cc Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 20 Jul 2022 10:21:04 +0000 Subject: [PATCH 07/39] Format bytes in CreatingSetsOnTheFlyTransform logs --- .../CreatingSetsOnTheFlyTransform.cpp | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp index 10b8513ee2b..4a1ea45cdef 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp @@ -38,6 +38,17 @@ ColumnsWithTypeAndName getColumnsByIndices(const Block & sample_block, const Chu return block.getColumnsWithTypeAndName(); } +std::string formatBytesHumanReadable(size_t bytes) +{ + if (bytes >= 1_GiB) + return fmt::format("{:.2f} GB", static_cast(bytes) / 1_GiB); + if (bytes >= 1_MiB) + return fmt::format("{:.2f} MB", static_cast(bytes) / 1_MiB); + if (bytes >= 1_KiB) + return fmt::format("{:.2f} KB", static_cast(bytes) / 1_KiB); + return fmt::format("{:.2f} B", static_cast(bytes)); +} + } @@ -66,9 +77,8 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) bool limit_exceeded = !set->insertFromBlock(key_cols); if (limit_exceeded) { - LOG_DEBUG(log, "Set limit exceeded, give up building set, after using {} KB", set->getTotalByteCount() / 1024); + LOG_DEBUG(log, "Set limit exceeded, give up building set, after using {}", formatBytesHumanReadable(set->getTotalByteCount())); // set->clear(); - // LOG_DEBUG(log, "Set limit exceeded, give up building set, after using {} KB", set->getTotalByteCount() / 1024); set.reset(); } } @@ -76,7 +86,8 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) if (input.isFinished()) { set->finishInsert(); - LOG_DEBUG(log, "Finish building set with {} rows, set size is {} MB", set->getTotalRowCount(), set->getTotalByteCount() / 1024 / 1024); + LOG_DEBUG(log, "Finish building set with {} rows, set size is {}", + set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); /// Release pointer to make it possible destroy it by consumer set.reset(); From 8f06430ebda36d52c80a0933ba0dc15b8b7c9fab Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 27 Jul 2022 13:35:22 +0000 Subject: [PATCH 08/39] Create sets for joins: upd --- src/Interpreters/InterpreterSelectQuery.cpp | 16 ++--- src/Interpreters/TableJoin.h | 24 ++++++-- .../QueryPlan/CreatingSetOnTheFlyStep.cpp | 17 ++++-- .../QueryPlan/CreatingSetOnTheFlyStep.h | 12 ++-- .../Optimizations/filterPushDown.cpp | 48 +++++++++------ .../CreatingSetsOnTheFlyTransform.cpp | 59 +++++++++++++------ .../CreatingSetsOnTheFlyTransform.h | 36 +++++++++-- src/QueryPipeline/Pipe.h | 2 +- 8 files changed, 146 insertions(+), 68 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index cb33f3657b2..3229313941f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1446,18 +1446,20 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

SetPtr + auto add_create_set = [&settings](QueryPlan & plan, const Names & key_names, bool is_right) { SizeLimits size_limits(0, settings.max_bytes_in_set_to_optimize_join, OverflowMode::BREAK); auto creating_set_step = std::make_unique(plan.getCurrentDataStream(), key_names, size_limits); - SetPtr set = creating_set_step->getSet(); + creating_set_step->setStepDescription(fmt::format("Create set for {} stream", is_right ? "right" : "left")); + auto set = creating_set_step->getSet(); plan.addStep(std::move(creating_set_step)); return set; }; - auto add_filter_by_set = [](QueryPlan & plan, const Names & key_names, SetPtr set) + auto add_filter_by_set = [](QueryPlan & plan, const Names & key_names, auto set, bool is_right) { auto filter_by_set_step = std::make_unique(plan.getCurrentDataStream(), key_names, set); + filter_by_set_step->setStepDescription(fmt::format("Filter {} stream by set", is_right ? "right" : "left")); plan.addStep(std::move(filter_by_set_step)); }; @@ -1467,11 +1469,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

0) { - SetPtr left_set = add_create_set(query_plan, join_clause.key_names_left); - SetPtr right_set = add_create_set(*joined_plan, join_clause.key_names_right); + auto left_set = add_create_set(query_plan, join_clause.key_names_left, false); + auto right_set = add_create_set(*joined_plan, join_clause.key_names_right, true); - add_filter_by_set(query_plan, join_clause.key_names_left, right_set); - add_filter_by_set(*joined_plan, join_clause.key_names_right, left_set); + add_filter_by_set(query_plan, join_clause.key_names_left, right_set, false); + add_filter_by_set(*joined_plan, join_clause.key_names_right, left_set, true); } add_sorting(query_plan, join_clause.key_names_left, false); diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index d0bf64fdebe..3835ef77deb 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -73,16 +73,32 @@ public: return key_names_right.size(); } - String formatDebug() const + String formatDebug(bool short_format = false) const { - return fmt::format("Left keys: [{}] Right keys [{}] Condition columns: '{}', '{}'", - fmt::join(key_names_left, ", "), fmt::join(key_names_right, ", "), - condColumnNames().first, condColumnNames().second); + const auto & [left_cond, right_cond] = condColumnNames(); + + if (short_format) + { + return fmt::format("({}) = ({}){}{}", fmt::join(key_names_left, ", "), fmt::join(key_names_right, ", "), + !left_cond.empty() ? " AND " + left_cond : "", !right_cond.empty() ? " AND " + right_cond : ""); + } + + return fmt::format( + "Left keys: [{}] Right keys [{}] Condition columns: '{}', '{}'", + fmt::join(key_names_left, ", "), fmt::join(key_names_right, ", "), left_cond, right_cond); } }; using Clauses = std::vector; + static std::string formatClauses(const Clauses & clauses, bool short_format = false) + { + std::vector res; + for (const auto & clause : clauses) + res.push_back("[" + clause.formatDebug(short_format) + "]"); + return fmt::format("{}", fmt::join(res, "; ")); + } + 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. diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp index 7a718a17ad7..8eaeecb597f 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp @@ -22,9 +22,9 @@ static ITransformingStep::Traits getTraits(bool is_filter) { { .preserves_distinct_columns = true, - .returns_single_stream = true, + .returns_single_stream = false, .preserves_number_of_streams = true, - .preserves_sorting = true, + .preserves_sorting = false, /// resize doesn't perserve sorting (TODO fixit) }, { .preserves_number_of_rows = !is_filter, @@ -39,7 +39,7 @@ CreatingSetOnTheFlyStep::CreatingSetOnTheFlyStep(const DataStream & input_stream if (input_streams.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); - set = std::make_shared(size_limits, false, true); + set = std::make_shared(size_limits, false, true); { ColumnsWithTypeAndName header; @@ -60,9 +60,12 @@ void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, pipeline.resize(1); pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { + if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - return std::make_shared(header, column_names, set); + auto res = std::make_shared(header, column_names, set); + res->setDescription(this->getStepDescription()); + return res; }); pipeline.resize(num_streams); } @@ -89,7 +92,7 @@ void CreatingSetOnTheFlyStep::updateOutputStream() } -FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, SetPtr set_) +FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, SetWithStatePtr set_) : ITransformingStep(input_stream_, input_stream_.header, getTraits(true)) , column_names(column_names_) , set(set_) @@ -105,7 +108,9 @@ void FilterBySetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, { if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - return std::make_shared(header, column_names, set); + auto res = std::make_shared(header, column_names, set); + res->setDescription(this->getStepDescription()); + return res; }); } diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h index 9482428534e..4966173152a 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h @@ -2,14 +2,10 @@ #include #include -#include - namespace DB { -using SetPtr = std::shared_ptr; - class CreatingSetOnTheFlyStep : public ITransformingStep { public: @@ -24,13 +20,13 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; - SetPtr getSet() const { return set; } + SetWithStatePtr getSet() const { return set; } private: void updateOutputStream() override; Names column_names; - SetPtr set; + SetWithStatePtr set; }; @@ -40,7 +36,7 @@ public: FilterBySetOnTheFlyStep( const DataStream & input_stream_, const Names & column_names_, - SetPtr set_); + SetWithStatePtr set_); String getName() const override { return "FilterBySetOnTheFly"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; @@ -52,7 +48,7 @@ private: void updateOutputStream() override; Names column_names; - SetPtr set; + SetWithStatePtr set; }; } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 6c6c8954ea4..99312a17ffd 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -22,6 +23,7 @@ #include #include #include +#include namespace DB::ErrorCodes { @@ -134,10 +136,24 @@ tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, con static size_t tryAddNewFilterStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, const Names & allowed_inputs, - bool can_remove_filter = true) + bool can_remove_filter = true, size_t child_idx = 0) { - if (auto split_filter = splitFilter(parent_node, allowed_inputs, 0)) - return tryAddNewFilterStep(parent_node, nodes, split_filter, can_remove_filter, 0); + if (auto split_filter = splitFilter(parent_node, allowed_inputs, child_idx)) + return tryAddNewFilterStep(parent_node, nodes, split_filter, can_remove_filter, child_idx); + return 0; +} + + +/// Push down filter through specified type of step +template +static size_t simplePushDownOverStep(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes, QueryPlanStepPtr & child) +{ + if (typeid_cast(child.get())) + { + Names allowed_inputs = child->getOutputStream().header.getNames(); + if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) + return updated_steps; + } return 0; } @@ -234,12 +250,8 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes return updated_steps; } - if (auto * distinct = typeid_cast(child.get())) - { - Names allowed_inputs = distinct->getOutputStream().header.getNames(); - if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) - return updated_steps; - } + if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) + return updated_steps; if (auto * join = typeid_cast(child.get())) { @@ -290,9 +302,9 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes const size_t updated_steps = tryAddNewFilterStep(parent_node, nodes, split_filter, can_remove_filter, child_idx); if (updated_steps > 0) { - LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), "Pushed down filter to {} side of join", kind); + LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), "Pushed down filter {} to the {} side of join", split_filter_column_name, kind); + return updated_steps; } - return updated_steps; }; if (size_t updated_steps = join_push_down(JoinKind::Left)) @@ -321,12 +333,14 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes // { // } - if (typeid_cast(child.get())) - { - Names allowed_inputs = child->getOutputStream().header.getNames(); - if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) - return updated_steps; - } + if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) + return updated_steps; + + if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) + return updated_steps; + + if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) + return updated_steps; if (auto * union_step = typeid_cast(child.get())) { diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp index 4a1ea45cdef..d52a669ee75 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp @@ -52,8 +52,9 @@ std::string formatBytesHumanReadable(size_t bytes) } -CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform(const Block & header_, const Names & column_names, SetPtr set_) +CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform(const Block & header_, const Names & column_names_, SetWithStatePtr set_) : ISimpleTransform(header_, header_, true) + , column_names(column_names_) , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) , set(set_) , log(&Poco::Logger::get(getName())) @@ -77,8 +78,10 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) bool limit_exceeded = !set->insertFromBlock(key_cols); if (limit_exceeded) { - LOG_DEBUG(log, "Set limit exceeded, give up building set, after using {}", formatBytesHumanReadable(set->getTotalByteCount())); + LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after using {}", + getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); // set->clear(); + set->state = SetWithState::State::Suspended; set.reset(); } } @@ -86,16 +89,18 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) if (input.isFinished()) { set->finishInsert(); - LOG_DEBUG(log, "Finish building set with {} rows, set size is {}", - set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + set->state = SetWithState::State::Finished; + LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", + getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); /// Release pointer to make it possible destroy it by consumer set.reset(); } } -FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_, const Names & column_names, SetPtr set_) +FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_, const Names & column_names_, SetWithStatePtr set_) : ISimpleTransform(header_, header_, true) + , column_names(column_names_) , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) , set(set_) , log(&Poco::Logger::get(getName())) @@ -108,38 +113,54 @@ FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_ IProcessor::Status FilterBySetOnTheFlyTransform::prepare() { auto status = ISimpleTransform::prepare(); + if (status == Status::Finished) + { + bool has_filter = set && set->state == SetWithState::State::Finished; + if (has_filter) + { + LOG_DEBUG(log, "Finished {} by [{}]: consumed {} rows in total, {} rows bypassed, result {} rows, {}% filtered", + Poco::toLower(getDescription()), fmt::join(column_names, ", "), + stat.consumed_rows, stat.consumed_rows_before_set, stat.result_rows, + static_cast(100 - 100.0 * stat.result_rows / stat.consumed_rows)); + } + else + { + LOG_DEBUG(log, "Finished {}: bypass {} rows", Poco::toLower(getDescription()), stat.consumed_rows); + } + + /// Release set to free memory + set = nullptr; + } return status; } void FilterBySetOnTheFlyTransform::transform(Chunk & chunk) { + stat.consumed_rows += chunk.getNumRows(); + stat.result_rows += chunk.getNumRows(); + bool can_filter = set && set->state == SetWithState::State::Finished; - if (!set) - return; + if (!can_filter) + stat.consumed_rows_before_set += chunk.getNumRows(); - if (!set->isCreated()) - return; - - if (chunk.getNumRows()) + if (can_filter && chunk.getNumRows()) { auto key_columns = getColumnsByIndices(key_sample_block, chunk, key_column_indices); ColumnPtr mask_col = set->execute(key_columns, false); const auto & mask = assert_cast(mask_col.get())->getData(); + stat.result_rows -= chunk.getNumRows(); + Columns columns = chunk.detachColumns(); - size_t num_rows = 0; + size_t result_num_rows = 0; for (auto & col : columns) { col = col->filter(mask, 0); - num_rows = col->size(); + result_num_rows = col->size(); } - chunk.setColumns(std::move(columns), num_rows); - } + stat.result_rows += result_num_rows; - if (input.isFinished()) - { - /// Release set to free memory - set.reset(); + chunk.setColumns(std::move(columns), result_num_rows); } } diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h index e36e850030f..ea91a96818a 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h @@ -2,13 +2,28 @@ #include #include #include +#include namespace DB { -class Set; -using SetPtr = std::shared_ptr; +class SetWithState : public Set +{ +public: + using Set::Set; + + enum class State + { + Creating, + Finished, + Suspended, + }; + + std::atomic state = State::Creating; +}; + +using SetWithStatePtr = std::shared_ptr; /* * Create a set on the fly. @@ -20,7 +35,7 @@ class CreatingSetsOnTheFlyTransform : public ISimpleTransform { public: explicit CreatingSetsOnTheFlyTransform( - const Block & header_, const Names & column_names_, SetPtr set_); + const Block & header_, const Names & column_names_, SetWithStatePtr set_); String getName() const override { return "CreatingSetsOnTheFlyTransform"; } @@ -29,10 +44,11 @@ public: void transform(Chunk & chunk) override; private: + Names column_names; std::vector key_column_indices; /// Set to fill - SetPtr set; + SetWithStatePtr set; Poco::Logger * log; }; @@ -46,7 +62,7 @@ class FilterBySetOnTheFlyTransform : public ISimpleTransform { public: explicit FilterBySetOnTheFlyTransform( - const Block & header_, const Names & column_names_, SetPtr set_); + const Block & header_, const Names & column_names_, SetWithStatePtr set_); String getName() const override { return "FilterBySetOnTheFlyTransform"; } @@ -58,10 +74,18 @@ private: /// Set::execute requires ColumnsWithTypesAndNames, so we need to convert Chunk to that format Block key_sample_block; + Names column_names; std::vector key_column_indices; /// Filter by this set when it's created - SetPtr set; + SetWithStatePtr set; + + struct Stat + { + size_t consumed_rows = 0; + size_t consumed_rows_before_set = 0; + size_t result_rows = 0; + } stat; Poco::Logger * log; }; diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index 52059f4ad19..664ab719b13 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -85,7 +85,7 @@ public: /// Add chain to every output port. void addChains(std::vector chains); - /// Changes the number of output ports if needed. Adds ResizeTransform. + /// Changes the number of output ports if needed. Adds (Strict)ResizeProcessor. void resize(size_t num_streams, bool force = false, bool strict = false); using Transformer = std::function; From 71708d595f9a73b1ba12bd2636064b8e900461f3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 1 Aug 2022 13:20:12 +0000 Subject: [PATCH 09/39] Create sets for joins: wip --- src/Interpreters/InterpreterSelectQuery.cpp | 9 +- src/Processors/DelayedPortsProcessor.cpp | 128 ++++++++++++++++++ src/Processors/DelayedPortsProcessor.h | 74 ++++++++++ src/Processors/Port.cpp | 2 +- .../QueryPlan/CreatingSetOnTheFlyStep.cpp | 31 ++++- .../QueryPlan/CreatingSetOnTheFlyStep.h | 34 ++++- src/Processors/ResizeProcessor.h | 7 + src/QueryPipeline/Pipe.cpp | 5 +- src/QueryPipeline/Pipe.h | 2 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 4 +- src/QueryPipeline/QueryPipelineBuilder.h | 2 +- 11 files changed, 286 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 3229313941f..74e25562d29 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1456,9 +1456,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(plan.getCurrentDataStream(), key_names, set); + auto filter_by_set_step = std::make_unique(plan.getCurrentDataStream(), key_names, set, ports_state); filter_by_set_step->setStepDescription(fmt::format("Filter {} stream by set", is_right ? "right" : "left")); plan.addStep(std::move(filter_by_set_step)); }; @@ -1472,8 +1472,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(); + add_filter_by_set(query_plan, join_clause.key_names_left, right_set, ports_state, false); + add_filter_by_set(*joined_plan, join_clause.key_names_right, left_set, ports_state, true); } add_sorting(query_plan, join_clause.key_names_left, false); diff --git a/src/Processors/DelayedPortsProcessor.cpp b/src/Processors/DelayedPortsProcessor.cpp index 24023529bca..48930ca9e51 100644 --- a/src/Processors/DelayedPortsProcessor.cpp +++ b/src/Processors/DelayedPortsProcessor.cpp @@ -170,4 +170,132 @@ IProcessor::Status DelayedPortsProcessor::prepare(const PortNumbers & updated_in return Status::PortFull; } + +NotifyProcessor::NotifyProcessor(const Block & header, size_t num_ports) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)) + , aux_in_port(Block(), this) + , aux_out_port(Block(), this) +{ + port_pairs.resize(num_ports); + + auto input_it = inputs.begin(); + auto output_it = outputs.begin(); + for (size_t i = 0; i < num_ports; ++i) + { + port_pairs[i].input_port = &*input_it; + ++input_it; + + port_pairs[i].output_port = &*output_it; + ++output_it; + } + +} + +void NotifyProcessor::finishPair(PortsPair & pair) +{ + if (!pair.is_finished) + { + pair.output_port->finish(); + pair.input_port->close(); + + pair.is_finished = true; + ++num_finished_pairs; + } +} + +bool NotifyProcessor::processPair(PortsPair & pair) +{ + if (pair.output_port->isFinished()) + { + finishPair(pair); + return false; + } + + if (pair.input_port->isFinished()) + { + finishPair(pair); + return false; + } + + if (!pair.output_port->canPush()) + { + pair.input_port->setNotNeeded(); + return false; + } + + pair.input_port->setNeeded(); + if (pair.input_port->hasData()) + { + Chunk chunk = pair.input_port->pull(true); + dataCallback(chunk); + pair.output_port->push(std::move(chunk)); + } + + return true; +} + +IProcessor::Status NotifyProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +{ + bool need_data = false; + + for (const auto & output_number : updated_outputs) + need_data = processPair(port_pairs[output_number]) || need_data; + for (const auto & input_number : updated_inputs) + need_data = processPair(port_pairs[input_number]) || need_data; + + if (num_finished_pairs == port_pairs.size()) + return Status::Finished; + + if (need_data) + return Status::NeedData; + + return Status::PortFull; +} + +IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +{ + auto status = processRegularPorts(updated_inputs, updated_outputs); + if (status != Status::Ready) + return status; + + if (ready == AuxPortState::NotInitialized && isReady()) + ready = AuxPortState::Triggered; + + if (ready == AuxPortState::Triggered) + { + if (aux_out_port.canPush()) + { + aux_out_port.push({}); + ready = AuxPortState::Finished; + return Status::Ready; + } + return Status::PortFull; + } + + if (waiting == AuxPortState::NotInitialized && isWaiting()) + { + aux_in_port.setNeeded(); + waiting = AuxPortState::Triggered; + } + + if (waiting == AuxPortState::Triggered) + { + if (aux_in_port.hasData()) + { + aux_in_port.pull(true); + waiting = AuxPortState::Finished; + return Status::Ready; + } + return Status::PortFull; + } + + return Status::Ready; +} + +std::pair NotifyProcessor::getAuxPorts() +{ + return std::make_pair(&aux_in_port, &aux_out_port); +} + + } diff --git a/src/Processors/DelayedPortsProcessor.h b/src/Processors/DelayedPortsProcessor.h index a6a9590e0c8..a5314ca36a1 100644 --- a/src/Processors/DelayedPortsProcessor.h +++ b/src/Processors/DelayedPortsProcessor.h @@ -1,5 +1,7 @@ #pragma once #include +#include +#include namespace DB { @@ -39,4 +41,76 @@ private: void finishPair(PortsPair & pair); }; + +class NotifyProcessor : public IProcessor +{ +public: + NotifyProcessor(const Block & header, size_t num_ports); + + String getName() const override { return "NotifyProcessor"; } + + Status prepare(const PortNumbers &, const PortNumbers &) override; + + std::pair getAuxPorts(); + + virtual bool isReady() const { return true; } + virtual bool isWaiting() const { return false; } + + virtual void dataCallback(const Chunk & chunk) { UNUSED(chunk); } + +private: + + enum class AuxPortState + { + NotInitialized, + Triggered, + Finished, + }; + + struct PortsPair + { + InputPort * input_port = nullptr; + OutputPort * output_port = nullptr; + bool is_finished = false; + }; + + bool processPair(PortsPair & pair); + void finishPair(PortsPair & pair); + Status processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs); + + std::vector port_pairs; + size_t num_finished_pairs = 0; + + InputPort aux_in_port; + OutputPort aux_out_port; + + AuxPortState ready = AuxPortState::NotInitialized; + AuxPortState waiting = AuxPortState::NotInitialized; +}; + + +class NotifyProcessor2 : public NotifyProcessor +{ +public: + using NotifyProcessor::NotifyProcessor; + + bool isReady() const override + { + return data_consumed > 10_MiB; + } + + bool isWaiting() const override + { + return data_consumed < 10_MiB; + } + + void dataCallback(const Chunk & chunk) override + { + data_consumed += chunk.allocatedBytes(); + } + +private: + size_t data_consumed = 0; +}; + } diff --git a/src/Processors/Port.cpp b/src/Processors/Port.cpp index 86431dbc6e6..a856dbcea3b 100644 --- a/src/Processors/Port.cpp +++ b/src/Processors/Port.cpp @@ -19,7 +19,7 @@ void connect(OutputPort & output, InputPort & input) auto out_name = output.getProcessor().getName(); auto in_name = input.getProcessor().getName(); - assertCompatibleHeader(output.getHeader(), input.getHeader(), fmt::format(" function connect between {} and {}", out_name, in_name)); + assertCompatibleHeader(output.getHeader(), input.getHeader(), fmt::format("function connect between {} and {}", out_name, in_name)); input.output_port = &output; output.input_port = &input; diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp index 8eaeecb597f..0bb363fc17d 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -92,18 +93,46 @@ void CreatingSetOnTheFlyStep::updateOutputStream() } -FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, SetWithStatePtr set_) +FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, + SetWithStatePtr set_, PortsStatePtr ports_state_) : ITransformingStep(input_stream_, input_stream_.header, getTraits(true)) , column_names(column_names_) , set(set_) + , ports_state(ports_state_) { if (input_streams.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); } + +static void connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs) +{ + auto input_it = inputs.begin(); + for (auto & port : ports) + { + connect(*port, *input_it); + input_it++; + } + assert(input_it == inputs.end()); +} + void FilterBySetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { UNUSED(settings); + + Block input_header = pipeline.getHeader(); + pipeline.transform([&input_header, this](OutputPortRawPtrs ports) + { + size_t num_streams = ports.size(); + + auto notifier = std::make_shared(input_header, num_streams); + + connectAllInputs(ports, notifier->getInputs()); + ports_state->tryConnectPorts(notifier->getAuxPorts()); + + return Processors{notifier}; + }, /* check_ports= */ false); + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type != QueryPipelineBuilder::StreamType::Main) diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h index 4966173152a..2a85d26836d 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h @@ -33,10 +33,40 @@ private: class FilterBySetOnTheFlyStep : public ITransformingStep { public: + + class PortsState : public boost::noncopyable + { + public: + using PortPair = std::pair; + + /// Remember ports passed on the first call and connect with ones from second call. + bool tryConnectPorts(PortPair rhs_ports) + { + std::lock_guard lock(mux); + if (input_port || output_port) + { + assert(input_port && output_port); + connect(*rhs_ports.second, *input_port); + connect(*output_port, *rhs_ports.first); + return true; + } + std::tie(input_port, output_port) = rhs_ports; + return false; + } + + private: + std::mutex mux; + InputPort * input_port = nullptr; + OutputPort * output_port = nullptr; + }; + + using PortsStatePtr = std::shared_ptr; + FilterBySetOnTheFlyStep( const DataStream & input_stream_, const Names & column_names_, - SetWithStatePtr set_); + SetWithStatePtr set_, + PortsStatePtr ports_state_); String getName() const override { return "FilterBySetOnTheFly"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; @@ -48,7 +78,9 @@ private: void updateOutputStream() override; Names column_names; + SetWithStatePtr set; + PortsStatePtr ports_state; }; } diff --git a/src/Processors/ResizeProcessor.h b/src/Processors/ResizeProcessor.h index f9c188e041a..364d1b4c883 100644 --- a/src/Processors/ResizeProcessor.h +++ b/src/Processors/ResizeProcessor.h @@ -85,6 +85,13 @@ public: { } + StrictResizeProcessor(InputPorts inputs_, OutputPorts outputs_) + : IProcessor(inputs_, outputs_) + , current_input(inputs.begin()) + , current_output(outputs.begin()) + { + } + String getName() const override { return "StrictResize"; } Status prepare(const PortNumbers &, const PortNumbers &) override; diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index ae342abeea5..224dd9e5145 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -770,7 +770,7 @@ void Pipe::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) header.clear(); } -void Pipe::transform(const Transformer & transformer) +void Pipe::transform(const Transformer & transformer, bool check_ports) { if (output_ports.empty()) throw Exception("Cannot transform empty Pipe", ErrorCodes::LOGICAL_ERROR); @@ -799,6 +799,9 @@ void Pipe::transform(const Transformer & transformer) { for (const auto & port : processor->getInputs()) { + if (!check_ports) + break; + if (!port.isConnected()) throw Exception( ErrorCodes::LOGICAL_ERROR, diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index 664ab719b13..79d19a18193 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -91,7 +91,7 @@ public: using Transformer = std::function; /// Transform Pipe in general way. - void transform(const Transformer & transformer); + void transform(const Transformer & transformer, bool check_ports = true); /// Unite several pipes together. They should have same header. static Pipe unitePipes(Pipes pipes); diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index a7db7442cba..877c19a493e 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -159,10 +159,10 @@ void QueryPipelineBuilder::addChain(Chain chain) pipe.addChains(std::move(chains)); } -void QueryPipelineBuilder::transform(const Transformer & transformer) +void QueryPipelineBuilder::transform(const Transformer & transformer, bool check_ports) { checkInitializedAndNotCompleted(); - pipe.transform(transformer); + pipe.transform(transformer, check_ports); } void QueryPipelineBuilder::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 100a2e07341..4edae83fe86 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -69,7 +69,7 @@ public: using Transformer = std::function; /// Transform pipeline in general way. - void transform(const Transformer & transformer); + void transform(const Transformer & transformer, bool check_ports = true); /// Add TotalsHavingTransform. Resize pipeline to single input. Adds totals port. void addTotalsHavingTransform(ProcessorPtr transform); From 31a167848df68b6a84bf1e6e5c150a6543b51b28 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 1 Aug 2022 14:03:57 +0000 Subject: [PATCH 10/39] Fix set finish condition in CreatingSetsOnTheFlyTransform --- .../CreatingSetsOnTheFlyTransform.cpp | 40 ++++++++++++------- 1 file changed, 26 insertions(+), 14 deletions(-) diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp index d52a669ee75..b3a3410dd9e 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp @@ -64,6 +64,29 @@ CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform(const Block & heade IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() { auto status = ISimpleTransform::prepare(); + + if (status == Status::Finished && set) + { + if (input.isFinished()) + { + set->finishInsert(); + set->state = SetWithState::State::Finished; + LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", + getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + } + else + { + /// Should not happen because processor places before join that reads all the data + /// But let's hanlde this case just for safety. + set->state = SetWithState::State::Suspended; + LOG_DEBUG(log, "{}: Processor finished, but not all input was read, cancelling building set after using {}", + getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); + } + + /// Release pointer to make it possible destroy it by consumer + set.reset(); + } + return status; } @@ -74,28 +97,17 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) if (chunk.getNumRows()) { - Columns key_cols = getColumnsByIndices(chunk, key_column_indices); - bool limit_exceeded = !set->insertFromBlock(key_cols); + Columns key_columns = getColumnsByIndices(chunk, key_column_indices); + bool limit_exceeded = !set->insertFromBlock(key_columns); if (limit_exceeded) { LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after using {}", getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); - // set->clear(); + // TODO(@vdimir): set->clear() ? set->state = SetWithState::State::Suspended; set.reset(); } } - - if (input.isFinished()) - { - set->finishInsert(); - set->state = SetWithState::State::Finished; - LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", - getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); - - /// Release pointer to make it possible destroy it by consumer - set.reset(); - } } FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_, const Names & column_names_, SetWithStatePtr set_) From c778bba13f18d555b417afd250841f30a3ed9cd2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 2 Aug 2022 13:46:02 +0000 Subject: [PATCH 11/39] Create sets for joins: wip --- src/Core/Settings.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 13 +- src/Processors/DelayedPortsProcessor.cpp | 166 ++++++++++++++---- src/Processors/DelayedPortsProcessor.h | 67 ++++--- src/Processors/Port.cpp | 10 +- src/Processors/Port.h | 8 +- .../QueryPlan/CreatingSetOnTheFlyStep.cpp | 58 ++++-- .../QueryPlan/CreatingSetOnTheFlyStep.h | 30 +++- .../CreatingSetsOnTheFlyTransform.cpp | 41 +++-- .../CreatingSetsOnTheFlyTransform.h | 9 +- src/QueryPipeline/Pipe.cpp | 7 +- 11 files changed, 301 insertions(+), 110 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 60f588f1618..8a81a8b0b29 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -366,7 +366,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \ - M(UInt64, max_bytes_in_set_to_optimize_join, 1_GiB, "Maximal size of the set to filter joined tables by each other row sets before joining.", 0) \ + M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining (-1 to disable).", 0) \ \ M(Bool, compatibility_ignore_collation_in_create_table, true, "Compatibility ignore collation in create table", 0) \ \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 74e25562d29..f16c81e93b1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1448,7 +1448,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(plan.getCurrentDataStream(), key_names, size_limits); creating_set_step->setStepDescription(fmt::format("Create set for {} stream", is_right ? "right" : "left")); auto set = creating_set_step->getSet(); @@ -1456,9 +1456,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(plan.getCurrentDataStream(), key_names, set, ports_state); + auto filter_by_set_step = std::make_unique( + plan.getCurrentDataStream(), rhs_header, key_names, settings.max_rows_in_set_to_optimize_join, set, ports_state); filter_by_set_step->setStepDescription(fmt::format("Filter {} stream by set", is_right ? "right" : "left")); plan.addStep(std::move(filter_by_set_step)); }; @@ -1467,14 +1468,14 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

getTableJoin().getOnlyClause(); - if (settings.max_bytes_in_set_to_optimize_join > 0) + if (settings.max_rows_in_set_to_optimize_join > 0) { auto left_set = add_create_set(query_plan, join_clause.key_names_left, false); auto right_set = add_create_set(*joined_plan, join_clause.key_names_right, true); auto ports_state = std::make_shared(); - add_filter_by_set(query_plan, join_clause.key_names_left, right_set, ports_state, false); - add_filter_by_set(*joined_plan, join_clause.key_names_right, left_set, ports_state, true); + add_filter_by_set(query_plan, joined_plan->getCurrentDataStream().header, join_clause.key_names_left, right_set, ports_state, false); + add_filter_by_set(*joined_plan, query_plan.getCurrentDataStream().header, join_clause.key_names_right, left_set, ports_state, true); } add_sorting(query_plan, join_clause.key_names_left, false); diff --git a/src/Processors/DelayedPortsProcessor.cpp b/src/Processors/DelayedPortsProcessor.cpp index 48930ca9e51..9a7855a3635 100644 --- a/src/Processors/DelayedPortsProcessor.cpp +++ b/src/Processors/DelayedPortsProcessor.cpp @@ -1,6 +1,8 @@ #include #include +#include +#include "Processors/Port.h" namespace DB @@ -170,12 +172,21 @@ IProcessor::Status DelayedPortsProcessor::prepare(const PortNumbers & updated_in return Status::PortFull; } - -NotifyProcessor::NotifyProcessor(const Block & header, size_t num_ports) - : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)) - , aux_in_port(Block(), this) - , aux_out_port(Block(), this) +static InputPorts createPortsList(const Block & header, const Block & last_header, size_t num_ports) { + InputPorts res(num_ports, header); + res.emplace_back(last_header); + return res; +} + +NotifyProcessor::NotifyProcessor(const Block & header, const Block & aux_header, size_t num_ports, StatePtr sync_state_) + : IProcessor(createPortsList(header, aux_header, num_ports), OutputPorts(num_ports + 1, header)) + , aux_in_port(inputs.back()) + , aux_out_port(outputs.back()) + , sync_state(sync_state_) + , idx(sync_state->idx++) +{ + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {}: idx {}", __FILE__, __LINE__, getDescription(), idx); port_pairs.resize(num_ports); auto input_it = inputs.begin(); @@ -188,7 +199,6 @@ NotifyProcessor::NotifyProcessor(const Block & header, size_t num_ports) port_pairs[i].output_port = &*output_it; ++output_it; } - } void NotifyProcessor::finishPair(PortsPair & pair) @@ -234,16 +244,38 @@ bool NotifyProcessor::processPair(PortsPair & pair) return true; } +bool NotifyProcessor::isPairsFinished() const +{ + return num_finished_pairs == port_pairs.size(); +} + IProcessor::Status NotifyProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) { + if (isPairsFinished()) + return Status::Finished; + bool need_data = false; - for (const auto & output_number : updated_outputs) - need_data = processPair(port_pairs[output_number]) || need_data; - for (const auto & input_number : updated_inputs) - need_data = processPair(port_pairs[input_number]) || need_data; + UNUSED(updated_inputs); + UNUSED(updated_outputs); - if (num_finished_pairs == port_pairs.size()) + // for (const auto & output_number : updated_outputs) + for (size_t output_number = 0; output_number < port_pairs.size(); ++output_number) + { + if (output_number >= port_pairs.size()) + continue; /// skip auxiliary port + need_data = processPair(port_pairs[output_number]) || need_data; + } + + // for (const auto & input_number : updated_inputs) + for (size_t input_number = 0; input_number < port_pairs.size(); ++input_number) + { + if (input_number >= port_pairs.size()) + continue; /// skip auxiliary port + need_data = processPair(port_pairs[input_number]) || need_data; + } + + if (isPairsFinished()) return Status::Finished; if (need_data) @@ -252,44 +284,102 @@ IProcessor::Status NotifyProcessor::processRegularPorts(const PortNumbers & upda return Status::PortFull; } +void NotifyProcessor::work() +{ + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} work {}", __FILE__, __LINE__, getDescription()); +} + +bool NotifyProcessor::sendPing() +{ + if (aux_out_port.canPush()) + { + Chunk chunk(aux_out_port.getHeader().cloneEmpty().getColumns(), 0); + aux_out_port.push(std::move(chunk)); + is_send = true; + aux_out_port.finish(); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} sendPing OK {} ({})", __FILE__, __LINE__, idx, log()); + return true; + } + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} sendPing NA {} ({})", __FILE__, __LINE__, idx, log()); + return false; +} + +bool NotifyProcessor::recievePing() +{ + if (aux_in_port.hasData()) + { + aux_in_port.pull(); + is_recieved = true; + aux_in_port.close(); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} recievePing OK {} ({})", __FILE__, __LINE__, idx, log()); + return true; + } + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} recievePing NA {} ({})", __FILE__, __LINE__, idx, log()); + return false; +} + + IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) { - auto status = processRegularPorts(updated_inputs, updated_outputs); - if (status != Status::Ready) - return status; - - if (ready == AuxPortState::NotInitialized && isReady()) - ready = AuxPortState::Triggered; - - if (ready == AuxPortState::Triggered) - { - if (aux_out_port.canPush()) - { - aux_out_port.push({}); - ready = AuxPortState::Finished; - return Status::Ready; - } - return Status::PortFull; - } - - if (waiting == AuxPortState::NotInitialized && isWaiting()) + if (!set_needed_once && !is_recieved && !aux_in_port.isFinished()) { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} set_needed_once {}: {}", __FILE__, __LINE__, getDescription(), idx); + set_needed_once = true; aux_in_port.setNeeded(); - waiting = AuxPortState::Triggered; } - if (waiting == AuxPortState::Triggered) + if (idx == 0 || is_send) { - if (aux_in_port.hasData()) + if (!is_recieved) { - aux_in_port.pull(true); - waiting = AuxPortState::Finished; - return Status::Ready; + bool recieved = recievePing(); + if (!recieved) + { + return Status::NeedData; + } } - return Status::PortFull; } - return Status::Ready; + if (idx == 1 || is_recieved) + { + if (!is_send && canSend()) + { + bool sent = sendPing(); + if (!sent) + return Status::PortFull; + } + } + + auto status = processRegularPorts(updated_inputs, updated_outputs); + if (status == Status::Finished) + { + if (idx == 0 || is_send) + { + if (!is_recieved) + { + bool recieved = recievePing(); + if (!recieved) + { + return Status::NeedData; + } + } + } + + if (idx == 1 || is_recieved) + { + if (!is_send && canSend()) + { + bool sent = sendPing(); + if (!sent) + return Status::PortFull; + } + } + } + if (status == Status::PortFull) + { + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} status {}", __FILE__, __LINE__, status); + } + return status; } std::pair NotifyProcessor::getAuxPorts() diff --git a/src/Processors/DelayedPortsProcessor.h b/src/Processors/DelayedPortsProcessor.h index a5314ca36a1..8b9c44bad29 100644 --- a/src/Processors/DelayedPortsProcessor.h +++ b/src/Processors/DelayedPortsProcessor.h @@ -2,6 +2,8 @@ #include #include #include +#include + namespace DB { @@ -45,28 +47,29 @@ private: class NotifyProcessor : public IProcessor { public: - NotifyProcessor(const Block & header, size_t num_ports); + struct State + { + std::atomic_bool waiting = false; + std::atomic_bool can_push = false; + std::atomic_size_t idx = 0; + }; + using StatePtr = std::shared_ptr; + + NotifyProcessor(const Block & header, const Block & aux_header, size_t num_ports, StatePtr sync_state_); String getName() const override { return "NotifyProcessor"; } Status prepare(const PortNumbers &, const PortNumbers &) override; + void work() override; std::pair getAuxPorts(); - virtual bool isReady() const { return true; } - virtual bool isWaiting() const { return false; } + + virtual bool canSend() const = 0; virtual void dataCallback(const Chunk & chunk) { UNUSED(chunk); } -private: - - enum class AuxPortState - { - NotInitialized, - Triggered, - Finished, - }; - +protected: struct PortsPair { InputPort * input_port = nullptr; @@ -74,6 +77,11 @@ private: bool is_finished = false; }; + bool sendPing(); + bool recievePing(); + virtual String log() = 0; + + bool isPairsFinished() const; bool processPair(PortsPair & pair); void finishPair(PortsPair & pair); Status processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs); @@ -81,36 +89,49 @@ private: std::vector port_pairs; size_t num_finished_pairs = 0; - InputPort aux_in_port; - OutputPort aux_out_port; + InputPort & aux_in_port; + OutputPort & aux_out_port; - AuxPortState ready = AuxPortState::NotInitialized; - AuxPortState waiting = AuxPortState::NotInitialized; + bool is_send = false; + bool is_recieved = false; + + bool set_needed_once = false; + StatePtr sync_state; + + size_t idx; }; class NotifyProcessor2 : public NotifyProcessor { public: - using NotifyProcessor::NotifyProcessor; - - bool isReady() const override + NotifyProcessor2(const Block & header, const Block & aux_header, size_t num_ports, size_t size_, NotifyProcessor::StatePtr sync_state_) + : NotifyProcessor(header, aux_header, num_ports, sync_state_) + , size(size_) { - return data_consumed > 10_MiB; } - bool isWaiting() const override + bool canSend() const override { - return data_consumed < 10_MiB; + return isPairsFinished() || data_consumed > size; } + void dataCallback(const Chunk & chunk) override { - data_consumed += chunk.allocatedBytes(); + data_consumed += chunk.getNumRows(); + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {}: data_consumed {}", __FILE__, __LINE__, getDescription(), data_consumed); + } + + String log() override + { + return fmt::format("data {} / {} = {:.2f}", data_consumed, size, data_consumed / float(size)); } private: size_t data_consumed = 0; + + size_t size; }; } diff --git a/src/Processors/Port.cpp b/src/Processors/Port.cpp index a856dbcea3b..79532dd4d6c 100644 --- a/src/Processors/Port.cpp +++ b/src/Processors/Port.cpp @@ -8,16 +8,16 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void connect(OutputPort & output, InputPort & input) +void connect(OutputPort & output, InputPort & input, bool reconnect) { - if (input.state) + if (!reconnect && input.state) throw Exception(ErrorCodes::LOGICAL_ERROR, "Port is already connected, (header: [{}])", input.header.dumpStructure()); - if (output.state) + if (!reconnect && output.state) throw Exception(ErrorCodes::LOGICAL_ERROR, "Port is already connected, (header: [{}])", output.header.dumpStructure()); - auto out_name = output.getProcessor().getName(); - auto in_name = input.getProcessor().getName(); + auto out_name = output.processor ? output.getProcessor().getName() : "null"; + auto in_name = input.processor ? input.getProcessor().getName() : "null"; assertCompatibleHeader(output.getHeader(), input.getHeader(), fmt::format("function connect between {} and {}", out_name, in_name)); diff --git a/src/Processors/Port.h b/src/Processors/Port.h index e3fb0e3e342..9163402f600 100644 --- a/src/Processors/Port.h +++ b/src/Processors/Port.h @@ -25,7 +25,7 @@ namespace ErrorCodes class Port { - friend void connect(OutputPort &, InputPort &); + friend void connect(OutputPort &, InputPort &, bool); friend class IProcessor; public: @@ -267,7 +267,7 @@ protected: /// * You can pull only if port hasData(). class InputPort : public Port { - friend void connect(OutputPort &, InputPort &); + friend void connect(OutputPort &, InputPort &, bool); private: OutputPort * output_port = nullptr; @@ -390,7 +390,7 @@ public: /// * You can push only if port doesn't hasData(). class OutputPort : public Port { - friend void connect(OutputPort &, InputPort &); + friend void connect(OutputPort &, InputPort &, bool); private: InputPort * input_port = nullptr; @@ -483,6 +483,6 @@ using InputPorts = std::list; using OutputPorts = std::list; -void connect(OutputPort & output, InputPort & input); +void connect(OutputPort & output, InputPort & input, bool reconnect = false); } diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp index 0bb363fc17d..d7d25071d1d 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp @@ -57,18 +57,17 @@ void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, { UNUSED(settings); size_t num_streams = pipeline.getNumStreams(); + // pipeline.resize(1); - pipeline.resize(1); pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { - if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - auto res = std::make_shared(header, column_names, set); + auto res = std::make_shared(header, column_names, num_streams, set); res->setDescription(this->getStepDescription()); return res; }); - pipeline.resize(num_streams); + // pipeline.resize(num_streams); } void CreatingSetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const @@ -93,10 +92,13 @@ void CreatingSetOnTheFlyStep::updateOutputStream() } -FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, +FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Block & rhs_input_stream_header_, + const Names & column_names_, size_t buffer_size_, SetWithStatePtr set_, PortsStatePtr ports_state_) : ITransformingStep(input_stream_, input_stream_.header, getTraits(true)) , column_names(column_names_) + , buffer_size(buffer_size_) + , rhs_input_stream_header(rhs_input_stream_header_.cloneEmpty()) , set(set_) , ports_state(ports_state_) { @@ -105,34 +107,61 @@ FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream } -static void connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs) +static InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) { auto input_it = inputs.begin(); - for (auto & port : ports) + for (size_t i = 0; i < num_ports; ++i) { - connect(*port, *input_it); + connect(*ports[i], *input_it); input_it++; } - assert(input_it == inputs.end()); + return input_it; } void FilterBySetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { UNUSED(settings); + UNUSED(connectAllInputs); + + UNUSED(buffer_size); Block input_header = pipeline.getHeader(); pipeline.transform([&input_header, this](OutputPortRawPtrs ports) { - size_t num_streams = ports.size(); + Processors transforms; - auto notifier = std::make_shared(input_header, num_streams); + size_t num_ports = ports.size(); - connectAllInputs(ports, notifier->getInputs()); - ports_state->tryConnectPorts(notifier->getAuxPorts()); + auto notifier = std::make_shared(input_header, rhs_input_stream_header, num_ports, buffer_size, ports_state->sync_state); + notifier->setDescription(getStepDescription()); - return Processors{notifier}; + auto input_it = connectAllInputs(ports, notifier->getInputs(), num_ports); + assert(&*input_it == notifier->getAuxPorts().first); + input_it++; + assert(input_it == notifier->getInputs().end()); + + ports_state->tryConnectPorts(notifier->getAuxPorts(), notifier.get()); + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} / {}", __FILE__, __LINE__, + notifier->getAuxPorts().first->isConnected(), notifier->getAuxPorts().second->isConnected()); + + auto & outputs = notifier->getOutputs(); + auto output_it = outputs.begin(); + for (size_t i = 0; i < outputs.size() - 1; ++i) + { + auto & port = *output_it++; + auto transform = std::make_shared(port.getHeader(), column_names, set); + transform->setDescription(this->getStepDescription()); + connect(port, transform->getInputPort()); + transforms.emplace_back(std::move(transform)); + } + output_it++; + assert(output_it == outputs.end()); + transforms.emplace_back(std::move(notifier)); + + return transforms; }, /* check_ports= */ false); + /* pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type != QueryPipelineBuilder::StreamType::Main) @@ -141,6 +170,7 @@ void FilterBySetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, res->setDescription(this->getStepDescription()); return res; }); + */ } void FilterBySetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h index 2a85d26836d..787e14c0e0e 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB @@ -37,34 +38,53 @@ public: class PortsState : public boost::noncopyable { public: + std::shared_ptr sync_state; + + explicit PortsState() + : sync_state(std::make_shared()) + { + } + using PortPair = std::pair; /// Remember ports passed on the first call and connect with ones from second call. - bool tryConnectPorts(PortPair rhs_ports) + bool tryConnectPorts(PortPair rhs_ports, IProcessor * proc) { + LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} {} / {} {}", __FILE__, __LINE__, + bool(input_port), input_port ? input_port->isConnected() : false, + bool(output_port), output_port ? output_port->isConnected() : false); std::lock_guard lock(mux); if (input_port || output_port) { assert(input_port && output_port); + assert(!input_port->isConnected()); connect(*rhs_ports.second, *input_port); - connect(*output_port, *rhs_ports.first); + connect(*output_port, *rhs_ports.first, /* reconnect= */ true); return true; } std::tie(input_port, output_port) = rhs_ports; + assert(input_port && output_port); + assert(!input_port->isConnected() && !output_port->isConnected()); + + dummy_input_port = std::make_unique(output_port->getHeader(), proc); + connect(*output_port, *dummy_input_port); return false; } - private: std::mutex mux; InputPort * input_port = nullptr; OutputPort * output_port = nullptr; + + std::unique_ptr dummy_input_port; }; using PortsStatePtr = std::shared_ptr; FilterBySetOnTheFlyStep( const DataStream & input_stream_, + const Block & rhs_input_stream_header_, const Names & column_names_, + size_t buffer_size_, SetWithStatePtr set_, PortsStatePtr ports_state_); @@ -79,6 +99,10 @@ private: Names column_names; + size_t buffer_size; + + Block rhs_input_stream_header; + SetWithStatePtr set; PortsStatePtr ports_state; }; diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp index b3a3410dd9e..1f9282d8baa 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -52,10 +53,12 @@ std::string formatBytesHumanReadable(size_t bytes) } -CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform(const Block & header_, const Names & column_names_, SetWithStatePtr set_) +CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform( + const Block & header_, const Names & column_names_, size_t num_streams_, SetWithStatePtr set_) : ISimpleTransform(header_, header_, true) , column_names(column_names_) , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) + , num_streams(num_streams_) , set(set_) , log(&Poco::Logger::get(getName())) { @@ -65,14 +68,18 @@ IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() { auto status = ISimpleTransform::prepare(); - if (status == Status::Finished && set) + if (status == Status::Finished && set && set->state == SetWithState::State::Creating) { if (input.isFinished()) { - set->finishInsert(); - set->state = SetWithState::State::Finished; - LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", - getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + set->finished_count++; + if (set->finished_count == num_streams) + { + set->finishInsert(); + set->state = SetWithState::State::Finished; + LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", + getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + } } else { @@ -82,29 +89,38 @@ IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() LOG_DEBUG(log, "{}: Processor finished, but not all input was read, cancelling building set after using {}", getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); } + } + if (status == Status::Finished && set && set->state != SetWithState::State::Creating) /// Release pointer to make it possible destroy it by consumer set.reset(); - } return status; } void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) { - if (!set) + if (!set || set->state != SetWithState::State::Creating) + { + if (set) + set.reset(); return; + } if (chunk.getNumRows()) { Columns key_columns = getColumnsByIndices(chunk, key_column_indices); + size_t prev_size = set->getTotalByteCount(); bool limit_exceeded = !set->insertFromBlock(key_columns); if (limit_exceeded) { - LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after using {}", - getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); + auto prev_state = set->state.exchange(SetWithState::State::Suspended); + if (prev_state == SetWithState::State::Creating) + { + LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after using {} ({} -> {} bytes)", + getDescription(), formatBytesHumanReadable(set->getTotalByteCount()), prev_size, set->getTotalByteCount()); + } // TODO(@vdimir): set->clear() ? - set->state = SetWithState::State::Suspended; set.reset(); } } @@ -150,8 +166,9 @@ void FilterBySetOnTheFlyTransform::transform(Chunk & chunk) { stat.consumed_rows += chunk.getNumRows(); stat.result_rows += chunk.getNumRows(); - bool can_filter = set && set->state == SetWithState::State::Finished; + bool can_filter = set && set->state == SetWithState::State::Finished; + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} / {} / {}", __FILE__, __LINE__, set->finished_count.load(), set->state.load(), set->getTotalRowCount()); if (!can_filter) stat.consumed_rows_before_set += chunk.getNumRows(); diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h index ea91a96818a..072095cc3db 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h +++ b/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h @@ -1,4 +1,6 @@ #pragma once +#include +#include #include #include #include @@ -21,6 +23,7 @@ public: }; std::atomic state = State::Creating; + std::atomic_size_t finished_count = 0; }; using SetWithStatePtr = std::shared_ptr; @@ -35,7 +38,7 @@ class CreatingSetsOnTheFlyTransform : public ISimpleTransform { public: explicit CreatingSetsOnTheFlyTransform( - const Block & header_, const Names & column_names_, SetWithStatePtr set_); + const Block & header_, const Names & column_names_, size_t num_streams_, SetWithStatePtr set_); String getName() const override { return "CreatingSetsOnTheFlyTransform"; } @@ -47,6 +50,8 @@ private: Names column_names; std::vector key_column_indices; + size_t num_streams; + /// Set to fill SetWithStatePtr set; @@ -56,7 +61,7 @@ private: /* * Filter the input chunk by the set. - * When set building is not comleted, just return the source data. + * When set building is not completed, just return the source data. */ class FilterBySetOnTheFlyTransform : public ISimpleTransform { diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index 224dd9e5145..291739079a2 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -784,6 +784,9 @@ void Pipe::transform(const Transformer & transformer, bool check_ports) for (const auto & port : output_ports) { + if (!check_ports) + break; + if (!port->isConnected()) throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -809,7 +812,7 @@ void Pipe::transform(const Transformer & transformer, bool check_ports) processor->getName()); const auto * connected_processor = &port.getOutputPort().getProcessor(); - if (!set.contains(connected_processor)) + if (check_ports && !set.contains(connected_processor)) throw Exception( ErrorCodes::LOGICAL_ERROR, "Transformation of Pipe is not valid because processor {} has input port which is connected with unknown processor {}", @@ -826,7 +829,7 @@ void Pipe::transform(const Transformer & transformer, bool check_ports) } const auto * connected_processor = &port.getInputPort().getProcessor(); - if (!set.contains(connected_processor)) + if (check_ports && !set.contains(connected_processor)) throw Exception( ErrorCodes::LOGICAL_ERROR, "Transformation of Pipe is not valid because processor {} has output port which is connected with unknown processor {}", From 51a51694d6030fc2ac289be992638133fc89d8ba Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 11:08:25 +0000 Subject: [PATCH 12/39] Create sets for joins: better code --- src/Interpreters/InterpreterSelectQuery.cpp | 44 ++-- src/Processors/DelayedPortsProcessor.cpp | 49 ++-- src/Processors/DelayedPortsProcessor.h | 75 +++--- .../QueryPlan/CreatingSetOnTheFlyStep.cpp | 243 +++++++++--------- .../QueryPlan/CreatingSetOnTheFlyStep.h | 98 ++----- .../Optimizations/filterPushDown.cpp | 3 - 6 files changed, 223 insertions(+), 289 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f16c81e93b1..212d112d503 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1423,7 +1423,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

context->getTemporaryVolume(), settings.min_free_disk_space_for_temporary_data, settings.optimize_sorting_by_input_stream_properties); - sorting_step->setStepDescription(fmt::format("Sort {} before JOIN", is_right ? "right" : "left")); + sorting_step->setStepDescription(fmt::format("Sort {} before JOIN", join_pos)); plan.addStep(std::move(sorting_step)); }; - auto add_create_set = [&settings](QueryPlan & plan, const Names & key_names, bool is_right) + auto crosswise_connection = CreatingSetOnTheFlyStep::createCrossConnection(); + auto add_create_set = [&settings, crosswise_connection](QueryPlan & plan, const DataStream & rhs_data_stream, const Names & key_names, JoinTableSide join_pos) { - SizeLimits size_limits(settings.max_rows_in_set_to_optimize_join, 0, OverflowMode::BREAK); - auto creating_set_step = std::make_unique(plan.getCurrentDataStream(), key_names, size_limits); - creating_set_step->setStepDescription(fmt::format("Create set for {} stream", is_right ? "right" : "left")); - auto set = creating_set_step->getSet(); - plan.addStep(std::move(creating_set_step)); - return set; - }; + size_t max_rows = std::max(100, settings.max_rows_in_set_to_optimize_join); - auto add_filter_by_set = [&settings](QueryPlan & plan, const Block & rhs_header, const Names & key_names, auto set, auto ports_state, bool is_right) - { - auto filter_by_set_step = std::make_unique( - plan.getCurrentDataStream(), rhs_header, key_names, settings.max_rows_in_set_to_optimize_join, set, ports_state); - filter_by_set_step->setStepDescription(fmt::format("Filter {} stream by set", is_right ? "right" : "left")); - plan.addStep(std::move(filter_by_set_step)); + auto creating_set_step = std::make_unique( + plan.getCurrentDataStream(), rhs_data_stream, key_names, max_rows, crosswise_connection, join_pos); + creating_set_step->setStepDescription(fmt::format("Create set and filter {} joined stream", join_pos)); + + auto * step_raw_ptr = creating_set_step.get(); + plan.addStep(std::move(creating_set_step)); + return step_raw_ptr; }; if (expressions.join->pipelineType() == JoinPipelineType::YShaped) { const auto & join_clause = expressions.join->getTableJoin().getOnlyClause(); - if (settings.max_rows_in_set_to_optimize_join > 0) + if (settings.max_rows_in_set_to_optimize_join >= 0) { - auto left_set = add_create_set(query_plan, join_clause.key_names_left, false); - auto right_set = add_create_set(*joined_plan, join_clause.key_names_right, true); - - auto ports_state = std::make_shared(); - add_filter_by_set(query_plan, joined_plan->getCurrentDataStream().header, join_clause.key_names_left, right_set, ports_state, false); - add_filter_by_set(*joined_plan, query_plan.getCurrentDataStream().header, join_clause.key_names_right, left_set, ports_state, true); + auto * left_set = add_create_set(query_plan, joined_plan->getCurrentDataStream(), join_clause.key_names_left, JoinTableSide::Left); + auto * right_set = add_create_set(*joined_plan, query_plan.getCurrentDataStream(), join_clause.key_names_right, JoinTableSide::Right); + left_set->setFiltering(right_set->getSet()); + right_set->setFiltering(left_set->getSet()); } - add_sorting(query_plan, join_clause.key_names_left, false); - add_sorting(*joined_plan, join_clause.key_names_right, true); + add_sorting(query_plan, join_clause.key_names_left, JoinTableSide::Left); + add_sorting(*joined_plan, join_clause.key_names_right, JoinTableSide::Right); } QueryPlanStepPtr join_step = std::make_unique( diff --git a/src/Processors/DelayedPortsProcessor.cpp b/src/Processors/DelayedPortsProcessor.cpp index 9a7855a3635..a4b903ec62b 100644 --- a/src/Processors/DelayedPortsProcessor.cpp +++ b/src/Processors/DelayedPortsProcessor.cpp @@ -179,14 +179,14 @@ static InputPorts createPortsList(const Block & header, const Block & last_heade return res; } -NotifyProcessor::NotifyProcessor(const Block & header, const Block & aux_header, size_t num_ports, StatePtr sync_state_) +PingPongProcessor::PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_) : IProcessor(createPortsList(header, aux_header, num_ports), OutputPorts(num_ports + 1, header)) , aux_in_port(inputs.back()) , aux_out_port(outputs.back()) - , sync_state(sync_state_) - , idx(sync_state->idx++) + , order(order_) { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {}: idx {}", __FILE__, __LINE__, getDescription(), idx); + assert(order == First || order == Second); + port_pairs.resize(num_ports); auto input_it = inputs.begin(); @@ -201,7 +201,7 @@ NotifyProcessor::NotifyProcessor(const Block & header, const Block & aux_header, } } -void NotifyProcessor::finishPair(PortsPair & pair) +void PingPongProcessor::finishPair(PortsPair & pair) { if (!pair.is_finished) { @@ -213,7 +213,7 @@ void NotifyProcessor::finishPair(PortsPair & pair) } } -bool NotifyProcessor::processPair(PortsPair & pair) +bool PingPongProcessor::processPair(PortsPair & pair) { if (pair.output_port->isFinished()) { @@ -237,19 +237,19 @@ bool NotifyProcessor::processPair(PortsPair & pair) if (pair.input_port->hasData()) { Chunk chunk = pair.input_port->pull(true); - dataCallback(chunk); + ready_to_send = isReady(chunk) || ready_to_send; pair.output_port->push(std::move(chunk)); } return true; } -bool NotifyProcessor::isPairsFinished() const +bool PingPongProcessor::isPairsFinished() const { return num_finished_pairs == port_pairs.size(); } -IProcessor::Status NotifyProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +IProcessor::Status PingPongProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) { if (isPairsFinished()) return Status::Finished; @@ -284,12 +284,7 @@ IProcessor::Status NotifyProcessor::processRegularPorts(const PortNumbers & upda return Status::PortFull; } -void NotifyProcessor::work() -{ - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} work {}", __FILE__, __LINE__, getDescription()); -} - -bool NotifyProcessor::sendPing() +bool PingPongProcessor::sendPing() { if (aux_out_port.canPush()) { @@ -297,38 +292,37 @@ bool NotifyProcessor::sendPing() aux_out_port.push(std::move(chunk)); is_send = true; aux_out_port.finish(); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} sendPing OK {} ({})", __FILE__, __LINE__, idx, log()); return true; } - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} sendPing NA {} ({})", __FILE__, __LINE__, idx, log()); return false; } -bool NotifyProcessor::recievePing() +bool PingPongProcessor::recievePing() { if (aux_in_port.hasData()) { aux_in_port.pull(); is_recieved = true; aux_in_port.close(); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} recievePing OK {} ({})", __FILE__, __LINE__, idx, log()); return true; } - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} recievePing NA {} ({})", __FILE__, __LINE__, idx, log()); return false; } +bool PingPongProcessor::canSend() const +{ + return !is_send && (ready_to_send || isPairsFinished()); +} -IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +IProcessor::Status PingPongProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) { if (!set_needed_once && !is_recieved && !aux_in_port.isFinished()) { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} set_needed_once {}: {}", __FILE__, __LINE__, getDescription(), idx); set_needed_once = true; aux_in_port.setNeeded(); } - if (idx == 0 || is_send) + if (order == First || is_send) { if (!is_recieved) { @@ -340,7 +334,7 @@ IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, } } - if (idx == 1 || is_recieved) + if (order == Second || is_recieved) { if (!is_send && canSend()) { @@ -353,7 +347,7 @@ IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, auto status = processRegularPorts(updated_inputs, updated_outputs); if (status == Status::Finished) { - if (idx == 0 || is_send) + if (order == First || is_send) { if (!is_recieved) { @@ -365,7 +359,7 @@ IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, } } - if (idx == 1 || is_recieved) + if (order == Second || is_recieved) { if (!is_send && canSend()) { @@ -382,10 +376,9 @@ IProcessor::Status NotifyProcessor::prepare(const PortNumbers & updated_inputs, return status; } -std::pair NotifyProcessor::getAuxPorts() +std::pair PingPongProcessor::getAuxPorts() { return std::make_pair(&aux_in_port, &aux_out_port); } - } diff --git a/src/Processors/DelayedPortsProcessor.h b/src/Processors/DelayedPortsProcessor.h index 8b9c44bad29..b36fa7c3877 100644 --- a/src/Processors/DelayedPortsProcessor.h +++ b/src/Processors/DelayedPortsProcessor.h @@ -44,30 +44,47 @@ private: }; -class NotifyProcessor : public IProcessor +/* + * Processor with N inputs and N outputs. Moves data from i-th input to i-th output as is. + * It has a pair of auxiliary ports to notify another instance by sending empty chunk after some condition holds. + * You should use this processor in pair of instances and connect auxiliary ports crosswise. + * + * ╭─┴───┴───┴───┴───┴─╮ ╭─┴───┴───┴───┴───┴─╮ + * │ ├─ aux ⟶│ │ + * │ PingPongProcessor │ │ PingPongProcessor │ + * │ │⟵ aux ─┤ │ + * ╰─┬───┬───┬───┬───┬─╯ ╰─┬───┬───┬───┬───┬─╯ + * + * One of the processors starts processing data, and another waits for notification. + * When `isReady` returns true, the first stops processing, sends a ping to another and waits for notification. + * After that, the second one also processes data until `isReady`, then send a notification back to the first one. + * After this roundtrip, processors bypass data from regular inputs to outputs. + */ +class PingPongProcessor : public IProcessor { public: - struct State + enum class Order : uint8_t { - std::atomic_bool waiting = false; - std::atomic_bool can_push = false; - std::atomic_size_t idx = 0; + /// Processor that starts processing data. + First, + /// Processor that waits for notification. + Second, }; - using StatePtr = std::shared_ptr; - NotifyProcessor(const Block & header, const Block & aux_header, size_t num_ports, StatePtr sync_state_); + using enum Order; - String getName() const override { return "NotifyProcessor"; } + /// The `aux_header` is a header from another instance of procssor. + /// It's required because all outputs should have the same structure. + /// We don't care about structure of another processor, because we send just empty chunk, but need to follow the contract. + PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_); + + String getName() const override { return "PingPongProcessor"; } Status prepare(const PortNumbers &, const PortNumbers &) override; - void work() override; std::pair getAuxPorts(); - - virtual bool canSend() const = 0; - - virtual void dataCallback(const Chunk & chunk) { UNUSED(chunk); } + virtual bool isReady(const Chunk & chunk) = 0; protected: struct PortsPair @@ -79,7 +96,7 @@ protected: bool sendPing(); bool recievePing(); - virtual String log() = 0; + bool canSend() const; bool isPairsFinished() const; bool processPair(PortsPair & pair); @@ -95,37 +112,27 @@ protected: bool is_send = false; bool is_recieved = false; - bool set_needed_once = false; - StatePtr sync_state; + bool ready_to_send = false; - size_t idx; + bool set_needed_once = false; + + Order order; }; - -class NotifyProcessor2 : public NotifyProcessor +/// Reads first N rows from two streams evenly. +class ReadHeadBalancedProceesor : public PingPongProcessor { public: - NotifyProcessor2(const Block & header, const Block & aux_header, size_t num_ports, size_t size_, NotifyProcessor::StatePtr sync_state_) - : NotifyProcessor(header, aux_header, num_ports, sync_state_) + ReadHeadBalancedProceesor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_, Order order_) + : PingPongProcessor(header, aux_header, num_ports, order_) , size(size_) { } - bool canSend() const override - { - return isPairsFinished() || data_consumed > size; - } - - - void dataCallback(const Chunk & chunk) override + bool isReady(const Chunk & chunk) override { data_consumed += chunk.getNumRows(); - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {}: data_consumed {}", __FILE__, __LINE__, getDescription(), data_consumed); - } - - String log() override - { - return fmt::format("data {} / {} = {:.2f}", data_consumed, size, data_consumed / float(size)); + return data_consumed > size; } private: diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp index d7d25071d1d..a56b53a6559 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp @@ -17,6 +17,23 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } + +namespace +{ + +InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) +{ + auto input_it = inputs.begin(); + for (size_t i = 0; i < num_ports; ++i) + { + connect(*ports[i], *input_it); + input_it++; + } + return input_it; +} + +} + static ITransformingStep::Traits getTraits(bool is_filter) { return ITransformingStep::Traits @@ -33,41 +50,126 @@ static ITransformingStep::Traits getTraits(bool is_filter) }; } -CreatingSetOnTheFlyStep::CreatingSetOnTheFlyStep(const DataStream & input_stream_, const Names & column_names_, const SizeLimits & size_limits) + +class CreatingSetOnTheFlyStep::CrosswiseConnection : public boost::noncopyable +{ +public: + using PortPair = std::pair; + + /// Remember ports passed on the first call and connect with ones from second call. + bool tryConnectPorts(PortPair rhs_ports, IProcessor * proc) + { + std::lock_guard lock(mux); + if (input_port || output_port) + { + assert(input_port && output_port); + assert(!input_port->isConnected()); + connect(*rhs_ports.second, *input_port); + connect(*output_port, *rhs_ports.first, /* reconnect= */ true); + return true; + } + std::tie(input_port, output_port) = rhs_ports; + assert(input_port && output_port); + assert(!input_port->isConnected() && !output_port->isConnected()); + + dummy_input_port = std::make_unique(output_port->getHeader(), proc); + connect(*output_port, *dummy_input_port); + return false; + } + +private: + std::mutex mux; + InputPort * input_port = nullptr; + OutputPort * output_port = nullptr; + + std::unique_ptr dummy_input_port; +}; + +CreatingSetOnTheFlyStep::CrosswiseConnectionPtr CreatingSetOnTheFlyStep::createCrossConnection() +{ + return std::make_shared(); +} + +CreatingSetOnTheFlyStep::CreatingSetOnTheFlyStep( + const DataStream & input_stream_, + const DataStream & rhs_input_stream_, + const Names & column_names_, + size_t max_rows_, + CrosswiseConnectionPtr crosswise_connection_, + JoinTableSide position_) : ITransformingStep(input_stream_, input_stream_.header, getTraits(false)) , column_names(column_names_) + , max_rows(max_rows_) + , rhs_input_stream_header(rhs_input_stream_.header) + , own_set(std::make_shared(SizeLimits(max_rows, 0, OverflowMode::BREAK), false, true)) + , filtering_set(nullptr) + , crosswise_connection(crosswise_connection_) + , position(position_) { + if (crosswise_connection == nullptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Crosswise connection is not initialized"); + if (input_streams.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); - set = std::make_shared(size_limits, false, true); - - { - ColumnsWithTypeAndName header; - for (const auto & name : column_names) - { - ColumnWithTypeAndName column = input_streams[0].header.getByName(name); - header.emplace_back(column); - } - set->setHeader(header); - } + ColumnsWithTypeAndName header; + for (const auto & name : column_names) + header.emplace_back(input_streams[0].header.getByName(name)); + own_set->setHeader(header); } -void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) +void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - UNUSED(settings); size_t num_streams = pipeline.getNumStreams(); - // pipeline.resize(1); - - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + pipeline.addSimpleTransform([this, num_streams](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - auto res = std::make_shared(header, column_names, num_streams, set); + auto res = std::make_shared(header, column_names, num_streams, own_set); res->setDescription(this->getStepDescription()); return res; }); - // pipeline.resize(num_streams); + + if (!filtering_set) + { + LOG_DEBUG(log, "filtering_set is null"); + return; + } + + Block input_header = pipeline.getHeader(); + pipeline.transform([&input_header, this](OutputPortRawPtrs ports) + { + Processors transforms; + + size_t num_ports = ports.size(); + + auto idx = position == JoinTableSide::Left ? PingPongProcessor::First : PingPongProcessor::Second; + auto notifier = std::make_shared(input_header, rhs_input_stream_header, num_ports, max_rows, idx); + notifier->setDescription(getStepDescription()); + + auto input_it = connectAllInputs(ports, notifier->getInputs(), num_ports); + assert(&*input_it == notifier->getAuxPorts().first); + input_it++; + assert(input_it == notifier->getInputs().end()); + + crosswise_connection->tryConnectPorts(notifier->getAuxPorts(), notifier.get()); + + auto & outputs = notifier->getOutputs(); + auto output_it = outputs.begin(); + for (size_t i = 0; i < outputs.size() - 1; ++i) + { + auto & port = *output_it++; + auto transform = std::make_shared(port.getHeader(), column_names, filtering_set); + transform->setDescription(this->getStepDescription()); + connect(port, transform->getInputPort()); + transforms.emplace_back(std::move(transform)); + } + output_it++; + assert(output_it == outputs.end()); + transforms.emplace_back(std::move(notifier)); + + return transforms; + }, /* check_ports= */ false); } void CreatingSetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const @@ -92,107 +194,4 @@ void CreatingSetOnTheFlyStep::updateOutputStream() } -FilterBySetOnTheFlyStep::FilterBySetOnTheFlyStep(const DataStream & input_stream_, const Block & rhs_input_stream_header_, - const Names & column_names_, size_t buffer_size_, - SetWithStatePtr set_, PortsStatePtr ports_state_) - : ITransformingStep(input_stream_, input_stream_.header, getTraits(true)) - , column_names(column_names_) - , buffer_size(buffer_size_) - , rhs_input_stream_header(rhs_input_stream_header_.cloneEmpty()) - , set(set_) - , ports_state(ports_state_) -{ - if (input_streams.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); -} - - -static InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) -{ - auto input_it = inputs.begin(); - for (size_t i = 0; i < num_ports; ++i) - { - connect(*ports[i], *input_it); - input_it++; - } - return input_it; -} - -void FilterBySetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) -{ - UNUSED(settings); - UNUSED(connectAllInputs); - - UNUSED(buffer_size); - - Block input_header = pipeline.getHeader(); - pipeline.transform([&input_header, this](OutputPortRawPtrs ports) - { - Processors transforms; - - size_t num_ports = ports.size(); - - auto notifier = std::make_shared(input_header, rhs_input_stream_header, num_ports, buffer_size, ports_state->sync_state); - notifier->setDescription(getStepDescription()); - - auto input_it = connectAllInputs(ports, notifier->getInputs(), num_ports); - assert(&*input_it == notifier->getAuxPorts().first); - input_it++; - assert(input_it == notifier->getInputs().end()); - - ports_state->tryConnectPorts(notifier->getAuxPorts(), notifier.get()); - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} / {}", __FILE__, __LINE__, - notifier->getAuxPorts().first->isConnected(), notifier->getAuxPorts().second->isConnected()); - - auto & outputs = notifier->getOutputs(); - auto output_it = outputs.begin(); - for (size_t i = 0; i < outputs.size() - 1; ++i) - { - auto & port = *output_it++; - auto transform = std::make_shared(port.getHeader(), column_names, set); - transform->setDescription(this->getStepDescription()); - connect(port, transform->getInputPort()); - transforms.emplace_back(std::move(transform)); - } - output_it++; - assert(output_it == outputs.end()); - transforms.emplace_back(std::move(notifier)); - - return transforms; - }, /* check_ports= */ false); - - /* - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr - { - if (stream_type != QueryPipelineBuilder::StreamType::Main) - return nullptr; - auto res = std::make_shared(header, column_names, set); - res->setDescription(this->getStepDescription()); - return res; - }); - */ -} - -void FilterBySetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const -{ - map.add(getName(), true); -} - -void FilterBySetOnTheFlyStep::describeActions(FormatSettings & settings) const -{ - String prefix(settings.offset, ' '); - settings.out << prefix << getName(); - - settings.out << '\n'; -} - -void FilterBySetOnTheFlyStep::updateOutputStream() -{ - if (input_streams.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); - - output_stream = input_streams[0]; -} - - } diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h index 787e14c0e0e..67f31a97009 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h @@ -10,10 +10,18 @@ namespace DB class CreatingSetOnTheFlyStep : public ITransformingStep { public: - explicit CreatingSetOnTheFlyStep( + /// Two instances of step need some shared state to connect processors crosswise + class CrosswiseConnection; + using CrosswiseConnectionPtr = std::shared_ptr; + static CrosswiseConnectionPtr createCrossConnection(); + + CreatingSetOnTheFlyStep( const DataStream & input_stream_, + const DataStream & rhs_input_stream_, const Names & column_names_, - const SizeLimits & size_limits = {}); + size_t max_rows_, + CrosswiseConnectionPtr crosswise_connection_, + JoinTableSide position_); String getName() const override { return "CreatingSetsOnTheFly"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; @@ -21,90 +29,26 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; - SetWithStatePtr getSet() const { return set; } - -private: - void updateOutputStream() override; - - Names column_names; - SetWithStatePtr set; -}; - - -class FilterBySetOnTheFlyStep : public ITransformingStep -{ -public: - - class PortsState : public boost::noncopyable - { - public: - std::shared_ptr sync_state; - - explicit PortsState() - : sync_state(std::make_shared()) - { - } - - using PortPair = std::pair; - - /// Remember ports passed on the first call and connect with ones from second call. - bool tryConnectPorts(PortPair rhs_ports, IProcessor * proc) - { - LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} {} / {} {}", __FILE__, __LINE__, - bool(input_port), input_port ? input_port->isConnected() : false, - bool(output_port), output_port ? output_port->isConnected() : false); - std::lock_guard lock(mux); - if (input_port || output_port) - { - assert(input_port && output_port); - assert(!input_port->isConnected()); - connect(*rhs_ports.second, *input_port); - connect(*output_port, *rhs_ports.first, /* reconnect= */ true); - return true; - } - std::tie(input_port, output_port) = rhs_ports; - assert(input_port && output_port); - assert(!input_port->isConnected() && !output_port->isConnected()); - - dummy_input_port = std::make_unique(output_port->getHeader(), proc); - connect(*output_port, *dummy_input_port); - return false; - } - private: - std::mutex mux; - InputPort * input_port = nullptr; - OutputPort * output_port = nullptr; - - std::unique_ptr dummy_input_port; - }; - - using PortsStatePtr = std::shared_ptr; - - FilterBySetOnTheFlyStep( - const DataStream & input_stream_, - const Block & rhs_input_stream_header_, - const Names & column_names_, - size_t buffer_size_, - SetWithStatePtr set_, - PortsStatePtr ports_state_); - - String getName() const override { return "FilterBySetOnTheFly"; } - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; - - void describeActions(JSONBuilder::JSONMap & map) const override; - void describeActions(FormatSettings & settings) const override; + SetWithStatePtr getSet() const { return own_set; } + void setFiltering(SetWithStatePtr filtering_set_) { filtering_set = filtering_set_; } private: void updateOutputStream() override; Names column_names; - size_t buffer_size; + size_t max_rows; Block rhs_input_stream_header; - SetWithStatePtr set; - PortsStatePtr ports_state; + SetWithStatePtr own_set; + SetWithStatePtr filtering_set; + + CrosswiseConnectionPtr crosswise_connection; + + JoinTableSide position; + + Poco::Logger * log = &Poco::Logger::get("CreatingSetOnTheFlyStep"); }; } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 99312a17ffd..f60f8192938 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -336,9 +336,6 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) return updated_steps; - if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) - return updated_steps; - if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) return updated_steps; From 6ae15db3ad445dd6ea379948106f1b43c4472141 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 11:15:04 +0000 Subject: [PATCH 13/39] check join kind for before applying max_rows_in_set_to_optimize_join --- src/Interpreters/InterpreterSelectQuery.cpp | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 212d112d503..15ed0e7d493 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1449,6 +1449,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(100, settings.max_rows_in_set_to_optimize_join); auto creating_set_step = std::make_unique( @@ -1462,14 +1463,21 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

pipelineType() == JoinPipelineType::YShaped) { - const auto & join_clause = expressions.join->getTableJoin().getOnlyClause(); + const auto & table_join = expressions.join->getTableJoin(); + const auto & join_clause = table_join.getOnlyClause(); - if (settings.max_rows_in_set_to_optimize_join >= 0) + auto join_kind = table_join.kind(); + bool kind_allows_filtering = isInner(join_kind) || isLeft(join_kind) || isRight(join_kind); + if (settings.max_rows_in_set_to_optimize_join >= 0 && kind_allows_filtering) { auto * left_set = add_create_set(query_plan, joined_plan->getCurrentDataStream(), join_clause.key_names_left, JoinTableSide::Left); auto * right_set = add_create_set(*joined_plan, query_plan.getCurrentDataStream(), join_clause.key_names_right, JoinTableSide::Right); - left_set->setFiltering(right_set->getSet()); - right_set->setFiltering(left_set->getSet()); + + if (isInnerOrLeft(join_kind)) + right_set->setFiltering(left_set->getSet()); + + if (isInnerOrRight(join_kind)) + left_set->setFiltering(right_set->getSet()); } add_sorting(query_plan, join_clause.key_names_left, JoinTableSide::Left); From e472e13c709ee2c72b54c3a4162aa8364ae371f9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 11:20:51 +0000 Subject: [PATCH 14/39] move PingPongProcessor/ReadHeadBalancedProceesor into separate file --- src/Processors/DelayedPortsProcessor.cpp | 211 ----------------- src/Processors/DelayedPortsProcessor.h | 102 --------- src/Processors/PingPongProcessor.cpp | 215 ++++++++++++++++++ src/Processors/PingPongProcessor.h | 108 +++++++++ .../QueryPlan/CreatingSetOnTheFlyStep.cpp | 2 +- 5 files changed, 324 insertions(+), 314 deletions(-) create mode 100644 src/Processors/PingPongProcessor.cpp create mode 100644 src/Processors/PingPongProcessor.h diff --git a/src/Processors/DelayedPortsProcessor.cpp b/src/Processors/DelayedPortsProcessor.cpp index a4b903ec62b..24023529bca 100644 --- a/src/Processors/DelayedPortsProcessor.cpp +++ b/src/Processors/DelayedPortsProcessor.cpp @@ -1,8 +1,6 @@ #include #include -#include -#include "Processors/Port.h" namespace DB @@ -172,213 +170,4 @@ IProcessor::Status DelayedPortsProcessor::prepare(const PortNumbers & updated_in return Status::PortFull; } -static InputPorts createPortsList(const Block & header, const Block & last_header, size_t num_ports) -{ - InputPorts res(num_ports, header); - res.emplace_back(last_header); - return res; -} - -PingPongProcessor::PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_) - : IProcessor(createPortsList(header, aux_header, num_ports), OutputPorts(num_ports + 1, header)) - , aux_in_port(inputs.back()) - , aux_out_port(outputs.back()) - , order(order_) -{ - assert(order == First || order == Second); - - port_pairs.resize(num_ports); - - auto input_it = inputs.begin(); - auto output_it = outputs.begin(); - for (size_t i = 0; i < num_ports; ++i) - { - port_pairs[i].input_port = &*input_it; - ++input_it; - - port_pairs[i].output_port = &*output_it; - ++output_it; - } -} - -void PingPongProcessor::finishPair(PortsPair & pair) -{ - if (!pair.is_finished) - { - pair.output_port->finish(); - pair.input_port->close(); - - pair.is_finished = true; - ++num_finished_pairs; - } -} - -bool PingPongProcessor::processPair(PortsPair & pair) -{ - if (pair.output_port->isFinished()) - { - finishPair(pair); - return false; - } - - if (pair.input_port->isFinished()) - { - finishPair(pair); - return false; - } - - if (!pair.output_port->canPush()) - { - pair.input_port->setNotNeeded(); - return false; - } - - pair.input_port->setNeeded(); - if (pair.input_port->hasData()) - { - Chunk chunk = pair.input_port->pull(true); - ready_to_send = isReady(chunk) || ready_to_send; - pair.output_port->push(std::move(chunk)); - } - - return true; -} - -bool PingPongProcessor::isPairsFinished() const -{ - return num_finished_pairs == port_pairs.size(); -} - -IProcessor::Status PingPongProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) -{ - if (isPairsFinished()) - return Status::Finished; - - bool need_data = false; - - UNUSED(updated_inputs); - UNUSED(updated_outputs); - - // for (const auto & output_number : updated_outputs) - for (size_t output_number = 0; output_number < port_pairs.size(); ++output_number) - { - if (output_number >= port_pairs.size()) - continue; /// skip auxiliary port - need_data = processPair(port_pairs[output_number]) || need_data; - } - - // for (const auto & input_number : updated_inputs) - for (size_t input_number = 0; input_number < port_pairs.size(); ++input_number) - { - if (input_number >= port_pairs.size()) - continue; /// skip auxiliary port - need_data = processPair(port_pairs[input_number]) || need_data; - } - - if (isPairsFinished()) - return Status::Finished; - - if (need_data) - return Status::NeedData; - - return Status::PortFull; -} - -bool PingPongProcessor::sendPing() -{ - if (aux_out_port.canPush()) - { - Chunk chunk(aux_out_port.getHeader().cloneEmpty().getColumns(), 0); - aux_out_port.push(std::move(chunk)); - is_send = true; - aux_out_port.finish(); - return true; - } - return false; -} - -bool PingPongProcessor::recievePing() -{ - if (aux_in_port.hasData()) - { - aux_in_port.pull(); - is_recieved = true; - aux_in_port.close(); - return true; - } - return false; -} - -bool PingPongProcessor::canSend() const -{ - return !is_send && (ready_to_send || isPairsFinished()); -} - -IProcessor::Status PingPongProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) -{ - if (!set_needed_once && !is_recieved && !aux_in_port.isFinished()) - { - set_needed_once = true; - aux_in_port.setNeeded(); - } - - if (order == First || is_send) - { - if (!is_recieved) - { - bool recieved = recievePing(); - if (!recieved) - { - return Status::NeedData; - } - } - } - - if (order == Second || is_recieved) - { - if (!is_send && canSend()) - { - bool sent = sendPing(); - if (!sent) - return Status::PortFull; - } - } - - auto status = processRegularPorts(updated_inputs, updated_outputs); - if (status == Status::Finished) - { - if (order == First || is_send) - { - if (!is_recieved) - { - bool recieved = recievePing(); - if (!recieved) - { - return Status::NeedData; - } - } - } - - if (order == Second || is_recieved) - { - if (!is_send && canSend()) - { - bool sent = sendPing(); - if (!sent) - return Status::PortFull; - } - } - } - if (status == Status::PortFull) - { - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} status {}", __FILE__, __LINE__, status); - } - return status; -} - -std::pair PingPongProcessor::getAuxPorts() -{ - return std::make_pair(&aux_in_port, &aux_out_port); -} - } diff --git a/src/Processors/DelayedPortsProcessor.h b/src/Processors/DelayedPortsProcessor.h index b36fa7c3877..a6a9590e0c8 100644 --- a/src/Processors/DelayedPortsProcessor.h +++ b/src/Processors/DelayedPortsProcessor.h @@ -1,9 +1,5 @@ #pragma once #include -#include -#include -#include - namespace DB { @@ -43,102 +39,4 @@ private: void finishPair(PortsPair & pair); }; - -/* - * Processor with N inputs and N outputs. Moves data from i-th input to i-th output as is. - * It has a pair of auxiliary ports to notify another instance by sending empty chunk after some condition holds. - * You should use this processor in pair of instances and connect auxiliary ports crosswise. - * - * ╭─┴───┴───┴───┴───┴─╮ ╭─┴───┴───┴───┴───┴─╮ - * │ ├─ aux ⟶│ │ - * │ PingPongProcessor │ │ PingPongProcessor │ - * │ │⟵ aux ─┤ │ - * ╰─┬───┬───┬───┬───┬─╯ ╰─┬───┬───┬───┬───┬─╯ - * - * One of the processors starts processing data, and another waits for notification. - * When `isReady` returns true, the first stops processing, sends a ping to another and waits for notification. - * After that, the second one also processes data until `isReady`, then send a notification back to the first one. - * After this roundtrip, processors bypass data from regular inputs to outputs. - */ -class PingPongProcessor : public IProcessor -{ -public: - enum class Order : uint8_t - { - /// Processor that starts processing data. - First, - /// Processor that waits for notification. - Second, - }; - - using enum Order; - - /// The `aux_header` is a header from another instance of procssor. - /// It's required because all outputs should have the same structure. - /// We don't care about structure of another processor, because we send just empty chunk, but need to follow the contract. - PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_); - - String getName() const override { return "PingPongProcessor"; } - - Status prepare(const PortNumbers &, const PortNumbers &) override; - - std::pair getAuxPorts(); - - virtual bool isReady(const Chunk & chunk) = 0; - -protected: - struct PortsPair - { - InputPort * input_port = nullptr; - OutputPort * output_port = nullptr; - bool is_finished = false; - }; - - bool sendPing(); - bool recievePing(); - bool canSend() const; - - bool isPairsFinished() const; - bool processPair(PortsPair & pair); - void finishPair(PortsPair & pair); - Status processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs); - - std::vector port_pairs; - size_t num_finished_pairs = 0; - - InputPort & aux_in_port; - OutputPort & aux_out_port; - - bool is_send = false; - bool is_recieved = false; - - bool ready_to_send = false; - - bool set_needed_once = false; - - Order order; -}; - -/// Reads first N rows from two streams evenly. -class ReadHeadBalancedProceesor : public PingPongProcessor -{ -public: - ReadHeadBalancedProceesor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_, Order order_) - : PingPongProcessor(header, aux_header, num_ports, order_) - , size(size_) - { - } - - bool isReady(const Chunk & chunk) override - { - data_consumed += chunk.getNumRows(); - return data_consumed > size; - } - -private: - size_t data_consumed = 0; - - size_t size; -}; - } diff --git a/src/Processors/PingPongProcessor.cpp b/src/Processors/PingPongProcessor.cpp new file mode 100644 index 00000000000..777d768f9a4 --- /dev/null +++ b/src/Processors/PingPongProcessor.cpp @@ -0,0 +1,215 @@ +#include + +namespace DB +{ + +static InputPorts createPortsList(const Block & header, const Block & last_header, size_t num_ports) +{ + InputPorts res(num_ports, header); + res.emplace_back(last_header); + return res; +} + +PingPongProcessor::PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_) + : IProcessor(createPortsList(header, aux_header, num_ports), OutputPorts(num_ports + 1, header)) + , aux_in_port(inputs.back()) + , aux_out_port(outputs.back()) + , order(order_) +{ + assert(order == First || order == Second); + + port_pairs.resize(num_ports); + + auto input_it = inputs.begin(); + auto output_it = outputs.begin(); + for (size_t i = 0; i < num_ports; ++i) + { + port_pairs[i].input_port = &*input_it; + ++input_it; + + port_pairs[i].output_port = &*output_it; + ++output_it; + } +} + +void PingPongProcessor::finishPair(PortsPair & pair) +{ + if (!pair.is_finished) + { + pair.output_port->finish(); + pair.input_port->close(); + + pair.is_finished = true; + ++num_finished_pairs; + } +} + +bool PingPongProcessor::processPair(PortsPair & pair) +{ + if (pair.output_port->isFinished()) + { + finishPair(pair); + return false; + } + + if (pair.input_port->isFinished()) + { + finishPair(pair); + return false; + } + + if (!pair.output_port->canPush()) + { + pair.input_port->setNotNeeded(); + return false; + } + + pair.input_port->setNeeded(); + if (pair.input_port->hasData()) + { + Chunk chunk = pair.input_port->pull(true); + ready_to_send = isReady(chunk) || ready_to_send; + pair.output_port->push(std::move(chunk)); + } + + return true; +} + +bool PingPongProcessor::isPairsFinished() const +{ + return num_finished_pairs == port_pairs.size(); +} + +IProcessor::Status PingPongProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +{ + if (isPairsFinished()) + return Status::Finished; + + bool need_data = false; + + UNUSED(updated_inputs); + UNUSED(updated_outputs); + + // for (const auto & output_number : updated_outputs) + for (size_t output_number = 0; output_number < port_pairs.size(); ++output_number) + { + if (output_number >= port_pairs.size()) + continue; /// skip auxiliary port + need_data = processPair(port_pairs[output_number]) || need_data; + } + + // for (const auto & input_number : updated_inputs) + for (size_t input_number = 0; input_number < port_pairs.size(); ++input_number) + { + if (input_number >= port_pairs.size()) + continue; /// skip auxiliary port + need_data = processPair(port_pairs[input_number]) || need_data; + } + + if (isPairsFinished()) + return Status::Finished; + + if (need_data) + return Status::NeedData; + + return Status::PortFull; +} + +bool PingPongProcessor::sendPing() +{ + if (aux_out_port.canPush()) + { + Chunk chunk(aux_out_port.getHeader().cloneEmpty().getColumns(), 0); + aux_out_port.push(std::move(chunk)); + is_send = true; + aux_out_port.finish(); + return true; + } + return false; +} + +bool PingPongProcessor::recievePing() +{ + if (aux_in_port.hasData()) + { + aux_in_port.pull(); + is_recieved = true; + aux_in_port.close(); + return true; + } + return false; +} + +bool PingPongProcessor::canSend() const +{ + return !is_send && (ready_to_send || isPairsFinished()); +} + +IProcessor::Status PingPongProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +{ + if (!set_needed_once && !is_recieved && !aux_in_port.isFinished()) + { + set_needed_once = true; + aux_in_port.setNeeded(); + } + + if (order == First || is_send) + { + if (!is_recieved) + { + bool recieved = recievePing(); + if (!recieved) + { + return Status::NeedData; + } + } + } + + if (order == Second || is_recieved) + { + if (!is_send && canSend()) + { + bool sent = sendPing(); + if (!sent) + return Status::PortFull; + } + } + + auto status = processRegularPorts(updated_inputs, updated_outputs); + if (status == Status::Finished) + { + if (order == First || is_send) + { + if (!is_recieved) + { + bool recieved = recievePing(); + if (!recieved) + { + return Status::NeedData; + } + } + } + + if (order == Second || is_recieved) + { + if (!is_send && canSend()) + { + bool sent = sendPing(); + if (!sent) + return Status::PortFull; + } + } + } + if (status == Status::PortFull) + { + // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} status {}", __FILE__, __LINE__, status); + } + return status; +} + +std::pair PingPongProcessor::getAuxPorts() +{ + return std::make_pair(&aux_in_port, &aux_out_port); +} + +} diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h new file mode 100644 index 00000000000..09cdc6c0ec5 --- /dev/null +++ b/src/Processors/PingPongProcessor.h @@ -0,0 +1,108 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +/* + * Processor with N inputs and N outputs. Moves data from i-th input to i-th output as is. + * It has a pair of auxiliary ports to notify another instance by sending empty chunk after some condition holds. + * You should use this processor in pair of instances and connect auxiliary ports crosswise. + * + * ╭─┴───┴───┴───┴───┴─╮ ╭─┴───┴───┴───┴───┴─╮ + * │ ├─ aux ⟶│ │ + * │ PingPongProcessor │ │ PingPongProcessor │ + * │ │⟵ aux ─┤ │ + * ╰─┬───┬───┬───┬───┬─╯ ╰─┬───┬───┬───┬───┬─╯ + * + * One of the processors starts processing data, and another waits for notification. + * When `isReady` returns true, the first stops processing, sends a ping to another and waits for notification. + * After that, the second one also processes data until `isReady`, then send a notification back to the first one. + * After this roundtrip, processors bypass data from regular inputs to outputs. + */ +class PingPongProcessor : public IProcessor +{ +public: + enum class Order : uint8_t + { + /// Processor that starts processing data. + First, + /// Processor that waits for notification. + Second, + }; + + using enum Order; + + /// The `aux_header` is a header from another instance of procssor. + /// It's required because all outputs should have the same structure. + /// We don't care about structure of another processor, because we send just empty chunk, but need to follow the contract. + PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_); + + String getName() const override { return "PingPongProcessor"; } + + Status prepare(const PortNumbers &, const PortNumbers &) override; + + std::pair getAuxPorts(); + + virtual bool isReady(const Chunk & chunk) = 0; + +protected: + struct PortsPair + { + InputPort * input_port = nullptr; + OutputPort * output_port = nullptr; + bool is_finished = false; + }; + + bool sendPing(); + bool recievePing(); + bool canSend() const; + + bool isPairsFinished() const; + bool processPair(PortsPair & pair); + void finishPair(PortsPair & pair); + Status processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs); + + std::vector port_pairs; + size_t num_finished_pairs = 0; + + InputPort & aux_in_port; + OutputPort & aux_out_port; + + bool is_send = false; + bool is_recieved = false; + + bool ready_to_send = false; + + bool set_needed_once = false; + + Order order; +}; + +/// Reads first N rows from two streams evenly. +class ReadHeadBalancedProceesor : public PingPongProcessor +{ +public: + ReadHeadBalancedProceesor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_, Order order_) + : PingPongProcessor(header, aux_header, num_ports, order_) + , size(size_) + { + } + + bool isReady(const Chunk & chunk) override + { + data_consumed += chunk.getNumRows(); + return data_consumed > size; + } + +private: + size_t data_consumed = 0; + + size_t size; +}; + +} diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp index a56b53a6559..df0b399e37c 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include namespace DB { From d82a75ae752cf5ec9120c80d0a909ab911ef2386 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 11:26:47 +0000 Subject: [PATCH 15/39] cleanup PingPongProcessor --- src/Processors/PingPongProcessor.cpp | 30 +++++----------------------- src/Processors/PingPongProcessor.h | 20 +++++++++---------- 2 files changed, 14 insertions(+), 36 deletions(-) diff --git a/src/Processors/PingPongProcessor.cpp b/src/Processors/PingPongProcessor.cpp index 777d768f9a4..e34de025bb1 100644 --- a/src/Processors/PingPongProcessor.cpp +++ b/src/Processors/PingPongProcessor.cpp @@ -80,31 +80,15 @@ bool PingPongProcessor::isPairsFinished() const return num_finished_pairs == port_pairs.size(); } -IProcessor::Status PingPongProcessor::processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +IProcessor::Status PingPongProcessor::processRegularPorts() { if (isPairsFinished()) return Status::Finished; bool need_data = false; - UNUSED(updated_inputs); - UNUSED(updated_outputs); - - // for (const auto & output_number : updated_outputs) - for (size_t output_number = 0; output_number < port_pairs.size(); ++output_number) - { - if (output_number >= port_pairs.size()) - continue; /// skip auxiliary port - need_data = processPair(port_pairs[output_number]) || need_data; - } - - // for (const auto & input_number : updated_inputs) - for (size_t input_number = 0; input_number < port_pairs.size(); ++input_number) - { - if (input_number >= port_pairs.size()) - continue; /// skip auxiliary port - need_data = processPair(port_pairs[input_number]) || need_data; - } + for (auto & pair : port_pairs) + need_data = processPair(pair) || need_data; if (isPairsFinished()) return Status::Finished; @@ -145,7 +129,7 @@ bool PingPongProcessor::canSend() const return !is_send && (ready_to_send || isPairsFinished()); } -IProcessor::Status PingPongProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +IProcessor::Status PingPongProcessor::prepare() { if (!set_needed_once && !is_recieved && !aux_in_port.isFinished()) { @@ -175,7 +159,7 @@ IProcessor::Status PingPongProcessor::prepare(const PortNumbers & updated_inputs } } - auto status = processRegularPorts(updated_inputs, updated_outputs); + auto status = processRegularPorts(); if (status == Status::Finished) { if (order == First || is_send) @@ -200,10 +184,6 @@ IProcessor::Status PingPongProcessor::prepare(const PortNumbers & updated_inputs } } } - if (status == Status::PortFull) - { - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} status {}", __FILE__, __LINE__, status); - } return status; } diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index 09cdc6c0ec5..676ad44ea96 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -44,7 +44,7 @@ public: String getName() const override { return "PingPongProcessor"; } - Status prepare(const PortNumbers &, const PortNumbers &) override; + Status prepare() override; std::pair getAuxPorts(); @@ -65,7 +65,7 @@ protected: bool isPairsFinished() const; bool processPair(PortsPair & pair); void finishPair(PortsPair & pair); - Status processRegularPorts(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs); + Status processRegularPorts(); std::vector port_pairs; size_t num_finished_pairs = 0; @@ -78,6 +78,7 @@ protected: bool ready_to_send = false; + /// Used to set 'needed' flag once for auxiliary input at first `prepare` call. bool set_needed_once = false; Order order; @@ -87,22 +88,19 @@ protected: class ReadHeadBalancedProceesor : public PingPongProcessor { public: - ReadHeadBalancedProceesor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_, Order order_) - : PingPongProcessor(header, aux_header, num_ports, order_) - , size(size_) - { - } + ReadHeadBalancedProceesor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_to_wait_, Order order_) + : PingPongProcessor(header, aux_header, num_ports, order_) , data_consumed(0) , size_to_wait(size_to_wait_) + {} bool isReady(const Chunk & chunk) override { data_consumed += chunk.getNumRows(); - return data_consumed > size; + return data_consumed > size_to_wait; } private: - size_t data_consumed = 0; - - size_t size; + size_t data_consumed; + size_t size_to_wait; }; } From 67a9acc8db9fd25635048e1e21b4b37d67606a90 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 11:28:28 +0000 Subject: [PATCH 16/39] rename CreatingSetOnTheFlyStep -> CreateSetAndFilterOnTheFlyStep --- src/Interpreters/InterpreterSelectQuery.cpp | 6 ++--- ...cpp => CreateSetAndFilterOnTheFlyStep.cpp} | 26 +++++++------------ ...tep.h => CreateSetAndFilterOnTheFlyStep.h} | 6 ++--- .../Optimizations/filterPushDown.cpp | 4 +-- 4 files changed, 18 insertions(+), 24 deletions(-) rename src/Processors/QueryPlan/{CreatingSetOnTheFlyStep.cpp => CreateSetAndFilterOnTheFlyStep.cpp} (86%) rename src/Processors/QueryPlan/{CreatingSetOnTheFlyStep.h => CreateSetAndFilterOnTheFlyStep.h} (88%) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 15ed0e7d493..72fe8190969 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -39,7 +39,7 @@ #include #include #include -#include +#include #include #include #include @@ -1446,13 +1446,13 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(100, settings.max_rows_in_set_to_optimize_join); - auto creating_set_step = std::make_unique( + auto creating_set_step = std::make_unique( plan.getCurrentDataStream(), rhs_data_stream, key_names, max_rows, crosswise_connection, join_pos); creating_set_step->setStepDescription(fmt::format("Create set and filter {} joined stream", join_pos)); diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp similarity index 86% rename from src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp rename to src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index df0b399e37c..9a49cf21216 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -17,11 +17,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - -namespace -{ - -InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) +static InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) { auto input_it = inputs.begin(); for (size_t i = 0; i < num_ports; ++i) @@ -32,8 +28,6 @@ InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inpu return input_it; } -} - static ITransformingStep::Traits getTraits(bool is_filter) { return ITransformingStep::Traits @@ -51,7 +45,7 @@ static ITransformingStep::Traits getTraits(bool is_filter) } -class CreatingSetOnTheFlyStep::CrosswiseConnection : public boost::noncopyable +class CreateSetAndFilterOnTheFlyStep::CrosswiseConnection : public boost::noncopyable { public: using PortPair = std::pair; @@ -85,12 +79,12 @@ private: std::unique_ptr dummy_input_port; }; -CreatingSetOnTheFlyStep::CrosswiseConnectionPtr CreatingSetOnTheFlyStep::createCrossConnection() +CreateSetAndFilterOnTheFlyStep::CrosswiseConnectionPtr CreateSetAndFilterOnTheFlyStep::createCrossConnection() { - return std::make_shared(); + return std::make_shared(); } -CreatingSetOnTheFlyStep::CreatingSetOnTheFlyStep( +CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( const DataStream & input_stream_, const DataStream & rhs_input_stream_, const Names & column_names_, @@ -118,7 +112,7 @@ CreatingSetOnTheFlyStep::CreatingSetOnTheFlyStep( own_set->setHeader(header); } -void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { size_t num_streams = pipeline.getNumStreams(); pipeline.addSimpleTransform([this, num_streams](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr @@ -172,12 +166,12 @@ void CreatingSetOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, }, /* check_ports= */ false); } -void CreatingSetOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const +void CreateSetAndFilterOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const { map.add(getName(), true); } -void CreatingSetOnTheFlyStep::describeActions(FormatSettings & settings) const +void CreateSetAndFilterOnTheFlyStep::describeActions(FormatSettings & settings) const { String prefix(settings.offset, ' '); settings.out << prefix << getName(); @@ -185,7 +179,7 @@ void CreatingSetOnTheFlyStep::describeActions(FormatSettings & settings) const settings.out << '\n'; } -void CreatingSetOnTheFlyStep::updateOutputStream() +void CreateSetAndFilterOnTheFlyStep::updateOutputStream() { if (input_streams.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); diff --git a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h similarity index 88% rename from src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h rename to src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h index 67f31a97009..c489ff7356a 100644 --- a/src/Processors/QueryPlan/CreatingSetOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h @@ -7,7 +7,7 @@ namespace DB { -class CreatingSetOnTheFlyStep : public ITransformingStep +class CreateSetAndFilterOnTheFlyStep : public ITransformingStep { public: /// Two instances of step need some shared state to connect processors crosswise @@ -15,7 +15,7 @@ public: using CrosswiseConnectionPtr = std::shared_ptr; static CrosswiseConnectionPtr createCrossConnection(); - CreatingSetOnTheFlyStep( + CreateSetAndFilterOnTheFlyStep( const DataStream & input_stream_, const DataStream & rhs_input_stream_, const Names & column_names_, @@ -48,7 +48,7 @@ private: JoinTableSide position; - Poco::Logger * log = &Poco::Logger::get("CreatingSetOnTheFlyStep"); + Poco::Logger * log = &Poco::Logger::get("CreateSetAndFilterOnTheFlyStep"); }; } diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index f60f8192938..7c0402bb4b9 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include @@ -336,7 +336,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) return updated_steps; - if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) + if (auto updated_steps = simplePushDownOverStep(parent_node, nodes, child)) return updated_steps; if (auto * union_step = typeid_cast(child.get())) From 7228091ff1ef7252289a9d113eb7803fc7f9dfb4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 11:44:57 +0000 Subject: [PATCH 17/39] rename CreateSetAndFilterOnTheFlyTransform --- src/Processors/PingPongProcessor.h | 4 +- .../CreateSetAndFilterOnTheFlyStep.cpp | 45 +++++++++++-------- .../CreateSetAndFilterOnTheFlyStep.h | 13 ++++-- ...> CreateSetAndFilterOnTheFlyTransform.cpp} | 5 ++- ... => CreateSetAndFilterOnTheFlyTransform.h} | 0 5 files changed, 42 insertions(+), 25 deletions(-) rename src/Processors/Transforms/{CreatingSetsOnTheFlyTransform.cpp => CreateSetAndFilterOnTheFlyTransform.cpp} (98%) rename src/Processors/Transforms/{CreatingSetsOnTheFlyTransform.h => CreateSetAndFilterOnTheFlyTransform.h} (100%) diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index 676ad44ea96..3471ea2d772 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -85,10 +85,10 @@ protected: }; /// Reads first N rows from two streams evenly. -class ReadHeadBalancedProceesor : public PingPongProcessor +class ReadHeadBalancedProcessor : public PingPongProcessor { public: - ReadHeadBalancedProceesor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_to_wait_, Order order_) + ReadHeadBalancedProcessor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_to_wait_, Order order_) : PingPongProcessor(header, aux_header, num_ports, order_) , data_consumed(0) , size_to_wait(size_to_wait_) {} diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index 9a49cf21216..5ec160f6251 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include @@ -51,6 +51,7 @@ public: using PortPair = std::pair; /// Remember ports passed on the first call and connect with ones from second call. + /// Thread-safe. bool tryConnectPorts(PortPair rhs_ports, IProcessor * proc) { std::lock_guard lock(mux); @@ -88,14 +89,14 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( const DataStream & input_stream_, const DataStream & rhs_input_stream_, const Names & column_names_, - size_t max_rows_, + size_t max_rows_in_set_, CrosswiseConnectionPtr crosswise_connection_, JoinTableSide position_) : ITransformingStep(input_stream_, input_stream_.header, getTraits(false)) , column_names(column_names_) - , max_rows(max_rows_) + , max_rows_in_set(max_rows_in_set_) , rhs_input_stream_header(rhs_input_stream_.header) - , own_set(std::make_shared(SizeLimits(max_rows, 0, OverflowMode::BREAK), false, true)) + , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), false, true)) , filtering_set(nullptr) , crosswise_connection(crosswise_connection_) , position(position_) @@ -126,29 +127,33 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi if (!filtering_set) { - LOG_DEBUG(log, "filtering_set is null"); + LOG_DEBUG(log, "Skip filtering {} stream", position); return; } Block input_header = pipeline.getHeader(); - pipeline.transform([&input_header, this](OutputPortRawPtrs ports) + auto pipeline_transform = [&input_header, this](OutputPortRawPtrs ports) { - Processors transforms; + Processors result_transforms; size_t num_ports = ports.size(); + /// Add balancing transform auto idx = position == JoinTableSide::Left ? PingPongProcessor::First : PingPongProcessor::Second; - auto notifier = std::make_shared(input_header, rhs_input_stream_header, num_ports, max_rows, idx); - notifier->setDescription(getStepDescription()); + auto stream_balancer = std::make_shared(input_header, rhs_input_stream_header, num_ports, max_rows_in_set, idx); + stream_balancer->setDescription(getStepDescription()); - auto input_it = connectAllInputs(ports, notifier->getInputs(), num_ports); - assert(&*input_it == notifier->getAuxPorts().first); + /// Regular inputs just bypass data for respective ports + auto input_it = connectAllInputs(ports, stream_balancer->getInputs(), num_ports); + assert(&*input_it == stream_balancer->getAuxPorts().first); input_it++; - assert(input_it == notifier->getInputs().end()); + assert(input_it == stream_balancer->getInputs().end()); - crosswise_connection->tryConnectPorts(notifier->getAuxPorts(), notifier.get()); + /// Connect auxilary ports + crosswise_connection->tryConnectPorts(stream_balancer->getAuxPorts(), stream_balancer.get()); - auto & outputs = notifier->getOutputs(); + /// Add filtering transform, ports just connected respectively + auto & outputs = stream_balancer->getOutputs(); auto output_it = outputs.begin(); for (size_t i = 0; i < outputs.size() - 1; ++i) { @@ -156,14 +161,18 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi auto transform = std::make_shared(port.getHeader(), column_names, filtering_set); transform->setDescription(this->getStepDescription()); connect(port, transform->getInputPort()); - transforms.emplace_back(std::move(transform)); + result_transforms.emplace_back(std::move(transform)); } output_it++; assert(output_it == outputs.end()); - transforms.emplace_back(std::move(notifier)); + result_transforms.emplace_back(std::move(stream_balancer)); - return transforms; - }, /* check_ports= */ false); + return result_transforms; + }; + + /// Auxilary port stream_balancer can be connected later (by crosswise_connection). + /// So, use unsafe `transform` with `check_ports = false` to avoid assertions + pipeline.transform(std::move(pipeline_transform), /* check_ports= */ false); } void CreateSetAndFilterOnTheFlyStep::describeActions(JSONBuilder::JSONMap & map) const diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h index c489ff7356a..bcaa87ed551 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h @@ -1,12 +1,17 @@ #pragma once #include -#include #include namespace DB { +/* + * Used to optimize JOIN when joining a small table over a large table. + * Currently applied only for the full sorting join. + * It tries to build a set for each stream. + * Once one stream is finished, it starts to filter another stream with this set. + */ class CreateSetAndFilterOnTheFlyStep : public ITransformingStep { public: @@ -19,7 +24,7 @@ public: const DataStream & input_stream_, const DataStream & rhs_input_stream_, const Names & column_names_, - size_t max_rows_, + size_t max_rows_in_set_, CrosswiseConnectionPtr crosswise_connection_, JoinTableSide position_); @@ -30,6 +35,8 @@ public: void describeActions(FormatSettings & settings) const override; SetWithStatePtr getSet() const { return own_set; } + + /// Set for another stream. void setFiltering(SetWithStatePtr filtering_set_) { filtering_set = filtering_set_; } private: @@ -37,7 +44,7 @@ private: Names column_names; - size_t max_rows; + size_t max_rows_in_set; Block rhs_input_stream_header; diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp similarity index 98% rename from src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp rename to src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 1f9282d8baa..27d033a4580 100644 --- a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -1,8 +1,9 @@ +#include + #include #include -#include -#include +#include #include #include #include diff --git a/src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h similarity index 100% rename from src/Processors/Transforms/CreatingSetsOnTheFlyTransform.h rename to src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h From 8e1632f8242ed3eaf8a62bf9faf06d33e74edfb3 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 12:21:34 +0000 Subject: [PATCH 18/39] Create sets for joins: better code --- .../CreateSetAndFilterOnTheFlyStep.h | 1 + .../CreateSetAndFilterOnTheFlyTransform.cpp | 54 +++++++++++-------- .../CreateSetAndFilterOnTheFlyTransform.h | 38 ++++++++----- 3 files changed, 58 insertions(+), 35 deletions(-) diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h index bcaa87ed551..6e38fd31349 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h @@ -1,5 +1,6 @@ #pragma once #include +#include #include diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 27d033a4580..8360eae9a8e 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -53,7 +53,6 @@ std::string formatBytesHumanReadable(size_t bytes) } - CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform( const Block & header_, const Names & column_names_, size_t num_streams_, SetWithStatePtr set_) : ISimpleTransform(header_, header_, true) @@ -67,35 +66,39 @@ CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform( IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() { - auto status = ISimpleTransform::prepare(); + IProcessor::Status status = ISimpleTransform::prepare(); - if (status == Status::Finished && set && set->state == SetWithState::State::Creating) + if (!set || status != Status::Finished) + /// Nothing to handle with set + return status; + + /// Finalize set + if (set->state == SetWithState::State::Creating) { if (input.isFinished()) { set->finished_count++; - if (set->finished_count == num_streams) - { - set->finishInsert(); - set->state = SetWithState::State::Finished; - LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", - getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); - } + if (set->finished_count != num_streams) + /// Not all instances of processor are finished + return status; + + set->finishInsert(); + set->state = SetWithState::State::Finished; + LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", + getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + set.reset(); } else { - /// Should not happen because processor places before join that reads all the data + /// Should not happen because processor inserted before join that reads all the data /// But let's hanlde this case just for safety. set->state = SetWithState::State::Suspended; LOG_DEBUG(log, "{}: Processor finished, but not all input was read, cancelling building set after using {}", getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); + set.reset(); } } - if (status == Status::Finished && set && set->state != SetWithState::State::Creating) - /// Release pointer to make it possible destroy it by consumer - set.reset(); - return status; } @@ -103,7 +106,8 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) { if (!set || set->state != SetWithState::State::Creating) { - if (set) + /// If set building suspended by another processor, release pointer + if (set != nullptr) set.reset(); return; } @@ -111,17 +115,18 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) if (chunk.getNumRows()) { Columns key_columns = getColumnsByIndices(chunk, key_column_indices); - size_t prev_size = set->getTotalByteCount(); bool limit_exceeded = !set->insertFromBlock(key_columns); if (limit_exceeded) { auto prev_state = set->state.exchange(SetWithState::State::Suspended); + /// Print log only after first state switch if (prev_state == SetWithState::State::Creating) { - LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after using {} ({} -> {} bytes)", - getDescription(), formatBytesHumanReadable(set->getTotalByteCount()), prev_size, set->getTotalByteCount()); + LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after reading {} rows and using {}", + getDescription(), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); } - // TODO(@vdimir): set->clear() ? + /// Probaply we need to clear set here, because it's unneded anymore + /// But now `Set` doesn't have such method, so reset pointer in all processors and then it should be freed set.reset(); } } @@ -142,15 +147,19 @@ FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_ IProcessor::Status FilterBySetOnTheFlyTransform::prepare() { auto status = ISimpleTransform::prepare(); + + if (set && set->state == SetWithState::State::Suspended) + set.reset(); + if (status == Status::Finished) { bool has_filter = set && set->state == SetWithState::State::Finished; if (has_filter) { - LOG_DEBUG(log, "Finished {} by [{}]: consumed {} rows in total, {} rows bypassed, result {} rows, {}% filtered", + LOG_DEBUG(log, "Finished {} by [{}]: consumed {} rows in total, {} rows bypassed, result {} rows, {:.2f}% filtered", Poco::toLower(getDescription()), fmt::join(column_names, ", "), stat.consumed_rows, stat.consumed_rows_before_set, stat.result_rows, - static_cast(100 - 100.0 * stat.result_rows / stat.consumed_rows)); + 100 - 100.0 * stat.result_rows / stat.consumed_rows); } else { @@ -169,7 +178,6 @@ void FilterBySetOnTheFlyTransform::transform(Chunk & chunk) stat.result_rows += chunk.getNumRows(); bool can_filter = set && set->state == SetWithState::State::Finished; - // LOG_DEBUG(&Poco::Logger::get("XXXX"), "{}:{} {} / {} / {}", __FILE__, __LINE__, set->finished_count.load(), set->state.load(), set->getTotalRowCount()); if (!can_filter) stat.consumed_rows_before_set += chunk.getNumRows(); diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h index 072095cc3db..4542535c7ef 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h @@ -1,4 +1,5 @@ #pragma once + #include #include #include @@ -6,39 +7,50 @@ #include #include - namespace DB { -class SetWithState : public Set +struct SetWithState : public Set { -public: using Set::Set; + /// Flow: Creating -> Finished or Suspended enum class State { + /// Set is not yet created, + /// Creating processor continues to build set. + /// Filtering bypasses data. Creating, + + /// Set is finished. + /// Creating processor is finished. + /// Filtering filter stream with this set. Finished, + + /// Set building is canceled (due to limit exceeded). + /// Creating and filtering processors bypass data. Suspended, }; std::atomic state = State::Creating; + + /// Track number of processors that are currently working on this set. + /// Last one finalizes set. std::atomic_size_t finished_count = 0; }; using SetWithStatePtr = std::shared_ptr; /* - * Create a set on the fly. + * Create a set on the fly for incomming stream. * The set is created from the key columns of the input block. * Data is not changed and returned as is. - * Can be executed only in one stream. + * Can be executed in parallel, but blocks on operations with set. */ class CreatingSetsOnTheFlyTransform : public ISimpleTransform { public: - explicit CreatingSetsOnTheFlyTransform( - const Block & header_, const Names & column_names_, size_t num_streams_, SetWithStatePtr set_); + CreatingSetsOnTheFlyTransform(const Block & header_, const Names & column_names_, size_t num_streams_, SetWithStatePtr set_); String getName() const override { return "CreatingSetsOnTheFlyTransform"; } @@ -58,7 +70,6 @@ private: Poco::Logger * log; }; - /* * Filter the input chunk by the set. * When set building is not completed, just return the source data. @@ -66,8 +77,7 @@ private: class FilterBySetOnTheFlyTransform : public ISimpleTransform { public: - explicit FilterBySetOnTheFlyTransform( - const Block & header_, const Names & column_names_, SetWithStatePtr set_); + FilterBySetOnTheFlyTransform(const Block & header_, const Names & column_names_, SetWithStatePtr set_); String getName() const override { return "FilterBySetOnTheFlyTransform"; } @@ -85,16 +95,20 @@ private: /// Filter by this set when it's created SetWithStatePtr set; + /// Statistics to log struct Stat { + /// Total number of rows size_t consumed_rows = 0; + + /// Number of bypassed rows (processed before set is created) size_t consumed_rows_before_set = 0; + + /// Number of rows that passed the filter size_t result_rows = 0; } stat; Poco::Logger * log; }; - - } From 714c53ab24569f5553050b2c17c5837fab903167 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 12:29:05 +0000 Subject: [PATCH 19/39] fix typos --- src/Processors/PingPongProcessor.cpp | 20 +++++++++---------- src/Processors/PingPongProcessor.h | 2 +- .../CreateSetAndFilterOnTheFlyStep.cpp | 4 ++-- .../CreateSetAndFilterOnTheFlyTransform.cpp | 4 ++-- .../CreateSetAndFilterOnTheFlyTransform.h | 2 +- 5 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Processors/PingPongProcessor.cpp b/src/Processors/PingPongProcessor.cpp index e34de025bb1..78f8a12e9d3 100644 --- a/src/Processors/PingPongProcessor.cpp +++ b/src/Processors/PingPongProcessor.cpp @@ -117,7 +117,7 @@ bool PingPongProcessor::recievePing() if (aux_in_port.hasData()) { aux_in_port.pull(); - is_recieved = true; + is_received = true; aux_in_port.close(); return true; } @@ -131,7 +131,7 @@ bool PingPongProcessor::canSend() const IProcessor::Status PingPongProcessor::prepare() { - if (!set_needed_once && !is_recieved && !aux_in_port.isFinished()) + if (!set_needed_once && !is_received && !aux_in_port.isFinished()) { set_needed_once = true; aux_in_port.setNeeded(); @@ -139,17 +139,17 @@ IProcessor::Status PingPongProcessor::prepare() if (order == First || is_send) { - if (!is_recieved) + if (!is_received) { - bool recieved = recievePing(); - if (!recieved) + bool received = recievePing(); + if (!received) { return Status::NeedData; } } } - if (order == Second || is_recieved) + if (order == Second || is_received) { if (!is_send && canSend()) { @@ -164,17 +164,17 @@ IProcessor::Status PingPongProcessor::prepare() { if (order == First || is_send) { - if (!is_recieved) + if (!is_received) { - bool recieved = recievePing(); - if (!recieved) + bool received = recievePing(); + if (!received) { return Status::NeedData; } } } - if (order == Second || is_recieved) + if (order == Second || is_received) { if (!is_send && canSend()) { diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index 3471ea2d772..cbcead79633 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -74,7 +74,7 @@ protected: OutputPort & aux_out_port; bool is_send = false; - bool is_recieved = false; + bool is_received = false; bool ready_to_send = false; diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index 5ec160f6251..b81bfd027ca 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -149,7 +149,7 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi input_it++; assert(input_it == stream_balancer->getInputs().end()); - /// Connect auxilary ports + /// Connect auxiliary ports crosswise_connection->tryConnectPorts(stream_balancer->getAuxPorts(), stream_balancer.get()); /// Add filtering transform, ports just connected respectively @@ -170,7 +170,7 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi return result_transforms; }; - /// Auxilary port stream_balancer can be connected later (by crosswise_connection). + /// Auxiliary port stream_balancer can be connected later (by crosswise_connection). /// So, use unsafe `transform` with `check_ports = false` to avoid assertions pipeline.transform(std::move(pipeline_transform), /* check_ports= */ false); } diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 8360eae9a8e..5ef614bd3bc 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -91,7 +91,7 @@ IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() else { /// Should not happen because processor inserted before join that reads all the data - /// But let's hanlde this case just for safety. + /// But let's handle this case just for safety. set->state = SetWithState::State::Suspended; LOG_DEBUG(log, "{}: Processor finished, but not all input was read, cancelling building set after using {}", getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); @@ -125,7 +125,7 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after reading {} rows and using {}", getDescription(), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); } - /// Probaply we need to clear set here, because it's unneded anymore + /// Probaply we need to clear set here, because it's unneeded anymore /// But now `Set` doesn't have such method, so reset pointer in all processors and then it should be freed set.reset(); } diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h index 4542535c7ef..9b625d0dfaf 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h @@ -42,7 +42,7 @@ struct SetWithState : public Set using SetWithStatePtr = std::shared_ptr; /* - * Create a set on the fly for incomming stream. + * Create a set on the fly for incoming stream. * The set is created from the key columns of the input block. * Data is not changed and returned as is. * Can be executed in parallel, but blocks on operations with set. From 51e02d09f65b758eea98c25ab9f99f0ce5bf24f1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 12:29:31 +0000 Subject: [PATCH 20/39] set preserves_sorting = true for CreateSetAndFilterOnTheFlyStep --- src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index b81bfd027ca..176cd229efa 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -36,7 +36,7 @@ static ITransformingStep::Traits getTraits(bool is_filter) .preserves_distinct_columns = true, .returns_single_stream = false, .preserves_number_of_streams = true, - .preserves_sorting = false, /// resize doesn't perserve sorting (TODO fixit) + .preserves_sorting = true, }, { .preserves_number_of_rows = !is_filter, From 470dcff89c6c83c952555aeff4fd3f289fad63e2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 12:52:24 +0000 Subject: [PATCH 21/39] Add tests/performance/join_set_filter.xml --- tests/performance/join_set_filter.xml | 46 +++++++++++++++++++++++++++ 1 file changed, 46 insertions(+) create mode 100644 tests/performance/join_set_filter.xml diff --git a/tests/performance/join_set_filter.xml b/tests/performance/join_set_filter.xml new file mode 100644 index 00000000000..cb493fb435b --- /dev/null +++ b/tests/performance/join_set_filter.xml @@ -0,0 +1,46 @@ + + + + table_size + + 100000000 + + + + + + 100000 + full_sorting_merge + + + + CREATE TABLE t1 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y + AS SELECT + sipHash64(number, 't1_x') % {table_size} AS x, + sipHash64(number, 't1_y') % {table_size} AS y + FROM numbers({table_size}) + + + + CREATE TABLE t2 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y + AS SELECT + sipHash64(number, 't2_x') % {table_size} AS x, + sipHash64(number, 't2_y') % {table_size} AS y + FROM numbers({table_size}) + + + SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE less(t1.y, 10000) + SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE less(t1.y, 10000) + + SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE greater(t1.y, {table_size} - 10000) + SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE greater(t1.y, {table_size} - 10000) + + SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 100 = 0 + SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE t1.y % 100 = 0 + + SELECT * FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 1000 = 0 + SELECT * FROM t2 JOIN t1 ON t1.x = t2.x WHERE t1.y % 1000 = 0 + + DROP TABLE IF EXISTS t1 + DROP TABLE IF EXISTS t2 + From c67ab33d90845b9156f4656f76c8c15fc233a270 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 Aug 2022 16:13:58 +0000 Subject: [PATCH 22/39] small fix CreateSetAndFilterOnTheFlyStep --- .../CreateSetAndFilterOnTheFlyStep.cpp | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index 176cd229efa..340ca07e561 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -28,7 +28,7 @@ static InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts return input_it; } -static ITransformingStep::Traits getTraits(bool is_filter) +static ITransformingStep::Traits getTraits() { return ITransformingStep::Traits { @@ -39,7 +39,7 @@ static ITransformingStep::Traits getTraits(bool is_filter) .preserves_sorting = true, }, { - .preserves_number_of_rows = !is_filter, + .preserves_number_of_rows = false, } }; } @@ -77,6 +77,8 @@ private: InputPort * input_port = nullptr; OutputPort * output_port = nullptr; + /// Output ports should always be connected, and we can't add a step to the pipeline without them. + /// So, connect the port from the first processor to this dummy port and then reconnect to the second processor. std::unique_ptr dummy_input_port; }; @@ -92,7 +94,7 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( size_t max_rows_in_set_, CrosswiseConnectionPtr crosswise_connection_, JoinTableSide position_) - : ITransformingStep(input_stream_, input_stream_.header, getTraits(false)) + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , column_names(column_names_) , max_rows_in_set(max_rows_in_set_) , rhs_input_stream_header(rhs_input_stream_.header) @@ -125,12 +127,6 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi return res; }); - if (!filtering_set) - { - LOG_DEBUG(log, "Skip filtering {} stream", position); - return; - } - Block input_header = pipeline.getHeader(); auto pipeline_transform = [&input_header, this](OutputPortRawPtrs ports) { @@ -152,6 +148,13 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi /// Connect auxiliary ports crosswise_connection->tryConnectPorts(stream_balancer->getAuxPorts(), stream_balancer.get()); + if (!filtering_set) + { + LOG_DEBUG(log, "Skip filtering {} stream", position); + result_transforms.emplace_back(std::move(stream_balancer)); + return result_transforms; + } + /// Add filtering transform, ports just connected respectively auto & outputs = stream_balancer->getOutputs(); auto output_it = outputs.begin(); From 95f87dc34e549ad32c509358d66226d973742916 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 4 Aug 2022 09:22:52 +0000 Subject: [PATCH 23/39] fix sanitizer assert in CreateSetAndFilterOnTheFlyStep --- src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp | 2 +- src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index 340ca07e561..ecd2e34e56e 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -107,7 +107,7 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( throw Exception(ErrorCodes::LOGICAL_ERROR, "Crosswise connection is not initialized"); if (input_streams.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Step requires exactly one input stream, got {}", input_streams.size()); ColumnsWithTypeAndName header; for (const auto & name : column_names) diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h index 6e38fd31349..4ef7fee8ada 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h @@ -29,7 +29,7 @@ public: CrosswiseConnectionPtr crosswise_connection_, JoinTableSide position_); - String getName() const override { return "CreatingSetsOnTheFly"; } + String getName() const override { return "CreateSetAndFilterOnTheFlyStep"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; void describeActions(JSONBuilder::JSONMap & map) const override; From e21763e75952423fdc0543d683b5c77ee2f07522 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 4 Aug 2022 09:23:25 +0000 Subject: [PATCH 24/39] remove new setting from join_set_filter.xml --- tests/performance/join_set_filter.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/performance/join_set_filter.xml b/tests/performance/join_set_filter.xml index cb493fb435b..7f7804853fc 100644 --- a/tests/performance/join_set_filter.xml +++ b/tests/performance/join_set_filter.xml @@ -9,7 +9,6 @@ - 100000 full_sorting_merge From afeff512b5809157709ad9c2a374a5f4e8d8a594 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Aug 2022 11:54:15 +0000 Subject: [PATCH 25/39] Aux port for ReadHeadBalancedProcessor is empty Block --- src/Interpreters/InterpreterSelectQuery.cpp | 8 ++++---- src/Processors/PingPongProcessor.cpp | 13 ++++++++----- src/Processors/PingPongProcessor.h | 12 +++++------- .../QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp | 4 +--- .../QueryPlan/CreateSetAndFilterOnTheFlyStep.h | 3 --- 5 files changed, 18 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 72fe8190969..a3fd08de558 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1447,13 +1447,13 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(100, settings.max_rows_in_set_to_optimize_join); auto creating_set_step = std::make_unique( - plan.getCurrentDataStream(), rhs_data_stream, key_names, max_rows, crosswise_connection, join_pos); + plan.getCurrentDataStream(), key_names, max_rows, crosswise_connection, join_pos); creating_set_step->setStepDescription(fmt::format("Create set and filter {} joined stream", join_pos)); auto * step_raw_ptr = creating_set_step.get(); @@ -1470,8 +1470,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

= 0 && kind_allows_filtering) { - auto * left_set = add_create_set(query_plan, joined_plan->getCurrentDataStream(), join_clause.key_names_left, JoinTableSide::Left); - auto * right_set = add_create_set(*joined_plan, query_plan.getCurrentDataStream(), join_clause.key_names_right, JoinTableSide::Right); + auto * left_set = add_create_set(query_plan, join_clause.key_names_left, JoinTableSide::Left); + auto * right_set = add_create_set(*joined_plan, join_clause.key_names_right, JoinTableSide::Right); if (isInnerOrLeft(join_kind)) right_set->setFiltering(left_set->getSet()); diff --git a/src/Processors/PingPongProcessor.cpp b/src/Processors/PingPongProcessor.cpp index 78f8a12e9d3..50c7a836d00 100644 --- a/src/Processors/PingPongProcessor.cpp +++ b/src/Processors/PingPongProcessor.cpp @@ -3,15 +3,18 @@ namespace DB { -static InputPorts createPortsList(const Block & header, const Block & last_header, size_t num_ports) +/// Create list with `num_ports` of regular ports and 1 auxiliary port with empty header. +template requires std::is_same_v || std::is_same_v +static T createPortsWithSpecial(const Block & header, size_t num_ports) { - InputPorts res(num_ports, header); - res.emplace_back(last_header); + T res(num_ports, header); + res.emplace_back(Block()); return res; } -PingPongProcessor::PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_) - : IProcessor(createPortsList(header, aux_header, num_ports), OutputPorts(num_ports + 1, header)) +PingPongProcessor::PingPongProcessor(const Block & header, size_t num_ports, Order order_) + : IProcessor(createPortsWithSpecial(header, num_ports), + createPortsWithSpecial(header, num_ports)) , aux_in_port(inputs.back()) , aux_out_port(outputs.back()) , order(order_) diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index cbcead79633..8e4eb4533bc 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -37,10 +37,7 @@ public: using enum Order; - /// The `aux_header` is a header from another instance of procssor. - /// It's required because all outputs should have the same structure. - /// We don't care about structure of another processor, because we send just empty chunk, but need to follow the contract. - PingPongProcessor(const Block & header, const Block & aux_header, size_t num_ports, Order order_); + PingPongProcessor(const Block & header, size_t num_ports, Order order_); String getName() const override { return "PingPongProcessor"; } @@ -88,9 +85,10 @@ protected: class ReadHeadBalancedProcessor : public PingPongProcessor { public: - ReadHeadBalancedProcessor(const Block & header, const Block & aux_header, size_t num_ports, size_t size_to_wait_, Order order_) - : PingPongProcessor(header, aux_header, num_ports, order_) , data_consumed(0) , size_to_wait(size_to_wait_) - {} + ReadHeadBalancedProcessor(const Block & header, size_t num_ports, size_t size_to_wait_, Order order_) + : PingPongProcessor(header, num_ports, order_) , data_consumed(0) , size_to_wait(size_to_wait_) + { + } bool isReady(const Chunk & chunk) override { diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index ecd2e34e56e..3787d5b2aba 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -89,7 +89,6 @@ CreateSetAndFilterOnTheFlyStep::CrosswiseConnectionPtr CreateSetAndFilterOnTheFl CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( const DataStream & input_stream_, - const DataStream & rhs_input_stream_, const Names & column_names_, size_t max_rows_in_set_, CrosswiseConnectionPtr crosswise_connection_, @@ -97,7 +96,6 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , column_names(column_names_) , max_rows_in_set(max_rows_in_set_) - , rhs_input_stream_header(rhs_input_stream_.header) , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), false, true)) , filtering_set(nullptr) , crosswise_connection(crosswise_connection_) @@ -136,7 +134,7 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi /// Add balancing transform auto idx = position == JoinTableSide::Left ? PingPongProcessor::First : PingPongProcessor::Second; - auto stream_balancer = std::make_shared(input_header, rhs_input_stream_header, num_ports, max_rows_in_set, idx); + auto stream_balancer = std::make_shared(input_header, num_ports, max_rows_in_set, idx); stream_balancer->setDescription(getStepDescription()); /// Regular inputs just bypass data for respective ports diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h index 4ef7fee8ada..8c2eef00af0 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.h @@ -23,7 +23,6 @@ public: CreateSetAndFilterOnTheFlyStep( const DataStream & input_stream_, - const DataStream & rhs_input_stream_, const Names & column_names_, size_t max_rows_in_set_, CrosswiseConnectionPtr crosswise_connection_, @@ -47,8 +46,6 @@ private: size_t max_rows_in_set; - Block rhs_input_stream_header; - SetWithStatePtr own_set; SetWithStatePtr filtering_set; From 2deb28d56e38e0f72ecbf97a559335892f468957 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Aug 2022 12:04:06 +0000 Subject: [PATCH 26/39] Add 02382_join_and_filtering_set --- .../02382_join_and_filtering_set.reference | 7 +++++++ .../02382_join_and_filtering_set.sql | 20 +++++++++++++++++++ 2 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/02382_join_and_filtering_set.reference create mode 100644 tests/queries/0_stateless/02382_join_and_filtering_set.sql diff --git a/tests/queries/0_stateless/02382_join_and_filtering_set.reference b/tests/queries/0_stateless/02382_join_and_filtering_set.reference new file mode 100644 index 00000000000..58c8ccca6a2 --- /dev/null +++ b/tests/queries/0_stateless/02382_join_and_filtering_set.reference @@ -0,0 +1,7 @@ +106 +46 +42 +51 +42 +24 +10 diff --git a/tests/queries/0_stateless/02382_join_and_filtering_set.sql b/tests/queries/0_stateless/02382_join_and_filtering_set.sql new file mode 100644 index 00000000000..4b425f22c87 --- /dev/null +++ b/tests/queries/0_stateless/02382_join_and_filtering_set.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y +AS SELECT sipHash64(number, 't1_x') % 100 AS x, sipHash64(number, 't1_y') % 100 AS y FROM numbers(100); + +CREATE TABLE t2 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y +AS SELECT sipHash64(number, 't2_x') % 100 AS x, sipHash64(number, 't2_y') % 100 AS y FROM numbers(100); + +SET max_rows_in_set_to_optimize_join = 1000; +SET join_algorithm = 'full_sorting_merge'; + +-- different combinations of conditions on key/attribute columns for the left/right tables +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x; +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 2 == 0; +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.x % 2 == 0; +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x WHERE t2.y % 2 == 0; +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x WHERE t2.x % 2 == 0; +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.y % 2 == 0 AND t2.y % 2 == 0; +SELECT count() FROM t1 JOIN t2 ON t1.x = t2.x WHERE t1.x % 2 == 0 AND t2.x % 2 == 0 AND t1.y % 2 == 0 AND t2.y % 2 == 0; From afb6b7d9cfbc290590ce5b90c9eb7f75810ec93f Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Aug 2022 12:39:39 +0000 Subject: [PATCH 27/39] Test plan and pipeline for filtering step for join --- src/Core/Settings.h | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 7 +-- src/Processors/PingPongProcessor.h | 4 +- .../02383_join_and_filtering_set.reference | 10 ++++ .../02383_join_and_filtering_set.sh | 55 +++++++++++++++++++ 5 files changed, 70 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02383_join_and_filtering_set.reference create mode 100755 tests/queries/0_stateless/02383_join_and_filtering_set.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8a81a8b0b29..9f65527f0db 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -366,7 +366,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \ - M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining (-1 to disable).", 0) \ + M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining.", 0) \ \ M(Bool, compatibility_ignore_collation_in_create_table, true, "Compatibility ignore collation in create table", 0) \ \ diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a3fd08de558..cfd0d4815c6 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1449,11 +1449,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(100, settings.max_rows_in_set_to_optimize_join); - auto creating_set_step = std::make_unique( - plan.getCurrentDataStream(), key_names, max_rows, crosswise_connection, join_pos); + plan.getCurrentDataStream(), key_names, settings.max_rows_in_set_to_optimize_join, crosswise_connection, join_pos); creating_set_step->setStepDescription(fmt::format("Create set and filter {} joined stream", join_pos)); auto * step_raw_ptr = creating_set_step.get(); @@ -1468,7 +1465,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

= 0 && kind_allows_filtering) + if (settings.max_rows_in_set_to_optimize_join > 0 && kind_allows_filtering) { auto * left_set = add_create_set(query_plan, join_clause.key_names_left, JoinTableSide::Left); auto * right_set = add_create_set(*joined_plan, join_clause.key_names_right, JoinTableSide::Right); diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index 8e4eb4533bc..2b461f8409e 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -39,8 +39,6 @@ public: PingPongProcessor(const Block & header, size_t num_ports, Order order_); - String getName() const override { return "PingPongProcessor"; } - Status prepare() override; std::pair getAuxPorts(); @@ -90,6 +88,8 @@ public: { } + String getName() const override { return "ReadHeadBalancedProcessor"; } + bool isReady(const Chunk & chunk) override { data_consumed += chunk.getNumRows(); diff --git a/tests/queries/0_stateless/02383_join_and_filtering_set.reference b/tests/queries/0_stateless/02383_join_and_filtering_set.reference new file mode 100644 index 00000000000..2ad282ca07f --- /dev/null +++ b/tests/queries/0_stateless/02383_join_and_filtering_set.reference @@ -0,0 +1,10 @@ +Ok +Ok +Ok +Ok +Ok +Ok +Ok +Ok +Ok +Ok diff --git a/tests/queries/0_stateless/02383_join_and_filtering_set.sh b/tests/queries/0_stateless/02383_join_and_filtering_set.sh new file mode 100755 index 00000000000..3356be58ff7 --- /dev/null +++ b/tests/queries/0_stateless/02383_join_and_filtering_set.sh @@ -0,0 +1,55 @@ +#!/usr/bin/env bash +# Tags: no-asan,no-msan,no-tsan,no-ubsan +# +# Test doesn't run complex queries, just test the logic of setting, so no need to run with different builds. +# Also, we run similar queries in 02382_join_and_filtering_set.sql which is enabled for these builds. +# + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -mn -q """ +CREATE TABLE t1 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y +AS SELECT sipHash64(number, 't1_x') % 100 AS x, sipHash64(number, 't1_y') % 100 AS y FROM numbers(100); + +CREATE TABLE t2 (x UInt64, y UInt64) ENGINE = MergeTree ORDER BY y +AS SELECT sipHash64(number, 't2_x') % 100 AS x, sipHash64(number, 't2_y') % 100 AS y FROM numbers(100); +""" + +# Arguments: +# - value of max_rows_in_set_to_optimize_join +# - join kind +# - expected number of steps in plan +# - expected number of steps in pipeline +function test() { + +PARAM_VALUE=$1 +JOIN_KIND=${2:-} + +EXPECTED_PLAN_STEPS=$3 +RES=$( + $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join=${PARAM_VALUE} --join_algorithm='full_sorting_merge' \ + -q "EXPLAIN PLAN SELECT count() FROM t1 ${JOIN_KIND} JOIN t2 ON t1.x = t2.x" | grep -o 'CreateSetAndFilterOnTheFlyStep' | wc -l +) +[ "$RES" -eq "$EXPECTED_PLAN_STEPS" ] && echo "Ok" || echo "Fail: $RES != $EXPECTED_PLAN_STEPS" + +EXPECTED_PIPELINE_STEPS=$4 +RES=$( + $CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join=${PARAM_VALUE} --join_algorithm='full_sorting_merge' \ + -q "EXPLAIN PIPELINE SELECT count() FROM t1 ${JOIN_KIND} JOIN t2 ON t1.x = t2.x" \ + | grep -o -e ReadHeadBalancedProcessor -e FilterBySetOnTheFlyTransform -e CreatingSetsOnTheFlyTransform | wc -l +) +[ "$RES" -eq "$EXPECTED_PIPELINE_STEPS" ] && echo "Ok" || echo "Fail: $RES != $EXPECTED_PIPELINE_STEPS" + +} + +test 1000 '' 2 6 + +# no filtering for left/right side +test 1000 'LEFT' 2 5 +test 1000 'RIGHT' 2 5 + +# when disabled no extra steps should be created +test 1000 'FULL' 0 0 +test 0 '' 0 0 From 7915b6948fee1173b705c4eed7636383a5288d69 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Aug 2022 13:03:14 +0000 Subject: [PATCH 28/39] Fix build after rebase --- src/Processors/QueryPlan/Optimizations/filterPushDown.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 7c0402bb4b9..dbf389163be 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -303,8 +303,8 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (updated_steps > 0) { LOG_DEBUG(&Poco::Logger::get("QueryPlanOptimizations"), "Pushed down filter {} to the {} side of join", split_filter_column_name, kind); - return updated_steps; } + return updated_steps; }; if (size_t updated_steps = join_push_down(JoinKind::Left)) From 3e06cf3a9dba2d7bedd74cf5c274e5199eb78de2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Aug 2022 10:10:47 +0000 Subject: [PATCH 29/39] Upd max_rows_in_set_to_optimize_join description --- 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 9f65527f0db..84bb086fbfb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -366,7 +366,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \ - M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining.", 0) \ + M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining. 0 - disable.", 0) \ \ M(Bool, compatibility_ignore_collation_in_create_table, true, "Compatibility ignore collation in create table", 0) \ \ From b0e2616aa916115e3f1011db29f75615d1bdb997 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Aug 2022 10:11:30 +0000 Subject: [PATCH 30/39] Style fixes in CreateSetAndFilterOnTheFlyTransform and related --- src/Processors/PingPongProcessor.cpp | 2 +- src/Processors/PingPongProcessor.h | 9 ++-- .../CreateSetAndFilterOnTheFlyStep.cpp | 48 ++++++++++--------- .../CreateSetAndFilterOnTheFlyTransform.cpp | 23 +++------ .../CreateSetAndFilterOnTheFlyTransform.h | 4 +- 5 files changed, 40 insertions(+), 46 deletions(-) diff --git a/src/Processors/PingPongProcessor.cpp b/src/Processors/PingPongProcessor.cpp index 50c7a836d00..e9d61386314 100644 --- a/src/Processors/PingPongProcessor.cpp +++ b/src/Processors/PingPongProcessor.cpp @@ -71,7 +71,7 @@ bool PingPongProcessor::processPair(PortsPair & pair) if (pair.input_port->hasData()) { Chunk chunk = pair.input_port->pull(true); - ready_to_send = isReady(chunk) || ready_to_send; + ready_to_send |= consume(chunk); pair.output_port->push(std::move(chunk)); } diff --git a/src/Processors/PingPongProcessor.h b/src/Processors/PingPongProcessor.h index 2b461f8409e..3dbe1178332 100644 --- a/src/Processors/PingPongProcessor.h +++ b/src/Processors/PingPongProcessor.h @@ -20,8 +20,8 @@ namespace DB * ╰─┬───┬───┬───┬───┬─╯ ╰─┬───┬───┬───┬───┬─╯ * * One of the processors starts processing data, and another waits for notification. - * When `isReady` returns true, the first stops processing, sends a ping to another and waits for notification. - * After that, the second one also processes data until `isReady`, then send a notification back to the first one. + * When `consume` returns true, the first stops processing, sends a ping to another and waits for notification. + * After that, the second one also processes data until `consume`, then send a notification back to the first one. * After this roundtrip, processors bypass data from regular inputs to outputs. */ class PingPongProcessor : public IProcessor @@ -43,7 +43,8 @@ public: std::pair getAuxPorts(); - virtual bool isReady(const Chunk & chunk) = 0; + /// Returns `true` when enough data consumed + virtual bool consume(const Chunk & chunk) = 0; protected: struct PortsPair @@ -90,7 +91,7 @@ public: String getName() const override { return "ReadHeadBalancedProcessor"; } - bool isReady(const Chunk & chunk) override + bool consume(const Chunk & chunk) override { data_consumed += chunk.getNumRows(); return data_consumed > size_to_wait; diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index 3787d5b2aba..e42642ceff8 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) +static void connectAllInputs(OutputPortRawPtrs ports, InputPorts & inputs, size_t num_ports) { auto input_it = inputs.begin(); for (size_t i = 0; i < num_ports; ++i) @@ -25,7 +25,14 @@ static InputPorts::iterator connectAllInputs(OutputPortRawPtrs ports, InputPorts connect(*ports[i], *input_it); input_it++; } - return input_it; +} + +static ColumnsWithTypeAndName getColumnSubset(const Block & block, const Names & column_names) +{ + ColumnsWithTypeAndName result; + for (const auto & name : column_names) + result.emplace_back(block.getByName(name)); + return result; } static ITransformingStep::Traits getTraits() @@ -44,7 +51,6 @@ static ITransformingStep::Traits getTraits() }; } - class CreateSetAndFilterOnTheFlyStep::CrosswiseConnection : public boost::noncopyable { public: @@ -52,8 +58,10 @@ public: /// Remember ports passed on the first call and connect with ones from second call. /// Thread-safe. - bool tryConnectPorts(PortPair rhs_ports, IProcessor * proc) + void connectPorts(PortPair rhs_ports, IProcessor * proc) { + assert(!rhs_ports.first->isConnected() && !rhs_ports.second->isConnected()); + std::lock_guard lock(mux); if (input_port || output_port) { @@ -61,15 +69,16 @@ public: assert(!input_port->isConnected()); connect(*rhs_ports.second, *input_port); connect(*output_port, *rhs_ports.first, /* reconnect= */ true); - return true; } - std::tie(input_port, output_port) = rhs_ports; - assert(input_port && output_port); - assert(!input_port->isConnected() && !output_port->isConnected()); + else + { + std::tie(input_port, output_port) = rhs_ports; + assert(input_port && output_port); + assert(!input_port->isConnected() && !output_port->isConnected()); - dummy_input_port = std::make_unique(output_port->getHeader(), proc); - connect(*output_port, *dummy_input_port); - return false; + dummy_input_port = std::make_unique(output_port->getHeader(), proc); + connect(*output_port, *dummy_input_port); + } } private: @@ -107,10 +116,7 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( if (input_streams.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Step requires exactly one input stream, got {}", input_streams.size()); - ColumnsWithTypeAndName header; - for (const auto & name : column_names) - header.emplace_back(input_streams[0].header.getByName(name)); - own_set->setHeader(header); + own_set->setHeader(getColumnSubset(input_streams[0].header, column_names)); } void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) @@ -138,13 +144,10 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi stream_balancer->setDescription(getStepDescription()); /// Regular inputs just bypass data for respective ports - auto input_it = connectAllInputs(ports, stream_balancer->getInputs(), num_ports); - assert(&*input_it == stream_balancer->getAuxPorts().first); - input_it++; - assert(input_it == stream_balancer->getInputs().end()); + connectAllInputs(ports, stream_balancer->getInputs(), num_ports); /// Connect auxiliary ports - crosswise_connection->tryConnectPorts(stream_balancer->getAuxPorts(), stream_balancer.get()); + crosswise_connection->connectPorts(stream_balancer->getAuxPorts(), stream_balancer.get()); if (!filtering_set) { @@ -164,8 +167,7 @@ void CreateSetAndFilterOnTheFlyStep::transformPipeline(QueryPipelineBuilder & pi connect(port, transform->getInputPort()); result_transforms.emplace_back(std::move(transform)); } - output_it++; - assert(output_it == outputs.end()); + assert(output_it == std::prev(outputs.end())); result_transforms.emplace_back(std::move(stream_balancer)); return result_transforms; @@ -194,6 +196,8 @@ void CreateSetAndFilterOnTheFlyStep::updateOutputStream() if (input_streams.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "{} requires exactly one input stream, got {}", getName(), input_streams.size()); + own_set->setHeader(getColumnSubset(input_streams[0].header, column_names)); + output_stream = input_streams[0]; } diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 5ef614bd3bc..fa30807eaee 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -5,6 +5,7 @@ #include #include +#include #include #include #include @@ -40,17 +41,6 @@ ColumnsWithTypeAndName getColumnsByIndices(const Block & sample_block, const Chu return block.getColumnsWithTypeAndName(); } -std::string formatBytesHumanReadable(size_t bytes) -{ - if (bytes >= 1_GiB) - return fmt::format("{:.2f} GB", static_cast(bytes) / 1_GiB); - if (bytes >= 1_MiB) - return fmt::format("{:.2f} MB", static_cast(bytes) / 1_MiB); - if (bytes >= 1_KiB) - return fmt::format("{:.2f} KB", static_cast(bytes) / 1_KiB); - return fmt::format("{:.2f} B", static_cast(bytes)); -} - } CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform( @@ -60,7 +50,6 @@ CreatingSetsOnTheFlyTransform::CreatingSetsOnTheFlyTransform( , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) , num_streams(num_streams_) , set(set_) - , log(&Poco::Logger::get(getName())) { } @@ -69,7 +58,7 @@ IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() IProcessor::Status status = ISimpleTransform::prepare(); if (!set || status != Status::Finished) - /// Nothing to handle with set + /// Nothing to do with set return status; /// Finalize set @@ -85,7 +74,8 @@ IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() set->finishInsert(); set->state = SetWithState::State::Finished; LOG_DEBUG(log, "{}: finish building set for [{}] with {} rows, set size is {}", - getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + getDescription(), fmt::join(column_names, ", "), set->getTotalRowCount(), + formatReadableSizeWithBinarySuffix(set->getTotalByteCount())); set.reset(); } else @@ -123,7 +113,7 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) if (prev_state == SetWithState::State::Creating) { LOG_DEBUG(log, "{}: set limit exceeded, give up building set, after reading {} rows and using {}", - getDescription(), set->getTotalRowCount(), formatBytesHumanReadable(set->getTotalByteCount())); + getDescription(), set->getTotalRowCount(), formatReadableSizeWithBinarySuffix(set->getTotalByteCount())); } /// Probaply we need to clear set here, because it's unneeded anymore /// But now `Set` doesn't have such method, so reset pointer in all processors and then it should be freed @@ -137,7 +127,6 @@ FilterBySetOnTheFlyTransform::FilterBySetOnTheFlyTransform(const Block & header_ , column_names(column_names_) , key_column_indices(getColumnIndices(inputs.front().getHeader(), column_names)) , set(set_) - , log(&Poco::Logger::get(getName())) { const auto & header = inputs.front().getHeader(); for (size_t idx : key_column_indices) @@ -193,7 +182,7 @@ void FilterBySetOnTheFlyTransform::transform(Chunk & chunk) size_t result_num_rows = 0; for (auto & col : columns) { - col = col->filter(mask, 0); + col = col->filter(mask, /* negative */ false); result_num_rows = col->size(); } stat.result_rows += result_num_rows; diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h index 9b625d0dfaf..d214a310a8c 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.h @@ -67,7 +67,7 @@ private: /// Set to fill SetWithStatePtr set; - Poco::Logger * log; + Poco::Logger * log = &Poco::Logger::get("CreatingSetsOnTheFlyTransform"); }; /* @@ -108,7 +108,7 @@ private: size_t result_rows = 0; } stat; - Poco::Logger * log; + Poco::Logger * log = &Poco::Logger::get("FilterBySetOnTheFlyTransform"); }; } From 24f62e84860df2236f9523b55afa77953206bd85 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 29 Aug 2022 11:27:08 +0000 Subject: [PATCH 31/39] Throw an error in CreatingSetsOnTheFlyTransform in case of input for finished --- .../Transforms/CreateSetAndFilterOnTheFlyTransform.cpp | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index fa30807eaee..1abadf2f727 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -81,11 +81,7 @@ IProcessor::Status CreatingSetsOnTheFlyTransform::prepare() else { /// Should not happen because processor inserted before join that reads all the data - /// But let's handle this case just for safety. - set->state = SetWithState::State::Suspended; - LOG_DEBUG(log, "{}: Processor finished, but not all input was read, cancelling building set after using {}", - getDescription(), formatBytesHumanReadable(set->getTotalByteCount())); - set.reset(); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Processor finished, but not all input was read"); } } From 7a85a943f1c0cdc0000c6daeb04d855b750f3bbe Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 30 Aug 2022 01:11:04 +0000 Subject: [PATCH 32/39] fix close lock on exec, refactoring, comments added --- .../decompressor.cpp | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 73d34f1fe8e..0c42f57367d 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -361,7 +361,7 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress #endif -uint32_t get_inode(const char * self) +uint32_t getInode(const char * self) { std::ifstream maps("/proc/self/maps"); if (maps.fail()) @@ -370,6 +370,10 @@ uint32_t get_inode(const char * self) return 0; } + /// Record example for /proc/self/maps: + /// address perms offset device inode pathname + /// 561a247de000-561a247e0000 r--p 00000000 103:01 1564 /usr/bin/cat + /// see "man 5 proc" for (std::string line; std::getline(maps, line);) { std::stringstream ss(line); // STYLE_CHECK_ALLOW_STD_STRING_STREAM @@ -405,7 +409,8 @@ int main(int/* argc*/, char* argv[]) else name = file_path; - uint32_t inode = get_inode(self); + /// get inode of this executable + uint32_t inode = getInode(self); if (inode == 0) { std::cerr << "Unable to obtain inode." << std::endl; @@ -421,6 +426,9 @@ int main(int/* argc*/, char* argv[]) return 1; } + /// lock file should be closed on exec call + fcntl(lock, F_SETFD, FD_CLOEXEC); + if (lockf(lock, F_LOCK, 0)) { perror("lockf"); @@ -434,7 +442,8 @@ int main(int/* argc*/, char* argv[]) return 1; } - /// if decompression was performed by another process + /// if decompression was performed by another process since this copy was started + /// then file refered by path "self" is already pointing to different inode if (input_info.st_ino != inode) { struct stat lock_info; @@ -444,6 +453,7 @@ int main(int/* argc*/, char* argv[]) return 1; } + /// size 1 of lock file indicates that another decompressor has found active executable if (lock_info.st_size == 1) execv(self, argv); @@ -512,6 +522,8 @@ int main(int/* argc*/, char* argv[]) if (has_exec) { + /// write one byte to the lock in case other copies of compressed are running to indicate that + /// execution should be performed write(lock, "1", 1); execv(self, argv); @@ -521,6 +533,7 @@ int main(int/* argc*/, char* argv[]) return 1; } + /// since inodes can be reused - it's a precaution if lock file already exists and have size of 1 ftruncate(lock, 0); printf("No target executable - decompression only was performed.\n"); From a9cab86a73333d27970a0b07e646b9b305b05275 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 30 Aug 2022 01:46:17 +0000 Subject: [PATCH 33/39] typo --- utils/self-extracting-executable/decompressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 0c42f57367d..ad4cd5b2f63 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -443,7 +443,7 @@ int main(int/* argc*/, char* argv[]) } /// if decompression was performed by another process since this copy was started - /// then file refered by path "self" is already pointing to different inode + /// then file referred by path "self" is already pointing to different inode if (input_info.st_ino != inode) { struct stat lock_info; From 0f6f3c73b0fc4540bf0af87620e77f27f87eba25 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 30 Aug 2022 11:57:28 +0000 Subject: [PATCH 34/39] Minor fix --- src/Core/Settings.h | 2 +- .../Transforms/CreateSetAndFilterOnTheFlyTransform.cpp | 5 +++++ tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql | 2 ++ 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 84bb086fbfb..eafc41b8861 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -366,7 +366,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \ - M(Int64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining. 0 - disable.", 0) \ + M(UInt64, max_rows_in_set_to_optimize_join, 100'000, "Maximal size of the set to filter joined tables by each other row sets before joining. 0 - disable.", 0) \ \ M(Bool, compatibility_ignore_collation_in_create_table, true, "Compatibility ignore collation in create table", 0) \ \ diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 1abadf2f727..4278eb8e8b2 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -14,6 +14,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql index a418a06803f..4ed6d965292 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.sql @@ -7,6 +7,8 @@ USING (key); SET join_algorithm = 'full_sorting_merge'; +SET max_rows_in_set_to_optimize_join = 0; + EXPLAIN actions=0, description=0, header=1 SELECT * FROM ( SELECT 'key2' AS key ) AS s1 JOIN ( SELECT 'key1' AS key, '1' AS value UNION ALL SELECT 'key2' AS key, '1' AS value ) AS s2 From e659b4553dac6e495dc99d7a17289851884c9132 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 31 Aug 2022 21:53:57 +0300 Subject: [PATCH 35/39] 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 f40c93c6f5d..7e060a9f687 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -533,7 +533,7 @@ class TestCase: else: # If --database is not specified, we will create temporary database with # unique name and we will recreate and drop it for each test - def random_str(length=6): + def random_str(length=8): alphabet = string.ascii_lowercase + string.digits # NOTE: it is important not to use default random generator, since it shares state. return "".join( From ecdea62cd5ab23254701d2a89c42c231d51a91da Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Sep 2022 14:26:50 +0300 Subject: [PATCH 36/39] Update clickhouse-test --- tests/clickhouse-test | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 7e060a9f687..14cf4d0674a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -995,9 +995,12 @@ class TestCase: args.timeout - (datetime.now() - start_time).total_seconds(), 20 ) try: + drop_database_query = "DROP DATABASE " + database + if args.replicated_database: + drop_database_query += " ON CLUSTER test_cluster_database_replicated" clickhouse_execute( args, - "DROP DATABASE " + database, + drop_database_query, timeout=seconds_left, settings={ "log_comment": args.testcase_basename, From fb42afbbaca54360acc3d17becdc53c241acddf4 Mon Sep 17 00:00:00 2001 From: Arthur Passos Date: Thu, 1 Sep 2022 08:59:14 -0300 Subject: [PATCH 37/39] CaresPTRResolver small safety improvement Previous to #40769, only `hostent::h_aliases` was being accessed. After that PR got merged, `hostent::h_name` started being accessed as well. This PR moves the first `hostent::h_aliases != nullptr` check that could prevent `hostent::h_name` from being accessed. During debugging, I observed that even when there are not aliases, `hostent::h_aliases` is not null. That's why it hasn't caused any problems, but proposing this change to be on the safe side. --- src/Common/CaresPTRResolver.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp index e5d48b864c8..a02909309b6 100644 --- a/src/Common/CaresPTRResolver.cpp +++ b/src/Common/CaresPTRResolver.cpp @@ -15,8 +15,8 @@ namespace DB static void callback(void * arg, int status, int, struct hostent * host) { - auto * ptr_records = reinterpret_cast*>(arg); - if (status == ARES_SUCCESS && host->h_aliases) + auto * ptr_records = static_cast*>(arg); + if (ptr_records && status == ARES_SUCCESS) { /* * In some cases (e.g /etc/hosts), hostent::h_name is filled and hostent::h_aliases is empty. @@ -28,11 +28,14 @@ namespace DB ptr_records->insert(ptr_record); } - int i = 0; - while (auto * ptr_record = host->h_aliases[i]) + if (host->h_aliases) { - ptr_records->insert(ptr_record); - i++; + int i = 0; + while (auto * ptr_record = host->h_aliases[i]) + { + ptr_records->insert(ptr_record); + i++; + } } } } From 304dc30f0f96750f89a9a00eb29efb3db2205209 Mon Sep 17 00:00:00 2001 From: Yuriy Chernyshov Date: Thu, 1 Sep 2022 18:15:56 +0300 Subject: [PATCH 38/39] Remove remaining usage of // Y_IGNORE magic comment --- src/IO/S3Common.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index fb9cff5d109..1ff1c609952 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -24,7 +24,7 @@ # include # include # include -# include // Y_IGNORE +# include # include # include From e64436fef3065b0a55477f7e4d1eeab10f75417d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 2 Sep 2022 08:54:48 +0000 Subject: [PATCH 39/39] Fix typos with new codespell --- base/base/find_symbols.h | 2 +- base/glibc-compatibility/memcpy/memcpy.h | 2 +- base/pcg-random/pcg_random.hpp | 2 +- base/pcg-random/pcg_uint128.hpp | 2 +- programs/client/Client.cpp | 2 +- programs/git-import/git-import.cpp | 2 +- src/Client/Connection.cpp | 2 +- src/Client/HedgedConnectionsFactory.cpp | 2 +- src/Common/IntervalKind.h | 2 +- src/Common/SLRUCachePolicy.h | 2 +- src/Common/Stopwatch.h | 2 +- src/Common/Volnitsky.h | 2 +- src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp | 2 +- src/Coordination/KeeperSnapshotManager.h | 2 +- src/Coordination/KeeperStorage.cpp | 2 +- src/Coordination/SessionExpiryQueue.h | 2 +- src/Coordination/tests/gtest_coordination.cpp | 2 +- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Core/SettingsEnums.h | 2 +- src/Dictionaries/HierarchyDictionariesUtils.h | 2 +- src/Dictionaries/MongoDBDictionarySource.cpp | 2 +- src/Dictionaries/getDictionaryConfigurationFromAST.cpp | 2 +- src/Disks/ObjectStorages/IMetadataStorage.h | 2 +- src/Functions/FunctionHelpers.h | 2 +- src/Interpreters/TreeOptimizer.cpp | 2 +- src/Parsers/InsertQuerySettingsPushDownVisitor.cpp | 2 +- src/Parsers/QueryWithOutputSettingsPushDownVisitor.h | 2 +- src/Processors/Sources/SQLiteSource.cpp | 2 +- src/Processors/Transforms/MergeJoinTransform.cpp | 2 +- src/Storages/AlterCommands.cpp | 2 +- src/Storages/Kafka/StorageKafka.cpp | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 2 +- src/Storages/MergeTree/MergeTreeBackgroundExecutor.h | 2 +- src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h | 2 +- src/Storages/MergeTree/ReplicatedMergeTreeQueue.h | 2 +- src/Storages/PartitionCommands.h | 2 +- src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/System/StorageSystemDistributionQueue.cpp | 2 +- utils/check-style/check-typos | 2 +- utils/check-style/codespell-ignore-words.list | 5 +++++ 42 files changed, 46 insertions(+), 41 deletions(-) diff --git a/base/base/find_symbols.h b/base/base/find_symbols.h index b28749afda6..83f53773ae7 100644 --- a/base/base/find_symbols.h +++ b/base/base/find_symbols.h @@ -15,7 +15,7 @@ * * Allow to search for next character from the set of 'symbols...' in a string. * It is similar to 'strpbrk', 'strcspn' (and 'strchr', 'memchr' in the case of one symbol and '\0'), - * but with the following differencies: + * but with the following differences: * - works with any memory ranges, including containing zero bytes; * - doesn't require terminating zero byte: end of memory range is passed explicitly; * - if not found, returns pointer to end instead of nullptr; diff --git a/base/glibc-compatibility/memcpy/memcpy.h b/base/glibc-compatibility/memcpy/memcpy.h index 9bee26a3722..ff27c970bac 100644 --- a/base/glibc-compatibility/memcpy/memcpy.h +++ b/base/glibc-compatibility/memcpy/memcpy.h @@ -63,7 +63,7 @@ * Very large size of memcpy typically indicates suboptimal (not cache friendly) algorithms in code or unrealistic scenarios, * so we don't pay attention to using non-temporary stores. * - * On recent Intel CPUs, the presence of "erms" makes "rep movsb" the most benefitial, + * On recent Intel CPUs, the presence of "erms" makes "rep movsb" the most beneficial, * even comparing to non-temporary aligned unrolled stores even with the most wide registers. * * memcpy can be written in asm, C or C++. The latter can also use inline asm. diff --git a/base/pcg-random/pcg_random.hpp b/base/pcg-random/pcg_random.hpp index 94e43e1007b..db7c3d7f66c 100644 --- a/base/pcg-random/pcg_random.hpp +++ b/base/pcg-random/pcg_random.hpp @@ -101,7 +101,7 @@ #endif /* - * The pcg_extras namespace contains some support code that is likley to + * The pcg_extras namespace contains some support code that is likely to * be useful for a variety of RNGs, including: * - 128-bit int support for platforms where it isn't available natively * - bit twiddling operations diff --git a/base/pcg-random/pcg_uint128.hpp b/base/pcg-random/pcg_uint128.hpp index 1a1f61b9366..3452ba6f1c4 100644 --- a/base/pcg-random/pcg_uint128.hpp +++ b/base/pcg-random/pcg_uint128.hpp @@ -22,7 +22,7 @@ /* * This code provides a a C++ class that can provide 128-bit (or higher) * integers. To produce 2K-bit integers, it uses two K-bit integers, - * placed in a union that allowes the code to also see them as four K/2 bit + * placed in a union that allows the code to also see them as four K/2 bit * integers (and access them either directly name, or by index). * * It may seem like we're reinventing the wheel here, because several diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index e27845de184..6506c23428a 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -723,7 +723,7 @@ bool Client::processWithFuzzing(const String & full_query) // queries, for lack of a better solution. // There is also a problem that fuzzer substitutes positive Int64 // literals or Decimal literals, which are then parsed back as - // UInt64, and suddenly duplicate alias substitition starts or stops + // UInt64, and suddenly duplicate alias substitution starts or stops // working (ASTWithAlias::formatImpl) or something like that. // So we compare not even the first and second formatting of the // query, but second and third. diff --git a/programs/git-import/git-import.cpp b/programs/git-import/git-import.cpp index 54f5d7f75ea..030ddd263fa 100644 --- a/programs/git-import/git-import.cpp +++ b/programs/git-import/git-import.cpp @@ -67,7 +67,7 @@ Run this tool inside your git repository. It will create .tsv files that can be The tool can process large enough repositories in a reasonable time. It has been tested on: - ClickHouse: 31 seconds; 3 million rows; -- LLVM: 8 minues; 62 million rows; +- LLVM: 8 minutes; 62 million rows; - Linux - 12 minutes; 85 million rows; - Chromium - 67 minutes; 343 million rows; (the numbers as of Sep 2020) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 30f5af5a540..7a663195655 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -557,7 +557,7 @@ void Connection::sendQuery( /// Send correct hash only for !INITIAL_QUERY, due to: /// - this will avoid extra protocol complexity for simplest cases /// - there is no need in hash for the INITIAL_QUERY anyway - /// (since there is no secure/unsecure changes) + /// (since there is no secure/non-secure changes) if (client_info && !cluster_secret.empty() && client_info->query_kind != ClientInfo::QueryKind::INITIAL_QUERY) { #if USE_SSL diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index a1b816deecb..81067f51d29 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -41,7 +41,7 @@ HedgedConnectionsFactory::HedgedConnectionsFactory( HedgedConnectionsFactory::~HedgedConnectionsFactory() { /// Stop anything that maybe in progress, - /// to avoid interfer with the subsequent connections. + /// to avoid interference with the subsequent connections. /// /// I.e. some replcas may be in the establishing state, /// this means that hedged connection is waiting for TablesStatusResponse, diff --git a/src/Common/IntervalKind.h b/src/Common/IntervalKind.h index 59e8d32f3e7..b46805655b1 100644 --- a/src/Common/IntervalKind.h +++ b/src/Common/IntervalKind.h @@ -64,7 +64,7 @@ struct IntervalKind const char * toNameOfFunctionExtractTimePart() const; /// Converts the string representation of an interval kind to its IntervalKind equivalent. - /// Returns false if the conversion unsucceeded. + /// Returns false if the conversion did not succeed. /// For example, `IntervalKind::tryParseString('second', result)` returns `result` equals `IntervalKind::Kind::Second`. static bool tryParseString(const std::string & kind, IntervalKind::Kind & result); }; diff --git a/src/Common/SLRUCachePolicy.h b/src/Common/SLRUCachePolicy.h index 10b043ebaca..8d4709c66a7 100644 --- a/src/Common/SLRUCachePolicy.h +++ b/src/Common/SLRUCachePolicy.h @@ -33,7 +33,7 @@ public: * max_protected_size shows how many of the most frequently used entries will not be evicted after a sequential scan. * max_protected_size == 0 means that the default protected size is equal to half of the total max size. */ - /// TODO: construct from special struct with cache policy parametrs (also with max_protected_size). + /// TODO: construct from special struct with cache policy parameters (also with max_protected_size). SLRUCachePolicy(size_t max_size_, size_t max_elements_size_ = 0, double size_ratio = 0.5, OnWeightLossFunction on_weight_loss_function_ = {}) : max_protected_size(max_size_ * std::min(1.0, size_ratio)) , max_size(max_size_) diff --git a/src/Common/Stopwatch.h b/src/Common/Stopwatch.h index 2b60bbde196..9e48c33f70d 100644 --- a/src/Common/Stopwatch.h +++ b/src/Common/Stopwatch.h @@ -31,7 +31,7 @@ inline UInt64 clock_gettime_ns_adjusted(UInt64 prev_time, clockid_t clock_type = } /** Differs from Poco::Stopwatch only by using 'clock_gettime' instead of 'gettimeofday', - * returns nanoseconds instead of microseconds, and also by other minor differencies. + * returns nanoseconds instead of microseconds, and also by other minor differences. */ class Stopwatch { diff --git a/src/Common/Volnitsky.h b/src/Common/Volnitsky.h index d7ca7d35277..6f5948b6564 100644 --- a/src/Common/Volnitsky.h +++ b/src/Common/Volnitsky.h @@ -497,7 +497,7 @@ private: /// last index of offsets that was not processed size_t last; - /// limit for adding to hashtable. In worst case with case insentive search, the table will be filled at most as half + /// limit for adding to hashtable. In worst case with case insensitive search, the table will be filled at most as half static constexpr size_t small_limit = VolnitskyTraits::hash_size / 8; public: diff --git a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp index 9e4d4a3241f..3e3d0e164fe 100644 --- a/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp +++ b/src/Compression/fuzzers/encrypted_decompress_fuzzer.cpp @@ -58,7 +58,7 @@ Fuzzing data consists of: else: read_key() if (7): - read_nonce (simillar to read_key) + read_nonce (similar to read_key) if (8): set current_key diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 4984e54f15f..c00ce9421e7 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -27,7 +27,7 @@ enum SnapshotVersion : uint8_t static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V5; -/// What is stored in binary shapsnot +/// What is stored in binary snapshot struct SnapshotDeserializationResult { /// Storage diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 397cd2c0c71..711f3874868 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -2192,7 +2192,7 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) } catch (...) { - LOG_FATAL(&Poco::Logger::get("KeeperStorage"), "Failed to rollback log. Terminating to avoid incosistencies"); + LOG_FATAL(&Poco::Logger::get("KeeperStorage"), "Failed to rollback log. Terminating to avoid inconsistencies"); std::terminate(); } } diff --git a/src/Coordination/SessionExpiryQueue.h b/src/Coordination/SessionExpiryQueue.h index 8581800834d..862ec35e2f6 100644 --- a/src/Coordination/SessionExpiryQueue.h +++ b/src/Coordination/SessionExpiryQueue.h @@ -53,7 +53,7 @@ public: /// Session was actually removed bool remove(int64_t session_id); - /// Update session expiry time (must be called on hearbeats) + /// Update session expiry time (must be called on heartbeats) void addNewSessionOrUpdate(int64_t session_id, int64_t timeout_ms); /// Get all expired sessions diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 493e76ee5fc..ee7d0b8ec83 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -1339,7 +1339,7 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, uint nuraft::async_result::handler_type when_done = [&snapshot_created] (bool & ret, nuraft::ptr &/*exception*/) { snapshot_created = ret; - std::cerr << "Snapshot finised\n"; + std::cerr << "Snapshot finished\n"; }; state_machine->create_snapshot(s, when_done); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2a987c840cd..baaeaccdae9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -149,7 +149,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) \ M(UInt64, parallel_distributed_insert_select, 0, "Process distributed INSERT SELECT query in the same cluster on local tables on every shard; if set to 1 - SELECT is executed on each shard; if set to 2 - SELECT and INSERT are executed on each shard", 0) \ M(UInt64, distributed_group_by_no_merge, 0, "If 1, Do not merge aggregation states from different servers for distributed queries (shards will process query up to the Complete stage, initiator just proxies the data from the shards). If 2 the initiator will apply ORDER BY and LIMIT stages (it is not in case when shard process query up to the Complete stage)", 0) \ - M(UInt64, distributed_push_down_limit, 1, "If 1, LIMIT will be applied on each shard separatelly. Usually you don't need to use it, since this will be done automatically if it is possible, i.e. for simple query SELECT FROM LIMIT.", 0) \ + M(UInt64, distributed_push_down_limit, 1, "If 1, LIMIT will be applied on each shard separately. Usually you don't need to use it, since this will be done automatically if it is possible, i.e. for simple query SELECT FROM LIMIT.", 0) \ M(Bool, optimize_distributed_group_by_sharding_key, true, "Optimize GROUP BY sharding_key queries (by avoiding costly aggregation on the initiator server).", 0) \ M(UInt64, optimize_skip_unused_shards_limit, 1000, "Limit for number of sharding key values, turns off optimize_skip_unused_shards if the limit is reached", 0) \ M(Bool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 8d0e69f4b29..be2def2c01a 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -89,7 +89,7 @@ static std::map sett {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature bu default"}}}, + {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 518c626fcc0..b5e908defc7 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -153,7 +153,7 @@ enum class HandleKafkaErrorMode { DEFAULT = 0, // Ignore errors with threshold. STREAM, // Put errors to stream in the virtual column named ``_error. - /*FIXED_SYSTEM_TABLE, Put errors to in a fixed system table likey system.kafka_errors. This is not implemented now. */ + /*FIXED_SYSTEM_TABLE, Put errors to in a fixed system table likely system.kafka_errors. This is not implemented now. */ /*CUSTOM_SYSTEM_TABLE, Put errors to in a custom system table. This is not implemented now. */ }; diff --git a/src/Dictionaries/HierarchyDictionariesUtils.h b/src/Dictionaries/HierarchyDictionariesUtils.h index 621290f40f9..c7508ddd220 100644 --- a/src/Dictionaries/HierarchyDictionariesUtils.h +++ b/src/Dictionaries/HierarchyDictionariesUtils.h @@ -180,7 +180,7 @@ namespace detail /** Returns array with UInt8 represent if key from in_keys array is in hierarchy of key from keys column. * If value in result array is 1 that means key from in_keys array is in hierarchy of key from - * keys array with same index, 0 therwise. + * keys array with same index, 0 otherwise. * For getting hierarchy implementation uses getKeysHierarchy function. * * Not: keys size must be equal to in_keys_size. diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index 1ede0ec5045..a735f426ec7 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -118,7 +118,7 @@ MongoDBDictionarySource::MongoDBDictionarySource( Poco::URI poco_uri(uri); // Parse database from URI. This is required for correctness -- the - // cursor is created using database name and colleciton name, so we have + // cursor is created using database name and collection name, so we have // to specify them properly. db = poco_uri.getPath(); // getPath() may return a leading slash, remove it. diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index edc3c34fe81..7d8253c47ce 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -244,7 +244,7 @@ void buildAttributeExpressionIfNeeded( root->appendChild(expression_element); } -/** Transofrms single dictionary attribute to configuration +/** Transforms single dictionary attribute to configuration * third_column UInt8 DEFAULT 2 EXPRESSION rand() % 100 * 77 * to * diff --git a/src/Disks/ObjectStorages/IMetadataStorage.h b/src/Disks/ObjectStorages/IMetadataStorage.h index 300d8ec59b9..3d6c772157d 100644 --- a/src/Disks/ObjectStorages/IMetadataStorage.h +++ b/src/Disks/ObjectStorages/IMetadataStorage.h @@ -124,7 +124,7 @@ public: virtual ~IMetadataStorage() = default; - /// ==== More specefic methods. Previous were almost general purpose. ==== + /// ==== More specific methods. Previous were almost general purpose. ==== /// Read multiple metadata files into strings and return mapping from file_path -> metadata virtual std::unordered_map getSerializedMetadata(const std::vector & file_paths) const = 0; diff --git a/src/Functions/FunctionHelpers.h b/src/Functions/FunctionHelpers.h index 8d33c820185..18a4e584080 100644 --- a/src/Functions/FunctionHelpers.h +++ b/src/Functions/FunctionHelpers.h @@ -134,7 +134,7 @@ using FunctionArgumentDescriptors = std::vector; * (e.g. depending on result type or other trait). * First, checks that number of arguments is as expected (including optional arguments). * Second, checks that mandatory args present and have valid type. - * Third, checks optional arguents types, skipping ones that are missing. + * Third, checks optional arguments types, skipping ones that are missing. * * Please note that if you have several optional arguments, like f([a, b, c]), * only these calls are considered valid: diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 07c7cd85d1b..eaf59731967 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -453,7 +453,7 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context return; /// Do not apply optimization for Distributed and Merge storages, - /// because we can't get the sorting key of their undelying tables + /// because we can't get the sorting key of their underlying tables /// and we can break the matching of the sorting key for `read_in_order` /// optimization by removing monotonous functions from the prefix of key. if (result.is_remote_storage || (result.storage && result.storage->getName() == "Merge")) diff --git a/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp b/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp index a3bca76816f..1cebdfde957 100644 --- a/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp +++ b/src/Parsers/InsertQuerySettingsPushDownVisitor.cpp @@ -55,7 +55,7 @@ void InsertQuerySettingsPushDownMatcher::visit(ASTSelectQuery & select_query, AS insert_settings.push_back(setting); else { - /// Do not ovewrite setting that was passed for INSERT + /// Do not overwrite setting that was passed for INSERT /// by settings that was passed for SELECT } } diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h index 2a7ed0125fa..fde8a07b555 100644 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h @@ -11,7 +11,7 @@ struct SettingChange; class SettingsChanges; /// Pushdown SETTINGS clause that goes after FORMAT to the SELECT query: -/// (since settings after FORMAT parsed separatelly not in the ParserSelectQuery but in ParserQueryWithOutput) +/// (since settings after FORMAT parsed separately not in the ParserSelectQuery but in ParserQueryWithOutput) /// /// SELECT 1 FORMAT Null SETTINGS max_block_size = 1 -> /// SELECT 1 SETTINGS max_block_size = 1 FORMAT Null SETTINGS max_block_size = 1 diff --git a/src/Processors/Sources/SQLiteSource.cpp b/src/Processors/Sources/SQLiteSource.cpp index 814480b63e3..60d39966659 100644 --- a/src/Processors/Sources/SQLiteSource.cpp +++ b/src/Processors/Sources/SQLiteSource.cpp @@ -39,7 +39,7 @@ SQLiteSource::SQLiteSource( if (status != SQLITE_OK) throw Exception(ErrorCodes::SQLITE_ENGINE_ERROR, - "Cannot prepate sqlite statement. Status: {}. Message: {}", + "Cannot prepare sqlite statement. Status: {}. Message: {}", status, sqlite3_errstr(status)); compiled_statement = std::unique_ptr(compiled_stmt, StatementDeleter()); diff --git a/src/Processors/Transforms/MergeJoinTransform.cpp b/src/Processors/Transforms/MergeJoinTransform.cpp index 6f842bec939..315fc4810ba 100644 --- a/src/Processors/Transforms/MergeJoinTransform.cpp +++ b/src/Processors/Transforms/MergeJoinTransform.cpp @@ -513,7 +513,7 @@ MergeJoinAlgorithm::Status MergeJoinAlgorithm::allJoin(JoinKind kind) Columns lcols; if (!left_to_right_key_remap.empty()) { - /// If we have remapped columns, then we need to get values from right columns insead of defaults + /// If we have remapped columns, then we need to get values from right columns instead of defaults const auto & indices = idx_map[0]; const auto & left_src = cursors[0]->getCurrent().getColumns(); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 7f99abf31fc..d370a67bfcc 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1274,7 +1274,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const throw Exception{"Table doesn't have SAMPLE BY, cannot remove", ErrorCodes::BAD_ARGUMENTS}; } - /// Collect default expressions for MODIFY and ADD comands + /// Collect default expressions for MODIFY and ADD commands if (command.type == AlterCommand::MODIFY_COLUMN || command.type == AlterCommand::ADD_COLUMN) { if (command.default_expression) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index b51457ba5d5..96987b0f60e 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -810,7 +810,7 @@ void registerStorageKafka(StorageFactory & factory) /** Arguments of engine is following: * - Kafka broker list * - List of topics - * - Group ID (may be a constaint expression with a string result) + * - Group ID (may be a constraint expression with a string result) * - Message format (string) * - Row delimiter * - Schema (optional, if the format supports it) diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 6a079aa832f..b37dec6cbf1 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -382,7 +382,7 @@ bool StorageLiveView::getNewBlocks() BlocksMetadataPtr new_blocks_metadata = std::make_shared(); /// can't set mergeable_blocks here or anywhere else outside the writeIntoLiveView function - /// as there could be a race codition when the new block has been inserted into + /// as there could be a race condition when the new block has been inserted into /// the source table by the PushingToViews chain and this method /// called before writeIntoLiveView function is called which can lead to /// the same block added twice to the mergeable_blocks leading to diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h index a2f97d6bbc6..0fc888dd6ad 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h @@ -155,7 +155,7 @@ private: * We use boost::circular_buffer as a container for queues not to do any allocations. * * Another nuisance that we faces with is than background operations always interact with an associated Storage. - * So, when a Storage want to shutdown, it must wait until all its background operaions are finished. + * So, when a Storage want to shutdown, it must wait until all its background operations are finished. */ template class MergeTreeBackgroundExecutor final : boost::noncopyable diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h index 71ef6edd7fb..3080e285ac8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h @@ -16,7 +16,7 @@ struct MergeTreeDataPartTTLInfo time_t max = 0; /// This TTL was computed on completely expired part. It doesn't make sense - /// to select such parts for TTL again. But make sense to recalcuate TTL + /// to select such parts for TTL again. But make sense to recalculate TTL /// again for merge with multiple parts. std::optional ttl_finished; bool finished() const { return ttl_finished.value_or(false); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index e8362e5cc6b..6d1a3efb01d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -279,7 +279,7 @@ private: /// Very large queue entries may appear occasionally. /// We cannot process MAX_MULTI_OPS at once because it will fail. /// But we have to process more than one entry at once because otherwise lagged replicas keep up slowly. - /// Let's start with one entry per transaction and icrease it exponentially towards MAX_MULTI_OPS. + /// Let's start with one entry per transaction and increase it exponentially towards MAX_MULTI_OPS. /// It will allow to make some progress before failing and remain operational even in extreme cases. size_t current_multi_batch_size = 1; diff --git a/src/Storages/PartitionCommands.h b/src/Storages/PartitionCommands.h index 9807c90bc23..4921cf8e53b 100644 --- a/src/Storages/PartitionCommands.h +++ b/src/Storages/PartitionCommands.h @@ -104,7 +104,7 @@ struct PartitionCommandResultInfo using PartitionCommandsResultInfo = std::vector; -/// Convert partition comands result to Source from single Chunk, which will be +/// Convert partition commands result to Source from single Chunk, which will be /// used to print info to the user. Tries to create narrowest table for given /// results. For example, if all commands were FREEZE commands, than /// old_part_name column will be absent. diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 91bf5eeccde..37caa66aae5 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -126,7 +126,7 @@ private: static void assertCorrectInsertion(StorageData::Buffer & buffer, size_t column_idx); - /// lsn - log sequnce nuumber, like wal offset (64 bit). + /// lsn - log sequence number, like wal offset (64 bit). static Int64 getLSNValue(const std::string & lsn) { UInt32 upper_half, lower_half; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 507047751f3..5ce46ecffc8 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -963,7 +963,7 @@ bool StorageMergeTree::merge( if (!merge_mutate_entry) return false; - /// Copying a vector of columns `deduplicate bu columns. + /// Copying a vector of columns `deduplicate by columns. IExecutableTask::TaskResultCallback f = [](bool) {}; auto task = std::make_shared( *this, metadata_snapshot, deduplicate, deduplicate_by_columns, merge_mutate_entry, table_lock_holder, f); diff --git a/src/Storages/System/StorageSystemDistributionQueue.cpp b/src/Storages/System/StorageSystemDistributionQueue.cpp index d8879c3655e..5297c4eb93c 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.cpp +++ b/src/Storages/System/StorageSystemDistributionQueue.cpp @@ -57,7 +57,7 @@ std::string maskDataPath(const std::string & path) size_t user_pw_end = masked_path.find('@', node_pos); if (user_pw_end == std::string::npos) { - /// Likey new format (use_compact_format_in_distributed_parts_names=1) + /// Likely new format (use_compact_format_in_distributed_parts_names=1) return path; } diff --git a/utils/check-style/check-typos b/utils/check-style/check-typos index 3819b6785d7..9194a9464a7 100755 --- a/utils/check-style/check-typos +++ b/utils/check-style/check-typos @@ -5,7 +5,7 @@ ROOT_PATH=$(git rev-parse --show-toplevel) codespell \ - --skip "*generated*,*gperf*,*.bin,*.mrk*,*.idx,checksums.txt,*.dat,*.pyc,*.kate-swp,*obfuscateQueries.cpp,d3-*.js,*.min.js,${ROOT_PATH}/utils/check-style/aspell-ignore" \ + --skip "*generated*,*gperf*,*.bin,*.mrk*,*.idx,checksums.txt,*.dat,*.pyc,*.kate-swp,*obfuscateQueries.cpp,d3-*.js,*.min.js,*.sum,${ROOT_PATH}/utils/check-style/aspell-ignore" \ --ignore-words "${ROOT_PATH}/utils/check-style/codespell-ignore-words.list" \ --exclude-file "${ROOT_PATH}/utils/check-style/codespell-ignore-lines.list" \ --quiet-level 2 \ diff --git a/utils/check-style/codespell-ignore-words.list b/utils/check-style/codespell-ignore-words.list index 062e8a1622b..fc021920041 100644 --- a/utils/check-style/codespell-ignore-words.list +++ b/utils/check-style/codespell-ignore-words.list @@ -16,3 +16,8 @@ ot te fo ba +ro +rightt +iiterator +hastable +nam