From ee1b3042848b0dc9c1362e4dce89a44bb7f8abbe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Dec 2023 20:56:10 +0100 Subject: [PATCH 001/254] Asynchronous load of tables by default --- programs/server/config.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 52a1c528040c..86fb7dfab1a9 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -371,7 +371,7 @@ - + true From 7cbbb1601b7d4418d061179968f89c25e0ee3b75 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 3 Mar 2024 22:39:29 +0100 Subject: [PATCH 002/254] Update test --- .../0_stateless/01676_dictget_in_default_expression.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01676_dictget_in_default_expression.sql b/tests/queries/0_stateless/01676_dictget_in_default_expression.sql index 54e46a2b7183..1785979f60b7 100644 --- a/tests/queries/0_stateless/01676_dictget_in_default_expression.sql +++ b/tests/queries/0_stateless/01676_dictget_in_default_expression.sql @@ -22,7 +22,8 @@ DETACH DATABASE test_01676; ATTACH DATABASE test_01676; SELECT 'status_after_detach_and_attach:'; -SELECT status FROM system.dictionaries WHERE database='test_01676' AND name='dict'; +-- It can be not loaded, or not even finish attaching in case of asynchronous tables loading. +SELECT COALESCE((SELECT status FROM system.dictionaries WHERE database='test_01676' AND name='dict')::String, 'NOT_LOADED'); INSERT INTO test_01676.table (x) VALUES (toInt64(4)); SELECT * FROM test_01676.table ORDER BY x; From 3c1e6b5d2c52d8f54705428199d007a0f58857bb Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 7 Mar 2024 13:11:42 +0000 Subject: [PATCH 003/254] init --- src/Interpreters/InterpreterInsertQuery.cpp | 17 +- src/Interpreters/SquashingTransform.cpp | 194 +++++++++++++++ src/Interpreters/SquashingTransform.h | 63 +++++ .../Transforms/BalancingTransform.cpp | 223 ++++++++++++++++++ .../Transforms/BalancingTransform.h | 128 ++++++++++ .../Transforms/SquashingChunksTransform.cpp | 136 ++++++++++- .../Transforms/SquashingChunksTransform.h | 34 ++- 7 files changed, 787 insertions(+), 8 deletions(-) create mode 100644 src/Processors/Transforms/BalancingTransform.cpp create mode 100644 src/Processors/Transforms/BalancingTransform.h diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index e27a8bd414ba..0041a0f08468 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -26,6 +26,7 @@ #include #include #include +#include #include #include #include @@ -604,9 +605,15 @@ BlockIO InterpreterInsertQuery::execute() { bool table_prefers_large_blocks = table->prefersLargeBlocks(); + pipeline.addTransform(std::make_shared( + header, + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, + settings.max_memory_usage, presink_chains.size())); + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { - return std::make_shared( + return std::make_shared( in_header, table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); @@ -668,6 +675,14 @@ BlockIO InterpreterInsertQuery::execute() table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); chain.addSource(std::move(squashing)); + + // auto balancing = std::make_shared( + // chain.getInputHeader(), + // table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + // table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, + // settings.max_memory_usage, true); + + // chain.addSource(std::move(balancing)); } auto context_ptr = getContext(); diff --git a/src/Interpreters/SquashingTransform.cpp b/src/Interpreters/SquashingTransform.cpp index 4ed0dddc1914..0d976bd967ad 100644 --- a/src/Interpreters/SquashingTransform.cpp +++ b/src/Interpreters/SquashingTransform.cpp @@ -1,4 +1,12 @@ +#include +#include #include +#include "DataTypes/Serializations/ISerialization.h" +#include "Processors/Chunk.h" +#include "base/sleep.h" +#include "base/types.h" +#include +#include namespace DB @@ -126,4 +134,190 @@ bool SquashingTransform::isEnoughSize(size_t rows, size_t bytes) const || (min_block_size_bytes && bytes >= min_block_size_bytes); } + +NewSquashingTransform::NewSquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_) + : min_block_size_rows(min_block_size_rows_) + , min_block_size_bytes(min_block_size_bytes_) +{ +} + +Block NewSquashingTransform::add(Chunk && input_chunk) +{ + return addImpl(std::move(input_chunk)); +} + +const ChunksToSquash * getInfoFromChunk(const Chunk & chunk) +{ + auto info = chunk.getChunkInfo(); + const auto * agg_info = typeid_cast(info.get()); + + return agg_info; +} + +template +Block NewSquashingTransform::addImpl(ReferenceType input_chunk) +{ + if (!input_chunk.hasChunkInfo()) + { + Block to_return; + std::swap(to_return, accumulated_block); + return to_return; + } + + const auto *info = getInfoFromChunk(input_chunk); + for (auto & one : info->chunks) + { + append(std::move(one), info->data_type); + } + + // if (isEnoughSize(accumulated_block)) + { + Block to_return; + std::swap(to_return, accumulated_block); + return to_return; + } +} + +template +void NewSquashingTransform::append(ReferenceType input_chunk, DataTypePtr data_type) +{ + if (input_chunk.getNumColumns() == 0) + return; + if (!accumulated_block) + { + for (const ColumnPtr& column : input_chunk.getColumns()) + { + ColumnWithTypeAndName col = ColumnWithTypeAndName(column, data_type, " "); + accumulated_block.insert(accumulated_block.columns(), col); + } + return; + } + + for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) + { + const auto source_column = input_chunk.getColumns()[i]; + + auto mutable_column = IColumn::mutate(std::move(accumulated_block.getByPosition(i).column)); + mutable_column->insertRangeFrom(*source_column, 0, source_column->size()); + accumulated_block.getByPosition(i).column = std::move(mutable_column); + } +} + + + +BalanceTransform::BalanceTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) + : min_block_size_rows(min_block_size_rows_) + , min_block_size_bytes(min_block_size_bytes_) + , header(std::move(header_)) +{ + // Use query-level memory tracker + if (auto * memory_tracker_child = CurrentThread::getMemoryTracker()) + memory_tracker = memory_tracker_child->getParent(); +} + +Chunk BalanceTransform::add(Block && input_block) +{ + return addImpl(std::move(input_block)); +} + +Chunk BalanceTransform::convertToChunk(std::vector &chunks) +{ + if (chunks.empty()) + return {}; + + auto info = std::make_shared(); + for (auto &chunk : chunks) + info->chunks.push_back(chunk.clone()); + info->data_type = data_type; + + if (!info->chunks.empty()) /// Note: This if is only for debugging, structure of chunk copies the structure of info + { /// it's possible to use only 'Chunk(header.cloneEmptyColumns(), 0, info)' + return Chunk({info->chunks[0].getColumns(), info->chunks[0].getNumRows(), info}); + } + + return Chunk(header.cloneEmptyColumns(), 0, info); +} + + +template +Chunk BalanceTransform::addImpl(ReferenceType input_block) +{ + Chunk input_chunk(input_block.getColumns(), input_block.rows()); + if (!data_type && !input_block.getDataTypes().empty()) + data_type = input_block.getDataTypes()[0]; + // /// End of input stream. + if (!input_chunk) + { + Chunk res_chunk = convertToChunk(chunks_to_merge_vec); + // // std::cerr << "end of stream. Adding info to chunk " << std::endl; + return res_chunk; + } + + if (isEnoughSize(chunks_to_merge_vec)) + chunks_to_merge_vec.clear(); + + if (input_chunk) + chunks_to_merge_vec.push_back(input_chunk.clone()); + // std::cerr << "pushing back data. size: " << chunks_to_merge_vec.size() << std::endl; + + if (isEnoughSize(chunks_to_merge_vec)) + { + // // // std::cerr << "enough size" << std::endl; + Chunk res_chunk = convertToChunk(chunks_to_merge_vec); + return res_chunk; + } + return input_chunk; +} + +bool BalanceTransform::isEnoughSize(const std::vector & chunks) +{ + size_t rows = 0; + size_t bytes = 0; + + for (const Chunk & chunk : chunks) + { + rows += chunk.getNumRows(); + bytes += chunk.bytes(); + } + auto free_memory = memory_tracker->getHardLimit() - memory_tracker->get(); + std::cerr << "========Just memory representation, free memory: " << free_memory << ", chunk size: " << bytes << std::endl + << " hardLimit: " << memory_tracker->getHardLimit() << " get(): " << memory_tracker->get() << std::endl; + checkAndWaitMemoryAvailability(bytes); + + free_memory = memory_tracker->getHardLimit() - memory_tracker->get(); + std::cerr << "========Just memory representation after, free memory: " << free_memory << ", chunk size: " << bytes << std::endl + << ", hardLimit: " << memory_tracker->getHardLimit() << ", get(): " << memory_tracker->get() << std::endl; + + return isEnoughSize(rows, bytes); +} + +void BalanceTransform::checkAndWaitMemoryAvailability(size_t bytes) +{ + // bytes_used += bytes; + if (const auto hard_limit = memory_tracker->getHardLimit() != 0) + { + auto free_memory = hard_limit - memory_tracker->get(); + while (Int64(bytes) >= free_memory) + { + // std::cerr << "========Waiting a while from memory, free memory: " << free_memory << ", chunk size: " << bytes << std::endl; + // sleepForMilliseconds(10); + // checkAndWaitMemoryAvailability(bytes); + free_memory = hard_limit - memory_tracker->get(); + } + } +} + +bool BalanceTransform::isEnoughSize(const Chunk & chunk) +{ + return isEnoughSize(chunk.getNumRows(), chunk.bytes()); +} + + +bool BalanceTransform::isEnoughSize(size_t rows, size_t bytes) const +{ + return (!min_block_size_rows && !min_block_size_bytes) + || (min_block_size_rows && rows >= min_block_size_rows) + || (min_block_size_bytes && bytes >= min_block_size_bytes); +} + } diff --git a/src/Interpreters/SquashingTransform.h b/src/Interpreters/SquashingTransform.h index b04d012bcd1d..0c2fe1ef12b4 100644 --- a/src/Interpreters/SquashingTransform.h +++ b/src/Interpreters/SquashingTransform.h @@ -1,11 +1,22 @@ #pragma once +#include +#include +#include #include +#include +#include "Common/MemoryTracker.h" +#include "DataTypes/Serializations/ISerialization.h" namespace DB { +struct ChunksToSquash : public ChunkInfo +{ + mutable std::vector chunks = {}; + DataTypePtr data_type = nullptr; +}; /** Merging consecutive passed blocks to specified minimum size. * @@ -47,4 +58,56 @@ class SquashingTransform bool isEnoughSize(size_t rows, size_t bytes) const; }; +class NewSquashingTransform +{ +public: + NewSquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_); + + Block add(Chunk && input_chunk); + +private: + size_t min_block_size_rows; + size_t min_block_size_bytes; + + Block accumulated_block; + + template + Block addImpl(ReferenceType chunk); + + template + void append(ReferenceType input_chunk, DataTypePtr data_type); + + bool isEnoughSize(const Block & block); + bool isEnoughSize(size_t rows, size_t bytes) const; +}; + +class BalanceTransform +{ +public: + BalanceTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); + + Chunk add(Block && input_block); + +private: + std::vector chunks_to_merge_vec = {}; + size_t min_block_size_rows; + size_t min_block_size_bytes; + + Chunk accumulated_block; + const Block header; + + template + Chunk addImpl(ReferenceType input_block); + + bool isEnoughSize(const Chunk & chunk); + bool isEnoughSize(const std::vector & chunks); + bool isEnoughSize(size_t rows, size_t bytes) const; + void checkAndWaitMemoryAvailability(size_t bytes); + DataTypePtr data_type = nullptr; + + MemoryTracker * memory_tracker; + + Chunk convertToChunk(std::vector &chunks); +}; + } diff --git a/src/Processors/Transforms/BalancingTransform.cpp b/src/Processors/Transforms/BalancingTransform.cpp new file mode 100644 index 000000000000..b899702561e3 --- /dev/null +++ b/src/Processors/Transforms/BalancingTransform.cpp @@ -0,0 +1,223 @@ +#include +#include +#include "Common/Logger.h" +#include "Common/logger_useful.h" +#include "Interpreters/SquashingTransform.h" +#include "Processors/Chunk.h" + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int MEMORY_LIMIT_EXCEEDED; +} + +LBalancingChunksTransform::LBalancingChunksTransform( + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage_, [[maybe_unused]] bool skip_empty_chunks_) + : ISimpleTransform(header, header, false), max_memory_usage(max_memory_usage_), squashing(min_block_size_rows, min_block_size_bytes), balance(header, min_block_size_rows, min_block_size_bytes) +{ +} + +void LBalancingChunksTransform::transform(Chunk & chunk) +{ + if (!finished) + { + Chunk res_chunk = balance.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); + if (res_chunk.hasChunkInfo()) + { + // std::cerr << "BalancingTransform: adding chunk " << std::endl; + + // { + // [[maybe_unused]]const auto * agg_info = typeid_cast(res_chunk.getChunkInfo().get()); + // std::cerr << "End of BalancingTransform: size of one group: " << agg_info->chunks.size() << std::endl; + // if (!agg_info->chunks.empty()) + // std::cerr << "!group is not empty, first column: " << agg_info->chunks[0].dumpStructure() << std::endl << std::endl; + // } + + } + else + LOG_TRACE(getLogger("balancing"), "{}, BalancingTransform: not adding chunk, not finished.", reinterpret_cast(this));/// ISSUE: it's not clear why finished label is not set + std::swap(res_chunk, chunk); + } + else + { + Chunk res_chunk = balance.add({}); + if (res_chunk.hasChunkInfo()) + { + // std::cerr << "BalancingTransform: finished adding, NumRows:" << res_chunk.getNumRows() << ", HasInfo: " << res_chunk.hasChunkInfo() << std::endl; + // { + // [[maybe_unused]]const auto * agg_info = typeid_cast(res_chunk.getChunkInfo().get()); + // std::cerr << "End of BalancingTransform: size of one group: " << agg_info->chunks.size() << std::endl; + // if (!agg_info->chunks.empty()) + // std::cerr << "!group is not empty, first column: " << agg_info->chunks[0].dumpStructure() << std::endl << std::endl; + // } + + } + else + LOG_TRACE(getLogger("balancing"), "{}, BalancingTransform: not adding chunk on finished", reinterpret_cast(this)); + std::swap(res_chunk, chunk); + } + LOG_TRACE(getLogger("balancing"), "{}, BalancingTransform: struct of output chunk: {}", reinterpret_cast(this), chunk.dumpStructure()); +} + +IProcessor::Status LBalancingChunksTransform::prepare() +{ + if (!finished && input.isFinished()) + { + finished = true; + return Status::Ready; + } + return ISimpleTransform::prepare(); +} + + +BalancingChunksTransform::BalancingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage_, size_t num_ports) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), max_memory_usage(max_memory_usage_), squashing(min_block_size_rows, min_block_size_bytes), balance(header, min_block_size_rows, min_block_size_bytes) +{ +} + +IProcessor::Status BalancingChunksTransform::prepare() +{ + Status status = Status::Ready; + + while (status == Status::Ready) + { + status = !has_data ? prepareConsume() + : prepareSend(); + } + + return status; +} + +IProcessor::Status BalancingChunksTransform::prepareConsume() +{ + LOG_TRACE(getLogger("balancingProcessor"), "prepareConsume"); + for (auto & input : inputs) + { + bool all_finished = true; + for (auto & output : outputs) + { + if (output.isFinished()) + continue; + + all_finished = false; + } + + if (all_finished) + { + input.close(); + return Status::Finished; + } + + if (input.isFinished()) + { + for (auto & output : outputs) + output.finish(); + + return Status::Finished; + } + + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + + chunk = input.pull(); + was_output_processed.assign(outputs.size(), false); + transform(chunk); + if (chunk.hasChunkInfo()) + { + LOG_TRACE(getLogger("balancingProcessor"), "hasData"); + has_data = true; + } + else + { + finished = true; + LOG_TRACE(getLogger("balancingProcessor"), "hasData, finished"); + transform(chunk); + has_data = true; + } + } + return Status::Ready; +} + +void BalancingChunksTransform::transform(Chunk & chunk_) +{ + if (!finished) + { + Chunk res_chunk = balance.add(getInputPorts().front().getHeader().cloneWithColumns(chunk_.detachColumns())); + if (res_chunk.hasChunkInfo()) + { + // std::cerr << "BalancingTransform: adding chunk " << std::endl; + + // { + // [[maybe_unused]]const auto * agg_info = typeid_cast(res_chunk.getChunkInfo().get()); + // std::cerr << "End of BalancingTransform: size of one group: " << agg_info->chunks.size() << std::endl; + // if (!agg_info->chunks.empty()) + // std::cerr << "!group is not empty, first column: " << agg_info->chunks[0].dumpStructure() << std::endl << std::endl; + // } + } + else + LOG_TRACE(getLogger("balancing"), "{}, BalancingTransform: not adding chunk, not finished.", reinterpret_cast(this));/// ISSUE: it's not clear why finished label is not set + std::swap(res_chunk, chunk_); + } + else + { + Chunk res_chunk = balance.add({}); + if (res_chunk.hasChunkInfo()) + { + // std::cerr << "BalancingTransform: finished adding, NumRows:" << res_chunk.getNumRows() << ", HasInfo: " << res_chunk.hasChunkInfo() << std::endl; + // { + // [[maybe_unused]]const auto * agg_info = typeid_cast(res_chunk.getChunkInfo().get()); + // std::cerr << "End of BalancingTransform: size of one group: " << agg_info->chunks.size() << std::endl; + // if (!agg_info->chunks.empty()) + // std::cerr << "!group is not empty, first column: " << agg_info->chunks[0].dumpStructure() << std::endl << std::endl; + // } + } + else + LOG_TRACE(getLogger("balancing"), "{}, BalancingTransform: not adding chunk on finished", reinterpret_cast(this)); + std::swap(res_chunk, chunk_); + } + LOG_TRACE(getLogger("balancing"), "{}, BalancingTransform: struct of output chunk: {}, hasInfo: {}", reinterpret_cast(this), chunk_.dumpStructure(), chunk.hasChunkInfo()); +} + +IProcessor::Status BalancingChunksTransform::prepareSend() +{ + LOG_TRACE(getLogger("balancingProcessor"), "prepareGenerate {}", chunk.dumpStructure()); + bool all_outputs_processed = true; + + size_t chunk_number = 0; + for (auto &output : outputs) + { + auto & was_processed = was_output_processed[chunk_number]; + ++chunk_number; + + if (!chunk.hasChunkInfo()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info must be not empty in prepareGenerate()"); + + if (was_processed) + continue; + + if (output.isFinished()) + continue; + + if (!output.canPush()) + { + all_outputs_processed = false; + continue; + } + + LOG_TRACE(getLogger("balancingProcessor"), "chunk struct: {}", chunk.dumpStructure()); + output.push(chunk.clone()); + was_processed = true; + } + + if (all_outputs_processed) + { + has_data = false; + return Status::Ready; + } + + return Status::PortFull; +} +} diff --git a/src/Processors/Transforms/BalancingTransform.h b/src/Processors/Transforms/BalancingTransform.h new file mode 100644 index 000000000000..d992a14cdd4f --- /dev/null +++ b/src/Processors/Transforms/BalancingTransform.h @@ -0,0 +1,128 @@ +#pragma once + +#include +#include +#include "Processors/Chunk.h" +#include "Processors/IProcessor.h" +#include "Processors/Transforms/ExceptionKeepingTransform.h" +#include + +namespace DB +{ + +class BalancingTransform : public ExceptionKeepingTransform +{ +public: + explicit BalancingTransform( + const Block & header, size_t max_memory_usage_); + + String getName() const override { return "BalancingTransform"; } + + void work() override; + + const Chunks & getChunks() const + { + return chunks; + } + +protected: + void onConsume(Chunk chunk) override; + GenerateResult onGenerate() override; + void onFinish() override; + +private: + size_t CalculateBlockSize(const Block & block); + Chunks chunks; + Blocks blocks; + size_t blocks_size; + Chunk cur_chunk; + Chunk finish_chunk; + size_t max_memory_usage; +}; + +/// Doesn't care about propagating exceptions and thus doesn't throw LOGICAL_ERROR if the following transform closes its input port. + + +/// Doesn't care about propagating exceptions and thus doesn't throw LOGICAL_ERROR if the following transform closes its input port. +class LBalancingChunksTransform : public ISimpleTransform +{ +public: + explicit LBalancingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage, bool skip_empty_chunks_); + + String getName() const override { return "LBalancingChunksTransform"; } + + const Chunks & getChunks() const + { + return chunks; + } + +protected: + void transform(Chunk &) override; + + IProcessor::Status prepare() override; + +private: + size_t CalculateBlockSize(const Block & block); + [[maybe_unused]] ChunksToSquash chunks_to_merge; + Chunks chunks; + Blocks blocks; + [[maybe_unused]] size_t blocks_size; + Chunk cur_chunk; + Chunk finish_chunk; + [[maybe_unused]] size_t max_memory_usage; + SquashingTransform squashing; + BalanceTransform balance; + [[maybe_unused]]size_t acc_size = 0; + + /// When consumption is finished we need to release the final chunk regardless of its size. + bool finished = false; +}; + +class BalancingChunksTransform : public IProcessor +{ +public: + BalancingChunksTransform( + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage_, size_t num_ports); + // explicit BalancingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage, bool skip_empty_chunks_); + + String getName() const override { return "BalancingChunksTransform"; } + + const Chunks & getChunks() const + { + return chunks; + } + + InputPorts & getInputPorts() { return inputs; } + OutputPorts & getOutputPorts() { return outputs; } + + Status prepare() override; + Status prepareConsume(); + Status prepareSend(); + + // void work() override; + void transform(Chunk & chunk); + +protected: + // void transform(Chunk &) ; + +private: + size_t CalculateBlockSize(const Block & block); + [[maybe_unused]] ChunksToSquash chunks_to_merge; + Chunks chunks; + Chunk chunk; + Blocks blocks; + [[maybe_unused]] size_t blocks_size; + Chunk cur_chunk; + Chunk finish_chunk; + [[maybe_unused]] size_t max_memory_usage; + SquashingTransform squashing; + BalanceTransform balance; + [[maybe_unused]]size_t acc_size = 0; + bool has_data = false; + std::vector was_output_processed; + + /// When consumption is finished we need to release the final chunk regardless of its size. + bool finished = false; +}; +} + diff --git a/src/Processors/Transforms/SquashingChunksTransform.cpp b/src/Processors/Transforms/SquashingChunksTransform.cpp index 7de9538e435c..22ce3ba93598 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.cpp +++ b/src/Processors/Transforms/SquashingChunksTransform.cpp @@ -1,4 +1,6 @@ #include +#include +#include "Common/logger_useful.h" namespace DB { @@ -12,7 +14,8 @@ SquashingChunksTransform::SquashingChunksTransform( void SquashingChunksTransform::onConsume(Chunk chunk) { - if (auto block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()))) + LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: !finished, hasInfo: {}", reinterpret_cast(this), chunk.hasChunkInfo()); + if (auto block = squashing.add(std::move(chunk))) { cur_chunk.setColumns(block.getColumns(), block.rows()); } @@ -29,7 +32,9 @@ SquashingChunksTransform::GenerateResult SquashingChunksTransform::onGenerate() void SquashingChunksTransform::onFinish() { auto block = squashing.add({}); + LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: finished, structure of block: {}", reinterpret_cast(this), block.dumpStructure()); finish_chunk.setColumns(block.getColumns(), block.rows()); + LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: finished, hasInfo: {}", reinterpret_cast(this), finish_chunk.hasChunkInfo()); } void SquashingChunksTransform::work() @@ -50,8 +55,8 @@ void SquashingChunksTransform::work() } SimpleSquashingChunksTransform::SimpleSquashingChunksTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) - : ISimpleTransform(header, header, true), squashing(min_block_size_rows, min_block_size_bytes) + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, [[maybe_unused]] bool skip_empty_chunks_) + : ISimpleTransform(header, header, false), squashing(min_block_size_rows, min_block_size_bytes) { } @@ -59,11 +64,13 @@ void SimpleSquashingChunksTransform::transform(Chunk & chunk) { if (!finished) { - if (auto block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()))) + LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: !finished, hasInfo: {}", reinterpret_cast(this), chunk.hasChunkInfo()); + if (auto block = squashing.add(std::move(chunk))) chunk.setColumns(block.getColumns(), block.rows()); } else { + LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: finished, hasInfo: {}", reinterpret_cast(this), chunk.hasChunkInfo()); auto block = squashing.add({}); chunk.setColumns(block.getColumns(), block.rows()); } @@ -79,4 +86,125 @@ IProcessor::Status SimpleSquashingChunksTransform::prepare() return ISimpleTransform::prepare(); } +//maybe it makes sense to pass not the IProcessor entity, but the SimpleTransform? anyway we have one input and one output +ProcessorSquashingTransform::ProcessorSquashingTransform( + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, [[maybe_unused]]size_t num_ports) + : IProcessor(InputPorts(1, header), OutputPorts(1, header)), squashing(min_block_size_rows, min_block_size_bytes) +{ +} + +IProcessor::Status ProcessorSquashingTransform::prepare() +{ + Status status = Status::Ready; + + while (status == Status::Ready) + { + status = !has_data ? prepareConsume() + : prepareGenerate(); + } + + return status; +} + +IProcessor::Status ProcessorSquashingTransform::prepareConsume() +{ + LOG_TRACE(getLogger("balancing"), "prepareConsume"); + for (auto & input : getInputPorts()) + { + bool all_finished = true; + for (auto & output : outputs) + { + if (output.isFinished()) + continue; + + all_finished = false; + } + + if (all_finished) + { + input.close(); + return Status::Finished; + } + + if (input.isFinished()) + { + for (auto & output : outputs) + output.finish(); + + return Status::Finished; + } + + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + + chunk = input.pull(); + has_data = true; + was_output_processed.assign(outputs.size(), false); + transform(chunk); + // if (chunk) + // chunks.push_back(std::move(chunk)); + } + return Status::Ready; +} + +void ProcessorSquashingTransform::transform(Chunk & chunk_) +{ + // [[maybe_unused]]const auto * agg_info = typeid_cast(chunk.getChunkInfo().get()); + // if (agg_info) + // { + // std::cerr << "Beginning of SquashingTransform: size of one group: " << agg_info->chunks.size() << std::endl; + // if (!agg_info->chunks.empty()) + // std::cerr << "!group is not empty, first column: " << agg_info->chunks[0].dumpStructure() << std::endl; + // } + LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: Struct of input chunk: {}", reinterpret_cast(this), chunk_.dumpStructure()); + if (!finished) + { + LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: !finished, hasInfo: {}", reinterpret_cast(this), chunk_.hasChunkInfo()); + if (auto block = squashing.add(std::move(chunk_))) + chunk_.setColumns(block.getColumns(), block.rows()); + } + else + { + LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: finished, hasInfo: {}", reinterpret_cast(this), chunk_.hasChunkInfo()); + auto block = squashing.add({}); + chunk_.setColumns(block.getColumns(), block.rows()); + } +} + +IProcessor::Status ProcessorSquashingTransform::prepareGenerate() +{ + LOG_TRACE(getLogger("squashingProcessor"), "prepareGenerate"); + bool all_outputs_processed = true; + + size_t chunk_number = 0; + for (auto &output : getOutputPorts()) + { + auto & was_processed = was_output_processed[chunk_number]; + ++chunk_number; + + if (was_processed) + continue; + + if (output.isFinished()) + continue; + + if (!output.canPush()) + { + all_outputs_processed = false; + continue; + } + + LOG_TRACE(getLogger("squashingProcessor"), "chunk struct: {}", chunk.dumpStructure()); + output.push(chunk.clone()); + was_processed = true; + } + + if (all_outputs_processed) + { + has_data = false; + return Status::Ready; + } + return Status::PortFull; +} } diff --git a/src/Processors/Transforms/SquashingChunksTransform.h b/src/Processors/Transforms/SquashingChunksTransform.h index f82e9e46a617..f140f5274d7d 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -23,7 +23,7 @@ class SquashingChunksTransform : public ExceptionKeepingTransform void onFinish() override; private: - SquashingTransform squashing; + NewSquashingTransform squashing; Chunk cur_chunk; Chunk finish_chunk; }; @@ -32,7 +32,7 @@ class SquashingChunksTransform : public ExceptionKeepingTransform class SimpleSquashingChunksTransform : public ISimpleTransform { public: - explicit SimpleSquashingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); + explicit SimpleSquashingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, bool skip_empty_chunks_ = true); String getName() const override { return "SimpleSquashingTransform"; } @@ -42,7 +42,35 @@ class SimpleSquashingChunksTransform : public ISimpleTransform IProcessor::Status prepare() override; private: - SquashingTransform squashing; + NewSquashingTransform squashing; + + bool finished = false; +}; + + +class ProcessorSquashingTransform : public IProcessor +{ +public: + explicit ProcessorSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports); + + String getName() const override { return "ProcessorSquashingTransform"; } + +protected: + InputPorts & getInputPorts() { return inputs; } + OutputPorts & getOutputPorts() { return outputs; } + + Status prepare() override; + Status prepareConsume(); + Status prepareGenerate(); + + // void work() override; + void transform(Chunk & chunk); + +private: + NewSquashingTransform squashing; + Chunk chunk; + bool has_data = false; + std::vector was_output_processed; /// When consumption is finished we need to release the final chunk regardless of its size. bool finished = false; From 852dd4c059f7163a28207be84c133212f683fa68 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 7 Mar 2024 13:42:01 +0000 Subject: [PATCH 004/254] refactor --- src/Interpreters/SquashingTransform.cpp | 27 +--- src/Interpreters/SquashingTransform.h | 3 +- .../Transforms/BalancingTransform.cpp | 93 +------------ .../Transforms/BalancingTransform.h | 90 +------------ .../Transforms/SquashingChunksTransform.cpp | 127 +----------------- .../Transforms/SquashingChunksTransform.h | 29 ---- 6 files changed, 9 insertions(+), 360 deletions(-) diff --git a/src/Interpreters/SquashingTransform.cpp b/src/Interpreters/SquashingTransform.cpp index 0d976bd967ad..a52b54653c1e 100644 --- a/src/Interpreters/SquashingTransform.cpp +++ b/src/Interpreters/SquashingTransform.cpp @@ -1,11 +1,4 @@ -#include -#include #include -#include "DataTypes/Serializations/ISerialization.h" -#include "Processors/Chunk.h" -#include "base/sleep.h" -#include "base/types.h" -#include #include @@ -135,6 +128,7 @@ bool SquashingTransform::isEnoughSize(size_t rows, size_t bytes) const } + NewSquashingTransform::NewSquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_) : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) @@ -170,7 +164,6 @@ Block NewSquashingTransform::addImpl(ReferenceType input_chunk) append(std::move(one), info->data_type); } - // if (isEnoughSize(accumulated_block)) { Block to_return; std::swap(to_return, accumulated_block); @@ -203,8 +196,6 @@ void NewSquashingTransform::append(ReferenceType input_chunk, DataTypePtr data_t } } - - BalanceTransform::BalanceTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) @@ -245,11 +236,9 @@ Chunk BalanceTransform::addImpl(ReferenceType input_block) Chunk input_chunk(input_block.getColumns(), input_block.rows()); if (!data_type && !input_block.getDataTypes().empty()) data_type = input_block.getDataTypes()[0]; - // /// End of input stream. if (!input_chunk) { Chunk res_chunk = convertToChunk(chunks_to_merge_vec); - // // std::cerr << "end of stream. Adding info to chunk " << std::endl; return res_chunk; } @@ -258,11 +247,9 @@ Chunk BalanceTransform::addImpl(ReferenceType input_block) if (input_chunk) chunks_to_merge_vec.push_back(input_chunk.clone()); - // std::cerr << "pushing back data. size: " << chunks_to_merge_vec.size() << std::endl; if (isEnoughSize(chunks_to_merge_vec)) { - // // // std::cerr << "enough size" << std::endl; Chunk res_chunk = convertToChunk(chunks_to_merge_vec); return res_chunk; } @@ -279,15 +266,8 @@ bool BalanceTransform::isEnoughSize(const std::vector & chunks) rows += chunk.getNumRows(); bytes += chunk.bytes(); } - auto free_memory = memory_tracker->getHardLimit() - memory_tracker->get(); - std::cerr << "========Just memory representation, free memory: " << free_memory << ", chunk size: " << bytes << std::endl - << " hardLimit: " << memory_tracker->getHardLimit() << " get(): " << memory_tracker->get() << std::endl; checkAndWaitMemoryAvailability(bytes); - free_memory = memory_tracker->getHardLimit() - memory_tracker->get(); - std::cerr << "========Just memory representation after, free memory: " << free_memory << ", chunk size: " << bytes << std::endl - << ", hardLimit: " << memory_tracker->getHardLimit() << ", get(): " << memory_tracker->get() << std::endl; - return isEnoughSize(rows, bytes); } @@ -298,12 +278,7 @@ void BalanceTransform::checkAndWaitMemoryAvailability(size_t bytes) { auto free_memory = hard_limit - memory_tracker->get(); while (Int64(bytes) >= free_memory) - { - // std::cerr << "========Waiting a while from memory, free memory: " << free_memory << ", chunk size: " << bytes << std::endl; - // sleepForMilliseconds(10); - // checkAndWaitMemoryAvailability(bytes); free_memory = hard_limit - memory_tracker->get(); - } } } diff --git a/src/Interpreters/SquashingTransform.h b/src/Interpreters/SquashingTransform.h index 0c2fe1ef12b4..fb6834e03bea 100644 --- a/src/Interpreters/SquashingTransform.h +++ b/src/Interpreters/SquashingTransform.h @@ -5,8 +5,7 @@ #include #include #include -#include "Common/MemoryTracker.h" -#include "DataTypes/Serializations/ISerialization.h" +#include namespace DB diff --git a/src/Processors/Transforms/BalancingTransform.cpp b/src/Processors/Transforms/BalancingTransform.cpp index b899702561e3..52d24fc9d013 100644 --- a/src/Processors/Transforms/BalancingTransform.cpp +++ b/src/Processors/Transforms/BalancingTransform.cpp @@ -1,9 +1,6 @@ -#include #include -#include "Common/Logger.h" -#include "Common/logger_useful.h" -#include "Interpreters/SquashingTransform.h" -#include "Processors/Chunk.h" +#include +#include namespace DB { @@ -13,67 +10,8 @@ namespace ErrorCodes extern const int MEMORY_LIMIT_EXCEEDED; } -LBalancingChunksTransform::LBalancingChunksTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage_, [[maybe_unused]] bool skip_empty_chunks_) - : ISimpleTransform(header, header, false), max_memory_usage(max_memory_usage_), squashing(min_block_size_rows, min_block_size_bytes), balance(header, min_block_size_rows, min_block_size_bytes) -{ -} - -void LBalancingChunksTransform::transform(Chunk & chunk) -{ - if (!finished) - { - Chunk res_chunk = balance.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); - if (res_chunk.hasChunkInfo()) - { - // std::cerr << "BalancingTransform: adding chunk " << std::endl; - - // { - // [[maybe_unused]]const auto * agg_info = typeid_cast(res_chunk.getChunkInfo().get()); - // std::cerr << "End of BalancingTransform: size of one group: " << agg_info->chunks.size() << std::endl; - // if (!agg_info->chunks.empty()) - // std::cerr << "!group is not empty, first column: " << agg_info->chunks[0].dumpStructure() << std::endl << std::endl; - // } - - } - else - LOG_TRACE(getLogger("balancing"), "{}, BalancingTransform: not adding chunk, not finished.", reinterpret_cast(this));/// ISSUE: it's not clear why finished label is not set - std::swap(res_chunk, chunk); - } - else - { - Chunk res_chunk = balance.add({}); - if (res_chunk.hasChunkInfo()) - { - // std::cerr << "BalancingTransform: finished adding, NumRows:" << res_chunk.getNumRows() << ", HasInfo: " << res_chunk.hasChunkInfo() << std::endl; - // { - // [[maybe_unused]]const auto * agg_info = typeid_cast(res_chunk.getChunkInfo().get()); - // std::cerr << "End of BalancingTransform: size of one group: " << agg_info->chunks.size() << std::endl; - // if (!agg_info->chunks.empty()) - // std::cerr << "!group is not empty, first column: " << agg_info->chunks[0].dumpStructure() << std::endl << std::endl; - // } - - } - else - LOG_TRACE(getLogger("balancing"), "{}, BalancingTransform: not adding chunk on finished", reinterpret_cast(this)); - std::swap(res_chunk, chunk); - } - LOG_TRACE(getLogger("balancing"), "{}, BalancingTransform: struct of output chunk: {}", reinterpret_cast(this), chunk.dumpStructure()); -} - -IProcessor::Status LBalancingChunksTransform::prepare() -{ - if (!finished && input.isFinished()) - { - finished = true; - return Status::Ready; - } - return ISimpleTransform::prepare(); -} - - BalancingChunksTransform::BalancingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage_, size_t num_ports) - : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), max_memory_usage(max_memory_usage_), squashing(min_block_size_rows, min_block_size_bytes), balance(header, min_block_size_rows, min_block_size_bytes) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), max_memory_usage(max_memory_usage_), balance(header, min_block_size_rows, min_block_size_bytes) { } @@ -146,36 +84,11 @@ void BalancingChunksTransform::transform(Chunk & chunk_) if (!finished) { Chunk res_chunk = balance.add(getInputPorts().front().getHeader().cloneWithColumns(chunk_.detachColumns())); - if (res_chunk.hasChunkInfo()) - { - // std::cerr << "BalancingTransform: adding chunk " << std::endl; - - // { - // [[maybe_unused]]const auto * agg_info = typeid_cast(res_chunk.getChunkInfo().get()); - // std::cerr << "End of BalancingTransform: size of one group: " << agg_info->chunks.size() << std::endl; - // if (!agg_info->chunks.empty()) - // std::cerr << "!group is not empty, first column: " << agg_info->chunks[0].dumpStructure() << std::endl << std::endl; - // } - } - else - LOG_TRACE(getLogger("balancing"), "{}, BalancingTransform: not adding chunk, not finished.", reinterpret_cast(this));/// ISSUE: it's not clear why finished label is not set std::swap(res_chunk, chunk_); } else { Chunk res_chunk = balance.add({}); - if (res_chunk.hasChunkInfo()) - { - // std::cerr << "BalancingTransform: finished adding, NumRows:" << res_chunk.getNumRows() << ", HasInfo: " << res_chunk.hasChunkInfo() << std::endl; - // { - // [[maybe_unused]]const auto * agg_info = typeid_cast(res_chunk.getChunkInfo().get()); - // std::cerr << "End of BalancingTransform: size of one group: " << agg_info->chunks.size() << std::endl; - // if (!agg_info->chunks.empty()) - // std::cerr << "!group is not empty, first column: " << agg_info->chunks[0].dumpStructure() << std::endl << std::endl; - // } - } - else - LOG_TRACE(getLogger("balancing"), "{}, BalancingTransform: not adding chunk on finished", reinterpret_cast(this)); std::swap(res_chunk, chunk_); } LOG_TRACE(getLogger("balancing"), "{}, BalancingTransform: struct of output chunk: {}, hasInfo: {}", reinterpret_cast(this), chunk_.dumpStructure(), chunk.hasChunkInfo()); diff --git a/src/Processors/Transforms/BalancingTransform.h b/src/Processors/Transforms/BalancingTransform.h index d992a14cdd4f..8812a0b8c172 100644 --- a/src/Processors/Transforms/BalancingTransform.h +++ b/src/Processors/Transforms/BalancingTransform.h @@ -1,96 +1,22 @@ #pragma once -#include #include -#include "Processors/Chunk.h" -#include "Processors/IProcessor.h" -#include "Processors/Transforms/ExceptionKeepingTransform.h" +#include #include namespace DB { -class BalancingTransform : public ExceptionKeepingTransform -{ -public: - explicit BalancingTransform( - const Block & header, size_t max_memory_usage_); - - String getName() const override { return "BalancingTransform"; } - - void work() override; - - const Chunks & getChunks() const - { - return chunks; - } - -protected: - void onConsume(Chunk chunk) override; - GenerateResult onGenerate() override; - void onFinish() override; - -private: - size_t CalculateBlockSize(const Block & block); - Chunks chunks; - Blocks blocks; - size_t blocks_size; - Chunk cur_chunk; - Chunk finish_chunk; - size_t max_memory_usage; -}; - -/// Doesn't care about propagating exceptions and thus doesn't throw LOGICAL_ERROR if the following transform closes its input port. - - -/// Doesn't care about propagating exceptions and thus doesn't throw LOGICAL_ERROR if the following transform closes its input port. -class LBalancingChunksTransform : public ISimpleTransform -{ -public: - explicit LBalancingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage, bool skip_empty_chunks_); - - String getName() const override { return "LBalancingChunksTransform"; } - - const Chunks & getChunks() const - { - return chunks; - } - -protected: - void transform(Chunk &) override; - - IProcessor::Status prepare() override; - -private: - size_t CalculateBlockSize(const Block & block); - [[maybe_unused]] ChunksToSquash chunks_to_merge; - Chunks chunks; - Blocks blocks; - [[maybe_unused]] size_t blocks_size; - Chunk cur_chunk; - Chunk finish_chunk; - [[maybe_unused]] size_t max_memory_usage; - SquashingTransform squashing; - BalanceTransform balance; - [[maybe_unused]]size_t acc_size = 0; - - /// When consumption is finished we need to release the final chunk regardless of its size. - bool finished = false; -}; class BalancingChunksTransform : public IProcessor { public: BalancingChunksTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage_, size_t num_ports); - // explicit BalancingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage, bool skip_empty_chunks_); String getName() const override { return "BalancingChunksTransform"; } - const Chunks & getChunks() const - { - return chunks; - } + InputPorts & getInputPorts() { return inputs; } OutputPorts & getOutputPorts() { return outputs; } @@ -99,25 +25,15 @@ class BalancingChunksTransform : public IProcessor Status prepareConsume(); Status prepareSend(); - // void work() override; void transform(Chunk & chunk); protected: - // void transform(Chunk &) ; private: size_t CalculateBlockSize(const Block & block); - [[maybe_unused]] ChunksToSquash chunks_to_merge; - Chunks chunks; Chunk chunk; - Blocks blocks; - [[maybe_unused]] size_t blocks_size; - Chunk cur_chunk; - Chunk finish_chunk; - [[maybe_unused]] size_t max_memory_usage; - SquashingTransform squashing; + size_t max_memory_usage; BalanceTransform balance; - [[maybe_unused]]size_t acc_size = 0; bool has_data = false; std::vector was_output_processed; diff --git a/src/Processors/Transforms/SquashingChunksTransform.cpp b/src/Processors/Transforms/SquashingChunksTransform.cpp index 22ce3ba93598..5b68df6b6c60 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.cpp +++ b/src/Processors/Transforms/SquashingChunksTransform.cpp @@ -1,6 +1,5 @@ #include -#include -#include "Common/logger_useful.h" +#include namespace DB { @@ -16,9 +15,7 @@ void SquashingChunksTransform::onConsume(Chunk chunk) { LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: !finished, hasInfo: {}", reinterpret_cast(this), chunk.hasChunkInfo()); if (auto block = squashing.add(std::move(chunk))) - { cur_chunk.setColumns(block.getColumns(), block.rows()); - } } SquashingChunksTransform::GenerateResult SquashingChunksTransform::onGenerate() @@ -85,126 +82,4 @@ IProcessor::Status SimpleSquashingChunksTransform::prepare() } return ISimpleTransform::prepare(); } - -//maybe it makes sense to pass not the IProcessor entity, but the SimpleTransform? anyway we have one input and one output -ProcessorSquashingTransform::ProcessorSquashingTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, [[maybe_unused]]size_t num_ports) - : IProcessor(InputPorts(1, header), OutputPorts(1, header)), squashing(min_block_size_rows, min_block_size_bytes) -{ -} - -IProcessor::Status ProcessorSquashingTransform::prepare() -{ - Status status = Status::Ready; - - while (status == Status::Ready) - { - status = !has_data ? prepareConsume() - : prepareGenerate(); - } - - return status; -} - -IProcessor::Status ProcessorSquashingTransform::prepareConsume() -{ - LOG_TRACE(getLogger("balancing"), "prepareConsume"); - for (auto & input : getInputPorts()) - { - bool all_finished = true; - for (auto & output : outputs) - { - if (output.isFinished()) - continue; - - all_finished = false; - } - - if (all_finished) - { - input.close(); - return Status::Finished; - } - - if (input.isFinished()) - { - for (auto & output : outputs) - output.finish(); - - return Status::Finished; - } - - input.setNeeded(); - if (!input.hasData()) - return Status::NeedData; - - chunk = input.pull(); - has_data = true; - was_output_processed.assign(outputs.size(), false); - transform(chunk); - // if (chunk) - // chunks.push_back(std::move(chunk)); - } - return Status::Ready; -} - -void ProcessorSquashingTransform::transform(Chunk & chunk_) -{ - // [[maybe_unused]]const auto * agg_info = typeid_cast(chunk.getChunkInfo().get()); - // if (agg_info) - // { - // std::cerr << "Beginning of SquashingTransform: size of one group: " << agg_info->chunks.size() << std::endl; - // if (!agg_info->chunks.empty()) - // std::cerr << "!group is not empty, first column: " << agg_info->chunks[0].dumpStructure() << std::endl; - // } - LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: Struct of input chunk: {}", reinterpret_cast(this), chunk_.dumpStructure()); - if (!finished) - { - LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: !finished, hasInfo: {}", reinterpret_cast(this), chunk_.hasChunkInfo()); - if (auto block = squashing.add(std::move(chunk_))) - chunk_.setColumns(block.getColumns(), block.rows()); - } - else - { - LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: finished, hasInfo: {}", reinterpret_cast(this), chunk_.hasChunkInfo()); - auto block = squashing.add({}); - chunk_.setColumns(block.getColumns(), block.rows()); - } -} - -IProcessor::Status ProcessorSquashingTransform::prepareGenerate() -{ - LOG_TRACE(getLogger("squashingProcessor"), "prepareGenerate"); - bool all_outputs_processed = true; - - size_t chunk_number = 0; - for (auto &output : getOutputPorts()) - { - auto & was_processed = was_output_processed[chunk_number]; - ++chunk_number; - - if (was_processed) - continue; - - if (output.isFinished()) - continue; - - if (!output.canPush()) - { - all_outputs_processed = false; - continue; - } - - LOG_TRACE(getLogger("squashingProcessor"), "chunk struct: {}", chunk.dumpStructure()); - output.push(chunk.clone()); - was_processed = true; - } - - if (all_outputs_processed) - { - has_data = false; - return Status::Ready; - } - return Status::PortFull; -} } diff --git a/src/Processors/Transforms/SquashingChunksTransform.h b/src/Processors/Transforms/SquashingChunksTransform.h index f140f5274d7d..5c7ad12889f1 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -46,33 +46,4 @@ class SimpleSquashingChunksTransform : public ISimpleTransform bool finished = false; }; - - -class ProcessorSquashingTransform : public IProcessor -{ -public: - explicit ProcessorSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports); - - String getName() const override { return "ProcessorSquashingTransform"; } - -protected: - InputPorts & getInputPorts() { return inputs; } - OutputPorts & getOutputPorts() { return outputs; } - - Status prepare() override; - Status prepareConsume(); - Status prepareGenerate(); - - // void work() override; - void transform(Chunk & chunk); - -private: - NewSquashingTransform squashing; - Chunk chunk; - bool has_data = false; - std::vector was_output_processed; - - /// When consumption is finished we need to release the final chunk regardless of its size. - bool finished = false; -}; } From 47efd981f04e21d3a8cad6ff86c91f0a4531d90e Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 7 Mar 2024 15:29:07 +0000 Subject: [PATCH 005/254] style fix --- src/Interpreters/SquashingTransform.cpp | 6 +----- src/Processors/Transforms/BalancingTransform.cpp | 2 +- src/Processors/Transforms/BalancingTransform.h | 2 -- 3 files changed, 2 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/SquashingTransform.cpp b/src/Interpreters/SquashingTransform.cpp index a52b54653c1e..4a259d0b7c90 100644 --- a/src/Interpreters/SquashingTransform.cpp +++ b/src/Interpreters/SquashingTransform.cpp @@ -128,7 +128,6 @@ bool SquashingTransform::isEnoughSize(size_t rows, size_t bytes) const } - NewSquashingTransform::NewSquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_) : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) @@ -160,10 +159,8 @@ Block NewSquashingTransform::addImpl(ReferenceType input_chunk) const auto *info = getInfoFromChunk(input_chunk); for (auto & one : info->chunks) - { append(std::move(one), info->data_type); - } - + { Block to_return; std::swap(to_return, accumulated_block); @@ -294,5 +291,4 @@ bool BalanceTransform::isEnoughSize(size_t rows, size_t bytes) const || (min_block_size_rows && rows >= min_block_size_rows) || (min_block_size_bytes && bytes >= min_block_size_bytes); } - } diff --git a/src/Processors/Transforms/BalancingTransform.cpp b/src/Processors/Transforms/BalancingTransform.cpp index 52d24fc9d013..58c2f052c1a7 100644 --- a/src/Processors/Transforms/BalancingTransform.cpp +++ b/src/Processors/Transforms/BalancingTransform.cpp @@ -7,7 +7,7 @@ namespace DB namespace ErrorCodes { - extern const int MEMORY_LIMIT_EXCEEDED; + extern const int LOGICAL_ERROR; } BalancingChunksTransform::BalancingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage_, size_t num_ports) diff --git a/src/Processors/Transforms/BalancingTransform.h b/src/Processors/Transforms/BalancingTransform.h index 8812a0b8c172..1b1d3ec6295b 100644 --- a/src/Processors/Transforms/BalancingTransform.h +++ b/src/Processors/Transforms/BalancingTransform.h @@ -16,8 +16,6 @@ class BalancingChunksTransform : public IProcessor String getName() const override { return "BalancingChunksTransform"; } - - InputPorts & getInputPorts() { return inputs; } OutputPorts & getOutputPorts() { return outputs; } From 1523864e027430dde21dd74dde019088472cf385 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Mar 2024 17:46:01 +0100 Subject: [PATCH 006/254] Detect Lexer errors earlier --- src/Parsers/TokenIterator.cpp | 9 ++++++++- src/Parsers/TokenIterator.h | 3 +++ src/Parsers/parseQuery.cpp | 6 ++++++ 3 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Parsers/TokenIterator.cpp b/src/Parsers/TokenIterator.cpp index fa792e7c8b5b..575a78d2f6c1 100644 --- a/src/Parsers/TokenIterator.cpp +++ b/src/Parsers/TokenIterator.cpp @@ -12,12 +12,19 @@ Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool do { Token token = lexer.nextToken(); - stop = token.isEnd() || token.type == TokenType::ErrorMaxQuerySizeExceeded; + stop = token.isEnd() || token.isError(); if (token.isSignificant() || (!skip_insignificant && !data.empty() && data.back().isSignificant())) data.emplace_back(std::move(token)); } while (!stop); } +std::optional Tokens::getError() const +{ + if (!data.empty() && data.back().isError()) + return data.back(); + return {}; +} + UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin) { /// We have just two kind of parentheses: () and []. diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index 8cb59aa12e2f..3b4521182412 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -5,6 +5,7 @@ #include #include +#include namespace DB @@ -34,6 +35,8 @@ class Tokens } ALWAYS_INLINE inline const Token & max() { return data[last_accessed_index]; } + + std::optional getError() const; }; diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 8f9977c0b8d8..ec39860b1330 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -257,6 +257,12 @@ ASTPtr tryParseQuery( return nullptr; } + if (auto error = tokens.getError(); error) + { + out_error_message = getLexicalErrorMessage(query_begin, all_queries_end, *error, hilite, query_description); + return nullptr; + } + Expected expected; ASTPtr res; const bool parse_res = parser.parse(token_iterator, res, expected); From 4d01114fed9b083cbd4bfb66f316ae9e4ff4caf3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Mar 2024 17:46:01 +0100 Subject: [PATCH 007/254] Detect Lexer errors earlier --- src/Parsers/TokenIterator.cpp | 9 ++++++++- src/Parsers/TokenIterator.h | 3 +++ src/Parsers/parseQuery.cpp | 6 ++++++ 3 files changed, 17 insertions(+), 1 deletion(-) diff --git a/src/Parsers/TokenIterator.cpp b/src/Parsers/TokenIterator.cpp index fa792e7c8b5b..575a78d2f6c1 100644 --- a/src/Parsers/TokenIterator.cpp +++ b/src/Parsers/TokenIterator.cpp @@ -12,12 +12,19 @@ Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool do { Token token = lexer.nextToken(); - stop = token.isEnd() || token.type == TokenType::ErrorMaxQuerySizeExceeded; + stop = token.isEnd() || token.isError(); if (token.isSignificant() || (!skip_insignificant && !data.empty() && data.back().isSignificant())) data.emplace_back(std::move(token)); } while (!stop); } +std::optional Tokens::getError() const +{ + if (!data.empty() && data.back().isError()) + return data.back(); + return {}; +} + UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin) { /// We have just two kind of parentheses: () and []. diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index 192f2f55e6a8..9c78bc3c57b2 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -5,6 +5,7 @@ #include #include +#include namespace DB @@ -34,6 +35,8 @@ class Tokens } ALWAYS_INLINE inline const Token & max() { return data[last_accessed_index]; } + + std::optional getError() const; }; diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 7aad0b010a5f..54c05ebc8169 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -258,6 +258,12 @@ ASTPtr tryParseQuery( return nullptr; } + if (auto error = tokens.getError(); error) + { + out_error_message = getLexicalErrorMessage(query_begin, all_queries_end, *error, hilite, query_description); + return nullptr; + } + Expected expected; ASTPtr res; const bool parse_res = parser.parse(token_iterator, res, expected); From 31e7e78cd0f722ccc58d39402a2422df8a62f05c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 21 Mar 2024 02:16:13 +0100 Subject: [PATCH 008/254] Better code --- src/Parsers/TokenIterator.cpp | 7 ------- src/Parsers/TokenIterator.h | 2 -- src/Parsers/parseQuery.cpp | 24 +++++++++++++++++++++--- 3 files changed, 21 insertions(+), 12 deletions(-) diff --git a/src/Parsers/TokenIterator.cpp b/src/Parsers/TokenIterator.cpp index 575a78d2f6c1..6ef01c163d34 100644 --- a/src/Parsers/TokenIterator.cpp +++ b/src/Parsers/TokenIterator.cpp @@ -18,13 +18,6 @@ Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool } while (!stop); } -std::optional Tokens::getError() const -{ - if (!data.empty() && data.back().isError()) - return data.back(); - return {}; -} - UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin) { /// We have just two kind of parentheses: () and []. diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index 9c78bc3c57b2..0dbea122cf2b 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -35,8 +35,6 @@ class Tokens } ALWAYS_INLINE inline const Token & max() { return data[last_accessed_index]; } - - std::optional getError() const; }; diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 54c05ebc8169..990870da4d74 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -258,10 +259,27 @@ ASTPtr tryParseQuery( return nullptr; } - if (auto error = tokens.getError(); error) + /** A shortcut - if Lexer found invalid tokens, fail early without full parsing. + * But there are certain cases when invalid tokens are permitted: + * 1. INSERT queries can have arbitrary data after the FORMAT clause, that is parsed by a different parser. + * 2. It can also be the case when there are multiple queries separated by semicolons, and the first queries are ok + * while subsequent queries have syntax errors. + * + * This shortcut is needed to avoid complex backtracking in case of obviously erroneous queries. + */ + IParser::Pos lookahead = token_iterator; + if (!ParserKeyword(Keyword::INSERT_INTO).ignore(lookahead)) { - out_error_message = getLexicalErrorMessage(query_begin, all_queries_end, *error, hilite, query_description); - return nullptr; + while (lookahead->type != TokenType::Semicolon && lookahead->type != TokenType::EndOfStream) + { + if (lookahead->isError()) + { + out_error_message = getLexicalErrorMessage(query_begin, all_queries_end, *lookahead, hilite, query_description); + return nullptr; + } + + ++lookahead; + } } Expected expected; From 139b0b65d7ac5afbb0e1687ab956b536730db11c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 21 Mar 2024 02:30:20 +0100 Subject: [PATCH 009/254] Add a test --- .../03015_parser_shortcut_lexer_errors.reference | 1 + .../0_stateless/03015_parser_shortcut_lexer_errors.sh | 7 +++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.reference create mode 100755 tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.sh diff --git a/tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.reference b/tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.reference new file mode 100644 index 000000000000..f83d884fd780 --- /dev/null +++ b/tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.reference @@ -0,0 +1 @@ +Syntax error diff --git a/tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.sh b/tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.sh new file mode 100755 index 000000000000..762201ed5fce --- /dev/null +++ b/tests/queries/0_stateless/03015_parser_shortcut_lexer_errors.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL --query "SELECT((((((((((SELECT(((((((((SELECT((((((((((SELECT(((((((((SELECT((((((((((SELECT(((((((((SELECT 1+)))))))))))))))))))))))))))))))))))))))))))))))))))))))))'" 2>&1 | grep -o -F 'Syntax error' From e71eeca356efde45dda499df9e1e08ea8941b4ed Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 22 Mar 2024 02:18:26 +0100 Subject: [PATCH 010/254] Fix error --- src/Parsers/TokenIterator.h | 3 ++- src/Parsers/parseQuery.cpp | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index 0dbea122cf2b..cf370a523ac2 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -5,7 +5,6 @@ #include #include -#include namespace DB @@ -35,6 +34,8 @@ class Tokens } ALWAYS_INLINE inline const Token & max() { return data[last_accessed_index]; } + + void reset() { last_accessed_index = 0; } }; diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 787349ce6f5e..2b67bbeb92af 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -283,6 +283,7 @@ ASTPtr tryParseQuery( ++lookahead; } } + tokens.reset(); Expected expected; ASTPtr res; From cfd2cd234f653ba11219d6810e3a7c87f469a065 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 19:17:50 +0100 Subject: [PATCH 011/254] Remove garbage from KQL, part 1 --- src/Parsers/ExpressionListParsers.cpp | 2 +- src/Parsers/Kusto/ParserKQLOperators.cpp | 90 ++++++++++++------------ src/Parsers/Kusto/ParserKQLStatement.cpp | 1 + 3 files changed, 47 insertions(+), 46 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 05691529f430..9e176bfd1073 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2163,7 +2163,7 @@ class KustoLayer : public Layer bool parse(IParser::Pos & pos, Expected & expected, Action & /*action*/) override { - /// kql(table|project ...) + /// kql('table|project ...') /// 0. Parse the kql query /// 1. Parse closing token if (state == 0) diff --git a/src/Parsers/Kusto/ParserKQLOperators.cpp b/src/Parsers/Kusto/ParserKQLOperators.cpp index d7364cb5fd73..c31c8711008c 100644 --- a/src/Parsers/Kusto/ParserKQLOperators.cpp +++ b/src/Parsers/Kusto/ParserKQLOperators.cpp @@ -1,20 +1,26 @@ #include #include #include -#include #include -#include #include #include #include #include #include -#include "KustoFunctions/IParserKQLFunction.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} namespace { -enum class KQLOperatorValue : uint16_t +enum class KQLOperatorValue { none, between, @@ -56,7 +62,8 @@ enum class KQLOperatorValue : uint16_t not_startswith_cs, }; -const std::unordered_map KQLOperator = { +const std::unordered_map KQLOperator = +{ {"between", KQLOperatorValue::between}, {"!between", KQLOperatorValue::not_between}, {"contains", KQLOperatorValue::contains}, @@ -96,44 +103,37 @@ const std::unordered_map KQLOperator = { {"!startswith_cs", KQLOperatorValue::not_startswith_cs}, }; -void rebuildSubqueryForInOperator(DB::ASTPtr & node, bool useLowerCase) +void rebuildSubqueryForInOperator(ASTPtr & node, bool useLowerCase) { //A sub-query for in operator in kql can have multiple columns, but only takes the first column. //A sub-query for in operator in ClickHouse can not have multiple columns //So only take the first column if there are multiple columns. //select * not working for subquery. (a tabular statement without project) - const auto selectColumns = node->children[0]->children[0]->as()->select(); + const auto selectColumns = node->children[0]->children[0]->as()->select(); while (selectColumns->children.size() > 1) selectColumns->children.pop_back(); if (useLowerCase) { - auto args = std::make_shared(); + auto args = std::make_shared(); args->children.push_back(selectColumns->children[0]); - auto func_lower = std::make_shared(); + auto func_lower = std::make_shared(); func_lower->name = "lower"; func_lower->children.push_back(selectColumns->children[0]); func_lower->arguments = args; - if (selectColumns->children[0]->as()) - func_lower->alias = std::move(selectColumns->children[0]->as()->alias); - else if (selectColumns->children[0]->as()) - func_lower->alias = std::move(selectColumns->children[0]->as()->alias); + if (selectColumns->children[0]->as()) + func_lower->alias = std::move(selectColumns->children[0]->as()->alias); + else if (selectColumns->children[0]->as()) + func_lower->alias = std::move(selectColumns->children[0]->as()->alias); - auto funcs = std::make_shared(); + auto funcs = std::make_shared(); funcs->children.push_back(func_lower); selectColumns->children[0] = std::move(funcs); } } } -namespace DB -{ - -namespace ErrorCodes -{ - extern const int SYNTAX_ERROR; -} String KQLOperators::genHasAnyAllOpExpr(std::vector & tokens, IParser::Pos & token_pos, String kql_op, String ch_op) { @@ -166,7 +166,7 @@ String KQLOperators::genHasAnyAllOpExpr(std::vector & tokens, IParser::P return new_expr; } -String genEqOpExprCis(std::vector & tokens, DB::IParser::Pos & token_pos, const String & ch_op) +String genEqOpExprCis(std::vector & tokens, IParser::Pos & token_pos, const String & ch_op) { String tmp_arg(token_pos->begin, token_pos->end); @@ -178,30 +178,30 @@ String genEqOpExprCis(std::vector & tokens, DB::IParser::Pos & token_pos new_expr += ch_op + " "; ++token_pos; - if (token_pos->type == DB::TokenType::StringLiteral || token_pos->type == DB::TokenType::QuotedIdentifier) - new_expr += "lower('" + DB::IParserKQLFunction::escapeSingleQuotes(String(token_pos->begin + 1, token_pos->end - 1)) + "')"; + if (token_pos->type == TokenType::StringLiteral || token_pos->type == TokenType::QuotedIdentifier) + new_expr += "lower('" + IParserKQLFunction::escapeSingleQuotes(String(token_pos->begin + 1, token_pos->end - 1)) + "')"; else - new_expr += "lower(" + DB::IParserKQLFunction::getExpression(token_pos) + ")"; + new_expr += "lower(" + IParserKQLFunction::getExpression(token_pos) + ")"; tokens.pop_back(); return new_expr; } -String genInOpExprCis(std::vector & tokens, DB::IParser::Pos & token_pos, const String & kql_op, const String & ch_op) +String genInOpExprCis(std::vector & tokens, IParser::Pos & token_pos, const String & kql_op, const String & ch_op) { - DB::ParserKQLTableFunction kqlfun_p; - DB::ParserToken s_lparen(DB::TokenType::OpeningRoundBracket); + ParserKQLTableFunction kqlfun_p; + ParserToken s_lparen(TokenType::OpeningRoundBracket); - DB::ASTPtr select; - DB::Expected expected; + ASTPtr select; + Expected expected; String new_expr; ++token_pos; if (!s_lparen.ignore(token_pos, expected)) - throw DB::Exception(DB::ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op); + throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op); if (tokens.empty()) - throw DB::Exception(DB::ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op); + throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op); new_expr = "lower(" + tokens.back() + ") "; tokens.pop_back(); @@ -218,39 +218,39 @@ String genInOpExprCis(std::vector & tokens, DB::IParser::Pos & token_pos --token_pos; new_expr += ch_op; - while (isValidKQLPos(token_pos) && token_pos->type != DB::TokenType::PipeMark && token_pos->type != DB::TokenType::Semicolon) + while (isValidKQLPos(token_pos) && token_pos->type != TokenType::PipeMark && token_pos->type != TokenType::Semicolon) { auto tmp_arg = String(token_pos->begin, token_pos->end); - if (token_pos->type != DB::TokenType::Comma && token_pos->type != DB::TokenType::ClosingRoundBracket - && token_pos->type != DB::TokenType::OpeningRoundBracket && token_pos->type != DB::TokenType::OpeningSquareBracket - && token_pos->type != DB::TokenType::ClosingSquareBracket && tmp_arg != "~" && tmp_arg != "dynamic") + if (token_pos->type != TokenType::Comma && token_pos->type != TokenType::ClosingRoundBracket + && token_pos->type != TokenType::OpeningRoundBracket && token_pos->type != TokenType::OpeningSquareBracket + && token_pos->type != TokenType::ClosingSquareBracket && tmp_arg != "~" && tmp_arg != "dynamic") { - if (token_pos->type == DB::TokenType::StringLiteral || token_pos->type == DB::TokenType::QuotedIdentifier) - new_expr += "lower('" + DB::IParserKQLFunction::escapeSingleQuotes(String(token_pos->begin + 1, token_pos->end - 1)) + "')"; + if (token_pos->type == TokenType::StringLiteral || token_pos->type == TokenType::QuotedIdentifier) + new_expr += "lower('" + IParserKQLFunction::escapeSingleQuotes(String(token_pos->begin + 1, token_pos->end - 1)) + "')"; else new_expr += "lower(" + tmp_arg + ")"; } else if (tmp_arg != "~" && tmp_arg != "dynamic" && tmp_arg != "[" && tmp_arg != "]") new_expr += tmp_arg; - if (token_pos->type == DB::TokenType::ClosingRoundBracket) + if (token_pos->type == TokenType::ClosingRoundBracket) break; ++token_pos; } return new_expr; } -std::string genInOpExpr(DB::IParser::Pos & token_pos, const std::string & kql_op, const std::string & ch_op) +std::string genInOpExpr(IParser::Pos & token_pos, const std::string & kql_op, const std::string & ch_op) { - DB::ParserKQLTableFunction kqlfun_p; - DB::ParserToken s_lparen(DB::TokenType::OpeningRoundBracket); + ParserKQLTableFunction kqlfun_p; + ParserToken s_lparen(TokenType::OpeningRoundBracket); - DB::ASTPtr select; - DB::Expected expected; + ASTPtr select; + Expected expected; ++token_pos; if (!s_lparen.ignore(token_pos, expected)) - throw DB::Exception(DB::ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op); + throw Exception(ErrorCodes::SYNTAX_ERROR, "Syntax error near {}", kql_op); auto pos = token_pos; if (kqlfun_p.parse(pos, select, expected)) diff --git a/src/Parsers/Kusto/ParserKQLStatement.cpp b/src/Parsers/Kusto/ParserKQLStatement.cpp index fbf2110e664b..3ca873bd986f 100644 --- a/src/Parsers/Kusto/ParserKQLStatement.cpp +++ b/src/Parsers/Kusto/ParserKQLStatement.cpp @@ -107,4 +107,5 @@ bool ParserKQLTableFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expe pos = begin; return false; } + } From 35d0220e19e61332bd29c8c1bf24ca301cc2287f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 19:26:12 +0100 Subject: [PATCH 012/254] Remove garbage from KQL, part 2 --- src/Parsers/Kusto/ParserKQLStatement.cpp | 59 +++++++----------------- 1 file changed, 17 insertions(+), 42 deletions(-) diff --git a/src/Parsers/Kusto/ParserKQLStatement.cpp b/src/Parsers/Kusto/ParserKQLStatement.cpp index 3ca873bd986f..7668779356e8 100644 --- a/src/Parsers/Kusto/ParserKQLStatement.cpp +++ b/src/Parsers/Kusto/ParserKQLStatement.cpp @@ -2,11 +2,11 @@ #include #include #include -#include #include #include -#include #include +#include + namespace DB { @@ -62,50 +62,25 @@ bool ParserKQLWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & exp bool ParserKQLTableFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - ParserKQLWithUnionQuery kql_p; - ASTPtr select; - ParserToken s_lparen(TokenType::OpeningRoundBracket); + ParserToken lparen(TokenType::OpeningRoundBracket); + ParserToken rparen(TokenType::OpeningRoundBracket); - auto begin = pos; - auto paren_count = 0; - String kql_statement; + ASTPtr string_literal; + ParserStringLiteral parser_string_literal; - if (s_lparen.ignore(pos, expected)) + if (!(lparen.ignore(pos, expected) + && parser_string_literal.parse(pos, string_literal, expected) + && rparen.ignore(pos, expected))) { - if (pos->type == TokenType::HereDoc) - { - kql_statement = String(pos->begin + 2, pos->end - 2); - } - else - { - ++paren_count; - auto pos_start = pos; - while (isValidKQLPos(pos)) - { - if (pos->type == TokenType::ClosingRoundBracket) - --paren_count; - if (pos->type == TokenType::OpeningRoundBracket) - ++paren_count; - - if (paren_count == 0) - break; - ++pos; - } - kql_statement = String(pos_start->begin, (--pos)->end); - } - ++pos; - Tokens token_kql(kql_statement.c_str(), kql_statement.c_str() + kql_statement.size()); - IParser::Pos pos_kql(token_kql, pos.max_depth, pos.max_backtracks); - - if (kql_p.parse(pos_kql, select, expected)) - { - node = select; - ++pos; - return true; - } + return false; } - pos = begin; - return false; + + String kql_statement = typeid_cast(*string_literal).value.safeGet(); + + Tokens token_kql(kql_statement.data(), kql_statement.data() + kql_statement.size()); + IParser::Pos pos_kql(token_kql, pos.max_depth, pos.max_backtracks); + + return ParserKQLWithUnionQuery().parse(pos_kql, node, expected); } } From c7917f747d0c7a8cc5c58d759511effa5bd321dc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 19:34:30 +0100 Subject: [PATCH 013/254] Remove garbage from KQL, part 3 --- src/Client/ClientBase.cpp | 1 - src/Parsers/Kusto/ParserKQLStatement.cpp | 2 +- src/Parsers/Kusto/parseKQLQuery.cpp | 2 +- src/Parsers/TokenIterator.cpp | 4 ++-- src/Parsers/TokenIterator.h | 5 ++++- 5 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 767a9b2b9f9e..aa7b857c07da 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -44,7 +44,6 @@ #include #include #include -#include #include #include diff --git a/src/Parsers/Kusto/ParserKQLStatement.cpp b/src/Parsers/Kusto/ParserKQLStatement.cpp index 7668779356e8..83c2aa64add6 100644 --- a/src/Parsers/Kusto/ParserKQLStatement.cpp +++ b/src/Parsers/Kusto/ParserKQLStatement.cpp @@ -77,7 +77,7 @@ bool ParserKQLTableFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expe String kql_statement = typeid_cast(*string_literal).value.safeGet(); - Tokens token_kql(kql_statement.data(), kql_statement.data() + kql_statement.size()); + Tokens token_kql(kql_statement.data(), kql_statement.data() + kql_statement.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos_kql(token_kql, pos.max_depth, pos.max_backtracks); return ParserKQLWithUnionQuery().parse(pos_kql, node, expected); diff --git a/src/Parsers/Kusto/parseKQLQuery.cpp b/src/Parsers/Kusto/parseKQLQuery.cpp index 34a009873f85..4501a34cd846 100644 --- a/src/Parsers/Kusto/parseKQLQuery.cpp +++ b/src/Parsers/Kusto/parseKQLQuery.cpp @@ -326,7 +326,7 @@ ASTPtr tryParseKQLQuery( bool skip_insignificant) { const char * query_begin = _out_query_end; - Tokens tokens(query_begin, all_queries_end, max_query_size, skip_insignificant); + Tokens tokens(query_begin, all_queries_end, max_query_size, skip_insignificant, /* greedy_errors= */ true); /// NOTE: consider use UInt32 for max_parser_depth setting. IParser::Pos token_iterator(tokens, static_cast(max_parser_depth), static_cast(max_parser_backtracks)); diff --git a/src/Parsers/TokenIterator.cpp b/src/Parsers/TokenIterator.cpp index 6ef01c163d34..4c7400950d67 100644 --- a/src/Parsers/TokenIterator.cpp +++ b/src/Parsers/TokenIterator.cpp @@ -4,7 +4,7 @@ namespace DB { -Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool skip_insignificant) +Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool skip_insignificant, bool greedy_errors) { Lexer lexer(begin, end, max_query_size); @@ -12,7 +12,7 @@ Tokens::Tokens(const char * begin, const char * end, size_t max_query_size, bool do { Token token = lexer.nextToken(); - stop = token.isEnd() || token.isError(); + stop = token.isEnd() || (!greedy_errors && token.isError()) || token.type == TokenType::ErrorMaxQuerySizeExceeded; if (token.isSignificant() || (!skip_insignificant && !data.empty() && data.back().isSignificant())) data.emplace_back(std::move(token)); } while (!stop); diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index cf370a523ac2..7bea09d3d296 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -24,7 +24,10 @@ class Tokens std::size_t last_accessed_index = 0; public: - Tokens(const char * begin, const char * end, size_t max_query_size = 0, bool skip_insignificant = true); + /// If skip_insignificant is true, it will transparently skip whitespaces and comments. + /// If greedy_errors is true, it will continue tokenization after the first error until the end. + Tokens(const char * begin, const char * end, size_t max_query_size = 0, + bool skip_insignificant = true, bool greedy_errors = false); ALWAYS_INLINE inline const Token & operator[](size_t index) { From 0058bfb2cac99c708fa85ffba8d3de552babe28b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 19:39:03 +0100 Subject: [PATCH 014/254] Remove garbage from KQL, part 3 --- src/Parsers/Kusto/ParserKQLStatement.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/Kusto/ParserKQLStatement.cpp b/src/Parsers/Kusto/ParserKQLStatement.cpp index 83c2aa64add6..0ac8eec00b7c 100644 --- a/src/Parsers/Kusto/ParserKQLStatement.cpp +++ b/src/Parsers/Kusto/ParserKQLStatement.cpp @@ -63,7 +63,7 @@ bool ParserKQLWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & exp bool ParserKQLTableFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserToken lparen(TokenType::OpeningRoundBracket); - ParserToken rparen(TokenType::OpeningRoundBracket); + ParserToken rparen(TokenType::ClosingRoundBracket); ASTPtr string_literal; ParserStringLiteral parser_string_literal; From b1364ee4658d14afe52a8d3f2eb64091d8127531 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 19:48:45 +0100 Subject: [PATCH 015/254] Remove garbage from KQL, part 4 --- .../Kusto/KustoFunctions/IParserKQLFunction.cpp | 4 ++-- src/Parsers/Kusto/ParserKQLDistinct.cpp | 2 +- src/Parsers/Kusto/ParserKQLExtend.cpp | 4 ++-- src/Parsers/Kusto/ParserKQLFilter.cpp | 2 +- src/Parsers/Kusto/ParserKQLLimit.cpp | 2 +- src/Parsers/Kusto/ParserKQLMVExpand.cpp | 2 +- src/Parsers/Kusto/ParserKQLMakeSeries.cpp | 6 +++--- src/Parsers/Kusto/ParserKQLPrint.cpp | 2 +- src/Parsers/Kusto/ParserKQLProject.cpp | 2 +- src/Parsers/Kusto/ParserKQLQuery.cpp | 12 ++++++------ src/Parsers/Kusto/ParserKQLSort.cpp | 2 +- src/Parsers/Kusto/ParserKQLSummarize.cpp | 4 ++-- 12 files changed, 22 insertions(+), 22 deletions(-) diff --git a/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp b/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp index 1d77007a37c2..061d6e8420df 100644 --- a/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp +++ b/src/Parsers/Kusto/KustoFunctions/IParserKQLFunction.cpp @@ -301,8 +301,8 @@ String IParserKQLFunction::kqlCallToExpression( }); const auto kql_call = std::format("{}({})", function_name, params_str); - DB::Tokens call_tokens(kql_call.c_str(), kql_call.c_str() + kql_call.length()); - DB::IParser::Pos tokens_pos(call_tokens, max_depth, max_backtracks); + Tokens call_tokens(kql_call.data(), kql_call.data() + kql_call.length(), 0, true, /* greedy_errors= */ true); + IParser::Pos tokens_pos(call_tokens, max_depth, max_backtracks); return DB::IParserKQLFunction::getExpression(tokens_pos); } diff --git a/src/Parsers/Kusto/ParserKQLDistinct.cpp b/src/Parsers/Kusto/ParserKQLDistinct.cpp index 3ec823a61b5c..43c8bcd7993a 100644 --- a/src/Parsers/Kusto/ParserKQLDistinct.cpp +++ b/src/Parsers/Kusto/ParserKQLDistinct.cpp @@ -11,7 +11,7 @@ bool ParserKQLDistinct::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) expr = getExprFromToken(pos); - Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); + Tokens tokens(expr.data(), expr.data() + expr.size(), 0, true, /* greedy_errors= */ true); IParser::Pos new_pos(tokens, pos.max_depth, pos.max_backtracks); if (!ParserNotEmptyExpressionList(false).parse(new_pos, select_expression_list, expected)) diff --git a/src/Parsers/Kusto/ParserKQLExtend.cpp b/src/Parsers/Kusto/ParserKQLExtend.cpp index 41ce296bd25d..b67bbf96e197 100644 --- a/src/Parsers/Kusto/ParserKQLExtend.cpp +++ b/src/Parsers/Kusto/ParserKQLExtend.cpp @@ -22,7 +22,7 @@ bool ParserKQLExtend ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) String except_str; String new_extend_str; - Tokens ntokens(extend_expr.c_str(), extend_expr.c_str() + extend_expr.size()); + Tokens ntokens(extend_expr.data(), extend_expr.data() + extend_expr.size(), 0, true, /* greedy_errors= */ true); IParser::Pos npos(ntokens, pos.max_depth, pos.max_backtracks); String alias; @@ -76,7 +76,7 @@ bool ParserKQLExtend ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) apply_alias(); String expr = std::format("SELECT * {}, {} from prev", except_str, new_extend_str); - Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); + Tokens tokens(expr.data(), expr.data() + expr.size(), 0, true, /* greedy_errors= */ true); IParser::Pos new_pos(tokens, pos.max_depth, pos.max_backtracks); if (!ParserSelectQuery().parse(new_pos, select_query, expected)) diff --git a/src/Parsers/Kusto/ParserKQLFilter.cpp b/src/Parsers/Kusto/ParserKQLFilter.cpp index b060ce8d2c7a..df71b954edcf 100644 --- a/src/Parsers/Kusto/ParserKQLFilter.cpp +++ b/src/Parsers/Kusto/ParserKQLFilter.cpp @@ -13,7 +13,7 @@ bool ParserKQLFilter::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) String expr = getExprFromToken(pos); ASTPtr where_expression; - Tokens token_filter(expr.c_str(), expr.c_str() + expr.size()); + Tokens token_filter(expr.data(), expr.data() + expr.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos_filter(token_filter, pos.max_depth, pos.max_backtracks); if (!ParserExpressionWithOptionalAlias(false).parse(pos_filter, where_expression, expected)) return false; diff --git a/src/Parsers/Kusto/ParserKQLLimit.cpp b/src/Parsers/Kusto/ParserKQLLimit.cpp index 0eb460757b19..dfa09272c9c3 100644 --- a/src/Parsers/Kusto/ParserKQLLimit.cpp +++ b/src/Parsers/Kusto/ParserKQLLimit.cpp @@ -13,7 +13,7 @@ bool ParserKQLLimit::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) auto expr = getExprFromToken(pos); - Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); + Tokens tokens(expr.data(), expr.data() + expr.size(), 0, true, /* greedy_errors= */ true); IParser::Pos new_pos(tokens, pos.max_depth, pos.max_backtracks); if (!ParserExpressionWithOptionalAlias(false).parse(new_pos, limit_length, expected)) diff --git a/src/Parsers/Kusto/ParserKQLMVExpand.cpp b/src/Parsers/Kusto/ParserKQLMVExpand.cpp index 9beb1c39e34e..2e9ad5a56c86 100644 --- a/src/Parsers/Kusto/ParserKQLMVExpand.cpp +++ b/src/Parsers/Kusto/ParserKQLMVExpand.cpp @@ -298,7 +298,7 @@ bool ParserKQLMVExpand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; const String setting_str = "enable_unaligned_array_join = 1"; - Tokens token_settings(setting_str.c_str(), setting_str.c_str() + setting_str.size()); + Tokens token_settings(setting_str.data(), setting_str.data() + setting_str.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos_settings(token_settings, pos.max_depth, pos.max_backtracks); if (!ParserSetQuery(true).parse(pos_settings, setting, expected)) diff --git a/src/Parsers/Kusto/ParserKQLMakeSeries.cpp b/src/Parsers/Kusto/ParserKQLMakeSeries.cpp index f1ad9d9738b1..505ba0495717 100644 --- a/src/Parsers/Kusto/ParserKQLMakeSeries.cpp +++ b/src/Parsers/Kusto/ParserKQLMakeSeries.cpp @@ -173,7 +173,7 @@ bool ParserKQLMakeSeries ::makeSeries(KQLMakeSeries & kql_make_series, ASTPtr & auto date_type_cast = [&](String & src) { - Tokens tokens(src.c_str(), src.c_str() + src.size()); + Tokens tokens(src.data(), src.data() + src.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos(tokens, max_depth, max_backtracks); String res; while (isValidKQLPos(pos)) @@ -200,7 +200,7 @@ bool ParserKQLMakeSeries ::makeSeries(KQLMakeSeries & kql_make_series, ASTPtr & auto get_group_expression_alias = [&] { std::vector group_expression_tokens; - Tokens tokens(group_expression.c_str(), group_expression.c_str() + group_expression.size()); + Tokens tokens(group_expression.data(), group_expression.data() + group_expression.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos(tokens, max_depth, max_backtracks); while (isValidKQLPos(pos)) { @@ -413,7 +413,7 @@ bool ParserKQLMakeSeries ::parseImpl(Pos & pos, ASTPtr & node, Expected & expect makeSeries(kql_make_series, node, pos.max_depth, pos.max_backtracks); - Tokens token_main_query(kql_make_series.main_query.c_str(), kql_make_series.main_query.c_str() + kql_make_series.main_query.size()); + Tokens token_main_query(kql_make_series.main_query.data(), kql_make_series.main_query.data() + kql_make_series.main_query.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos_main_query(token_main_query, pos.max_depth, pos.max_backtracks); if (!ParserNotEmptyExpressionList(true).parse(pos_main_query, select_expression_list, expected)) diff --git a/src/Parsers/Kusto/ParserKQLPrint.cpp b/src/Parsers/Kusto/ParserKQLPrint.cpp index 37483439f143..24f1fcfae774 100644 --- a/src/Parsers/Kusto/ParserKQLPrint.cpp +++ b/src/Parsers/Kusto/ParserKQLPrint.cpp @@ -9,7 +9,7 @@ bool ParserKQLPrint::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr select_expression_list; const String expr = getExprFromToken(pos); - Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); + Tokens tokens(expr.data(), expr.data() + expr.size(), 0, true, /* greedy_errors= */ true); IParser::Pos new_pos(tokens, pos.max_depth, pos.max_backtracks); if (!ParserNotEmptyExpressionList(true).parse(new_pos, select_expression_list, expected)) diff --git a/src/Parsers/Kusto/ParserKQLProject.cpp b/src/Parsers/Kusto/ParserKQLProject.cpp index eab9ee082c5a..b0debbd408da 100644 --- a/src/Parsers/Kusto/ParserKQLProject.cpp +++ b/src/Parsers/Kusto/ParserKQLProject.cpp @@ -11,7 +11,7 @@ bool ParserKQLProject ::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) expr = getExprFromToken(pos); - Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); + Tokens tokens(expr.data(), expr.data() + expr.size(), 0, true, /* greedy_errors= */ true); IParser::Pos new_pos(tokens, pos.max_depth, pos.max_backtracks); if (!ParserNotEmptyExpressionList(false).parse(new_pos, select_expression_list, expected)) diff --git a/src/Parsers/Kusto/ParserKQLQuery.cpp b/src/Parsers/Kusto/ParserKQLQuery.cpp index 99b2d1da8907..9e8994bfa33e 100644 --- a/src/Parsers/Kusto/ParserKQLQuery.cpp +++ b/src/Parsers/Kusto/ParserKQLQuery.cpp @@ -37,7 +37,7 @@ bool ParserKQLBase::parseByString(String expr, ASTPtr & node, uint32_t max_depth { Expected expected; - Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); + Tokens tokens(expr.data(), expr.data() + expr.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos(tokens, max_depth, max_backtracks); return parse(pos, node, expected); } @@ -45,7 +45,7 @@ bool ParserKQLBase::parseByString(String expr, ASTPtr & node, uint32_t max_depth bool ParserKQLBase::parseSQLQueryByString(ParserPtr && parser, String & query, ASTPtr & select_node, uint32_t max_depth, uint32_t max_backtracks) { Expected expected; - Tokens token_subquery(query.c_str(), query.c_str() + query.size()); + Tokens token_subquery(query.data(), query.data() + query.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos_subquery(token_subquery, max_depth, max_backtracks); if (!parser->parse(pos_subquery, select_node, expected)) return false; @@ -123,7 +123,7 @@ bool ParserKQLBase::setSubQuerySource(ASTPtr & select_query, ASTPtr & source, bo String ParserKQLBase::getExprFromToken(const String & text, uint32_t max_depth, uint32_t max_backtracks) { - Tokens tokens(text.c_str(), text.c_str() + text.size()); + Tokens tokens(text.data(), text.data() + text.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos(tokens, max_depth, max_backtracks); return getExprFromToken(pos); @@ -522,7 +522,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) --last_pos; String sub_query = std::format("({})", String(operation_pos.front().second->begin, last_pos->end)); - Tokens token_subquery(sub_query.c_str(), sub_query.c_str() + sub_query.size()); + Tokens token_subquery(sub_query.data(), sub_query.data() + sub_query.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos_subquery(token_subquery, pos.max_depth, pos.max_backtracks); if (!ParserKQLSubquery().parse(pos_subquery, tables, expected)) @@ -543,7 +543,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) auto oprator = getOperator(op_str); if (oprator) { - Tokens token_clause(op_calsue.c_str(), op_calsue.c_str() + op_calsue.size()); + Tokens token_clause(op_calsue.data(), op_calsue.data() + op_calsue.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos_clause(token_clause, pos.max_depth, pos.max_backtracks); if (!oprator->parse(pos_clause, node, expected)) return false; @@ -576,7 +576,7 @@ bool ParserKQLQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!node->as()->select()) { auto expr = String("*"); - Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); + Tokens tokens(expr.data(), expr.data() + expr.size(), 0, true, /* greedy_errors= */ true); IParser::Pos new_pos(tokens, pos.max_depth, pos.max_backtracks); if (!std::make_unique()->parse(new_pos, node, expected)) return false; diff --git a/src/Parsers/Kusto/ParserKQLSort.cpp b/src/Parsers/Kusto/ParserKQLSort.cpp index 852ba50698d1..9797cd96157a 100644 --- a/src/Parsers/Kusto/ParserKQLSort.cpp +++ b/src/Parsers/Kusto/ParserKQLSort.cpp @@ -18,7 +18,7 @@ bool ParserKQLSort::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) auto expr = getExprFromToken(pos); - Tokens tokens(expr.c_str(), expr.c_str() + expr.size()); + Tokens tokens(expr.data(), expr.data() + expr.size(), 0, true, /* greedy_errors= */ true); IParser::Pos new_pos(tokens, pos.max_depth, pos.max_backtracks); auto pos_backup = new_pos; diff --git a/src/Parsers/Kusto/ParserKQLSummarize.cpp b/src/Parsers/Kusto/ParserKQLSummarize.cpp index 47d706d0b4bc..751878283651 100644 --- a/src/Parsers/Kusto/ParserKQLSummarize.cpp +++ b/src/Parsers/Kusto/ParserKQLSummarize.cpp @@ -194,7 +194,7 @@ bool ParserKQLSummarize::parseImpl(Pos & pos, ASTPtr & node, Expected & expected String converted_columns = getExprFromToken(expr_columns, pos.max_depth, pos.max_backtracks); - Tokens token_converted_columns(converted_columns.c_str(), converted_columns.c_str() + converted_columns.size()); + Tokens token_converted_columns(converted_columns.data(), converted_columns.data() + converted_columns.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos_converted_columns(token_converted_columns, pos.max_depth, pos.max_backtracks); if (!ParserNotEmptyExpressionList(true).parse(pos_converted_columns, select_expression_list, expected)) @@ -206,7 +206,7 @@ bool ParserKQLSummarize::parseImpl(Pos & pos, ASTPtr & node, Expected & expected { String converted_groupby = getExprFromToken(expr_groupby, pos.max_depth, pos.max_backtracks); - Tokens token_converted_groupby(converted_groupby.c_str(), converted_groupby.c_str() + converted_groupby.size()); + Tokens token_converted_groupby(converted_groupby.data(), converted_groupby.data() + converted_groupby.size(), 0, true, /* greedy_errors= */ true); IParser::Pos postoken_converted_groupby(token_converted_groupby, pos.max_depth, pos.max_backtracks); if (!ParserNotEmptyExpressionList(false).parse(postoken_converted_groupby, group_expression_list, expected)) From 38130ef5a34a2a1398b9d633e99bb8ed69a1aa92 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 21:39:31 +0100 Subject: [PATCH 016/254] Fix build --- src/Client/ClientBase.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index aa7b857c07da..17a862ea5481 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -43,6 +43,7 @@ #include #include #include +#include #include #include From 6f11f85801e4b1f5027568ce035bccaa6e26dd20 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 21:40:47 +0100 Subject: [PATCH 017/254] Remove unused header --- src/Client/ClientBase.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 17a862ea5481..03d26c5dcb6a 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -49,7 +49,6 @@ #include #include -#include #include #include #include From 7ce344d9554d61aabd7b9e5ce946a030ae18b615 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 04:03:55 +0100 Subject: [PATCH 018/254] The code of KQL appeared to be even more idiotic that I expected --- src/Parsers/Kusto/ParserKQLStatement.cpp | 39 +++++++++++++++++++----- 1 file changed, 31 insertions(+), 8 deletions(-) diff --git a/src/Parsers/Kusto/ParserKQLStatement.cpp b/src/Parsers/Kusto/ParserKQLStatement.cpp index 0ac8eec00b7c..77c15aa7561b 100644 --- a/src/Parsers/Kusto/ParserKQLStatement.cpp +++ b/src/Parsers/Kusto/ParserKQLStatement.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -62,25 +63,47 @@ bool ParserKQLWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & exp bool ParserKQLTableFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { + /// TODO: This code is idiotic, see https://github.com/ClickHouse/ClickHouse/issues/61742 ParserToken lparen(TokenType::OpeningRoundBracket); - ParserToken rparen(TokenType::ClosingRoundBracket); ASTPtr string_literal; ParserStringLiteral parser_string_literal; - if (!(lparen.ignore(pos, expected) - && parser_string_literal.parse(pos, string_literal, expected) - && rparen.ignore(pos, expected))) - { + if (!lparen.ignore(pos, expected)) return false; - } - String kql_statement = typeid_cast(*string_literal).value.safeGet(); + size_t paren_count = 0; + String kql_statement; + if (parser_string_literal.parse(pos, string_literal, expected)) + { + kql_statement = typeid_cast(*string_literal).value.safeGet(); + } + else + { + ++paren_count; + auto pos_start = pos; + while (isValidKQLPos(pos)) + { + if (pos->type == TokenType::ClosingRoundBracket) + --paren_count; + if (pos->type == TokenType::OpeningRoundBracket) + ++paren_count; + + if (paren_count == 0) + break; + ++pos; + } + kql_statement = String(pos_start->begin, (--pos)->end); + } + ++pos; Tokens token_kql(kql_statement.data(), kql_statement.data() + kql_statement.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos_kql(token_kql, pos.max_depth, pos.max_backtracks); - return ParserKQLWithUnionQuery().parse(pos_kql, node, expected); + if (!ParserKQLWithUnionQuery().parse(pos_kql, node, expected)) + return false; + ++pos; + return true; } } From 3f685dec2648bf0cd9de2f9bdb216076ae8798c5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 25 Mar 2024 04:14:45 +0100 Subject: [PATCH 019/254] Idiotic KQL --- src/Parsers/Kusto/ParserKQLStatement.cpp | 6 ++++-- src/Parsers/Kusto/ParserKQLStatement.h | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Parsers/Kusto/ParserKQLStatement.cpp b/src/Parsers/Kusto/ParserKQLStatement.cpp index 77c15aa7561b..059744682e56 100644 --- a/src/Parsers/Kusto/ParserKQLStatement.cpp +++ b/src/Parsers/Kusto/ParserKQLStatement.cpp @@ -64,6 +64,7 @@ bool ParserKQLWithUnionQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & exp bool ParserKQLTableFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { /// TODO: This code is idiotic, see https://github.com/ClickHouse/ClickHouse/issues/61742 + ParserToken lparen(TokenType::OpeningRoundBracket); ASTPtr string_literal; @@ -93,9 +94,10 @@ bool ParserKQLTableFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expe break; ++pos; } - kql_statement = String(pos_start->begin, (--pos)->end); + --pos; + kql_statement = String(pos_start->begin, pos->end); + ++pos; } - ++pos; Tokens token_kql(kql_statement.data(), kql_statement.data() + kql_statement.size(), 0, true, /* greedy_errors= */ true); IParser::Pos pos_kql(token_kql, pos.max_depth, pos.max_backtracks); diff --git a/src/Parsers/Kusto/ParserKQLStatement.h b/src/Parsers/Kusto/ParserKQLStatement.h index fe9b9adfa2ac..b1cd782d36bd 100644 --- a/src/Parsers/Kusto/ParserKQLStatement.h +++ b/src/Parsers/Kusto/ParserKQLStatement.h @@ -45,7 +45,7 @@ class ParserKQLWithUnionQuery : public IParserBase class ParserKQLTableFunction : public IParserBase { protected: - const char * getName() const override { return "KQL() function"; } + const char * getName() const override { return "KQL function"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; From 149e2af36925ebc36bdd9bee2466ed7424bb259c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 10 Apr 2024 17:46:30 +0000 Subject: [PATCH 020/254] fix for one thread --- src/Interpreters/SquashingTransform.cpp | 5 +- src/Interpreters/SquashingTransform.h | 4 + .../Transforms/BalancingTransform.cpp | 92 +++++++++++-------- 3 files changed, 57 insertions(+), 44 deletions(-) diff --git a/src/Interpreters/SquashingTransform.cpp b/src/Interpreters/SquashingTransform.cpp index 4a259d0b7c90..62c819a6105b 100644 --- a/src/Interpreters/SquashingTransform.cpp +++ b/src/Interpreters/SquashingTransform.cpp @@ -218,10 +218,7 @@ Chunk BalanceTransform::convertToChunk(std::vector &chunks) info->chunks.push_back(chunk.clone()); info->data_type = data_type; - if (!info->chunks.empty()) /// Note: This if is only for debugging, structure of chunk copies the structure of info - { /// it's possible to use only 'Chunk(header.cloneEmptyColumns(), 0, info)' - return Chunk({info->chunks[0].getColumns(), info->chunks[0].getNumRows(), info}); - } + chunks.clear(); return Chunk(header.cloneEmptyColumns(), 0, info); } diff --git a/src/Interpreters/SquashingTransform.h b/src/Interpreters/SquashingTransform.h index fb6834e03bea..4551b76e25f3 100644 --- a/src/Interpreters/SquashingTransform.h +++ b/src/Interpreters/SquashingTransform.h @@ -86,6 +86,10 @@ class BalanceTransform BalanceTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); Chunk add(Block && input_block); + bool isDataLeft() + { + return !chunks_to_merge_vec.empty(); + } private: std::vector chunks_to_merge_vec = {}; diff --git a/src/Processors/Transforms/BalancingTransform.cpp b/src/Processors/Transforms/BalancingTransform.cpp index 58c2f052c1a7..a6a79f65ea49 100644 --- a/src/Processors/Transforms/BalancingTransform.cpp +++ b/src/Processors/Transforms/BalancingTransform.cpp @@ -1,6 +1,7 @@ #include #include #include +#include "Processors/IProcessor.h" namespace DB { @@ -17,6 +18,7 @@ BalancingChunksTransform::BalancingChunksTransform(const Block & header, size_t IProcessor::Status BalancingChunksTransform::prepare() { + LOG_TRACE(getLogger("balancingProcessor"), "prepare"); Status status = Status::Ready; while (status == Status::Ready) @@ -31,49 +33,58 @@ IProcessor::Status BalancingChunksTransform::prepare() IProcessor::Status BalancingChunksTransform::prepareConsume() { LOG_TRACE(getLogger("balancingProcessor"), "prepareConsume"); - for (auto & input : inputs) + finished = false; + while (!chunk.hasChunkInfo()) { - bool all_finished = true; - for (auto & output : outputs) - { - if (output.isFinished()) - continue; - - all_finished = false; - } - - if (all_finished) - { - input.close(); - return Status::Finished; - } - - if (input.isFinished()) + for (auto & input : inputs) { + bool all_finished = true; for (auto & output : outputs) - output.finish(); - - return Status::Finished; - } - - input.setNeeded(); - if (!input.hasData()) - return Status::NeedData; - - chunk = input.pull(); - was_output_processed.assign(outputs.size(), false); - transform(chunk); - if (chunk.hasChunkInfo()) - { - LOG_TRACE(getLogger("balancingProcessor"), "hasData"); - has_data = true; - } - else - { - finished = true; - LOG_TRACE(getLogger("balancingProcessor"), "hasData, finished"); + { + if (output.isFinished()) + continue; + + all_finished = false; + } + + if (all_finished) + { + input.close(); + return Status::Finished; + } + + if (input.isFinished()) + { + for (auto & output : outputs) + output.finish(); + + return Status::Finished; + } + + input.setNeeded(); + if (!input.hasData()) + { + finished = true; + if (!balance.isDataLeft()) + return Status::NeedData; + else + { + transform(chunk); + has_data = true; + return Status::Ready; + } + } + + chunk = input.pull(); transform(chunk); - has_data = true; + was_output_processed.assign(outputs.size(), false); + if (chunk.hasChunkInfo()) + { + LOG_TRACE(getLogger("balancingProcessor"), "hasData"); + has_data = true; + return Status::Ready; + } + } } return Status::Ready; @@ -121,8 +132,9 @@ IProcessor::Status BalancingChunksTransform::prepareSend() } LOG_TRACE(getLogger("balancingProcessor"), "chunk struct: {}", chunk.dumpStructure()); - output.push(chunk.clone()); + output.push(std::move(chunk)); was_processed = true; + break; } if (all_outputs_processed) From 94eb0782a945f6276481dc14262cd90d27dd1ebd Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 10 Apr 2024 22:22:13 +0000 Subject: [PATCH 021/254] fix for multiple threads --- src/Processors/Transforms/BalancingTransform.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/BalancingTransform.cpp b/src/Processors/Transforms/BalancingTransform.cpp index a6a79f65ea49..73672be5da48 100644 --- a/src/Processors/Transforms/BalancingTransform.cpp +++ b/src/Processors/Transforms/BalancingTransform.cpp @@ -53,7 +53,7 @@ IProcessor::Status BalancingChunksTransform::prepareConsume() return Status::Finished; } - if (input.isFinished()) + if (input.isFinished() && !balance.isDataLeft()) { for (auto & output : outputs) output.finish(); @@ -117,7 +117,10 @@ IProcessor::Status BalancingChunksTransform::prepareSend() ++chunk_number; if (!chunk.hasChunkInfo()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk info must be not empty in prepareGenerate()"); + { + has_data = false; + return Status::Ready; + } if (was_processed) continue; From e5e076a778c951eb66e07e2b85de1d82fbd60bff Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 10 Apr 2024 22:34:50 +0000 Subject: [PATCH 022/254] style fix --- src/Processors/Transforms/BalancingTransform.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Processors/Transforms/BalancingTransform.cpp b/src/Processors/Transforms/BalancingTransform.cpp index 73672be5da48..deb1abdb2fe1 100644 --- a/src/Processors/Transforms/BalancingTransform.cpp +++ b/src/Processors/Transforms/BalancingTransform.cpp @@ -6,11 +6,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - BalancingChunksTransform::BalancingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage_, size_t num_ports) : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), max_memory_usage(max_memory_usage_), balance(header, min_block_size_rows, min_block_size_bytes) { From 59718eafebcc23255d20ef73b400a4f9e4aa6219 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 11 Apr 2024 14:59:39 +0000 Subject: [PATCH 023/254] fix for unmatching types --- src/Interpreters/SquashingTransform.cpp | 26 ++++++++++++++++--------- src/Interpreters/SquashingTransform.h | 6 +++--- 2 files changed, 20 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/SquashingTransform.cpp b/src/Interpreters/SquashingTransform.cpp index 62c819a6105b..ca74bb7894a1 100644 --- a/src/Interpreters/SquashingTransform.cpp +++ b/src/Interpreters/SquashingTransform.cpp @@ -1,5 +1,7 @@ #include +#include "Common/logger_useful.h" #include +#include "IO/WriteHelpers.h" namespace DB @@ -141,7 +143,7 @@ Block NewSquashingTransform::add(Chunk && input_chunk) const ChunksToSquash * getInfoFromChunk(const Chunk & chunk) { - auto info = chunk.getChunkInfo(); + const auto& info = chunk.getChunkInfo(); const auto * agg_info = typeid_cast(info.get()); return agg_info; @@ -158,8 +160,10 @@ Block NewSquashingTransform::addImpl(ReferenceType input_chunk) } const auto *info = getInfoFromChunk(input_chunk); - for (auto & one : info->chunks) - append(std::move(one), info->data_type); + for (size_t i = 0; i < info->chunks.size(); i++) + append(std::move(info->chunks[i]), info->data_types); + // for (auto & one : info->chunks) + // append(std::move(one), info->data_types); { Block to_return; @@ -169,15 +173,19 @@ Block NewSquashingTransform::addImpl(ReferenceType input_chunk) } template -void NewSquashingTransform::append(ReferenceType input_chunk, DataTypePtr data_type) +void NewSquashingTransform::append(ReferenceType input_chunk, DataTypes data_types) { + // LOG_TRACE(getLogger("Squashing"), "data_type: {}", data_type->getName()); if (input_chunk.getNumColumns() == 0) return; if (!accumulated_block) { - for (const ColumnPtr& column : input_chunk.getColumns()) + // for (const ColumnPtr& column : input_chunk.getColumns()) + for (size_t i = 0; i < input_chunk.getNumColumns(); ++ i) { - ColumnWithTypeAndName col = ColumnWithTypeAndName(column, data_type, " "); + String name = data_types[i]->getName() + toString(i); + LOG_TRACE(getLogger("Squashing"), "data_type: {}", data_types[i]->getName()); + ColumnWithTypeAndName col = ColumnWithTypeAndName(input_chunk.getColumns()[i], data_types[i], name); accumulated_block.insert(accumulated_block.columns(), col); } return; @@ -216,7 +224,7 @@ Chunk BalanceTransform::convertToChunk(std::vector &chunks) auto info = std::make_shared(); for (auto &chunk : chunks) info->chunks.push_back(chunk.clone()); - info->data_type = data_type; + info->data_types = data_types; chunks.clear(); @@ -228,8 +236,8 @@ template Chunk BalanceTransform::addImpl(ReferenceType input_block) { Chunk input_chunk(input_block.getColumns(), input_block.rows()); - if (!data_type && !input_block.getDataTypes().empty()) - data_type = input_block.getDataTypes()[0]; + if (!input_block.getDataTypes().empty()) + data_types = input_block.getDataTypes(); if (!input_chunk) { Chunk res_chunk = convertToChunk(chunks_to_merge_vec); diff --git a/src/Interpreters/SquashingTransform.h b/src/Interpreters/SquashingTransform.h index 4551b76e25f3..b04d12b9bcd4 100644 --- a/src/Interpreters/SquashingTransform.h +++ b/src/Interpreters/SquashingTransform.h @@ -14,7 +14,7 @@ namespace DB struct ChunksToSquash : public ChunkInfo { mutable std::vector chunks = {}; - DataTypePtr data_type = nullptr; + DataTypes data_types = {}; }; /** Merging consecutive passed blocks to specified minimum size. @@ -74,7 +74,7 @@ class NewSquashingTransform Block addImpl(ReferenceType chunk); template - void append(ReferenceType input_chunk, DataTypePtr data_type); + void append(ReferenceType input_chunk, DataTypes data_types); bool isEnoughSize(const Block & block); bool isEnoughSize(size_t rows, size_t bytes) const; @@ -106,7 +106,7 @@ class BalanceTransform bool isEnoughSize(const std::vector & chunks); bool isEnoughSize(size_t rows, size_t bytes) const; void checkAndWaitMemoryAvailability(size_t bytes); - DataTypePtr data_type = nullptr; + DataTypes data_types = {}; MemoryTracker * memory_tracker; From 635b17aad2f8b53ae284a76698847774ef91a6e1 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 11 Apr 2024 15:46:09 +0000 Subject: [PATCH 024/254] chunk -> block in chunkInfo --- src/Interpreters/SquashingTransform.cpp | 150 ++++++++++++++++++ src/Interpreters/SquashingTransform.h | 57 +++++++ .../Transforms/BalancingTransform.h | 2 +- 3 files changed, 208 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/SquashingTransform.cpp b/src/Interpreters/SquashingTransform.cpp index ca74bb7894a1..ebd8a5f0c6e8 100644 --- a/src/Interpreters/SquashingTransform.cpp +++ b/src/Interpreters/SquashingTransform.cpp @@ -296,4 +296,154 @@ bool BalanceTransform::isEnoughSize(size_t rows, size_t bytes) const || (min_block_size_rows && rows >= min_block_size_rows) || (min_block_size_bytes && bytes >= min_block_size_bytes); } + +NewSquashingBlockTransform::NewSquashingBlockTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_) + : min_block_size_rows(min_block_size_rows_) + , min_block_size_bytes(min_block_size_bytes_) +{ +} + +Block NewSquashingBlockTransform::add(Chunk && input_chunk) +{ + return addImpl(std::move(input_chunk)); +} + +const BlocksToSquash * getInfoFromChunkBlock(const Chunk & chunk) +{ + const auto& info = chunk.getChunkInfo(); + const auto * agg_info = typeid_cast(info.get()); + + return agg_info; +} + +Block NewSquashingBlockTransform::addImpl(Chunk && input_chunk) +{ + if (!input_chunk.hasChunkInfo()) + { + Block to_return; + std::swap(to_return, accumulated_block); + return to_return; + } + + const auto *info = getInfoFromChunkBlock(input_chunk); + for (auto & block : info->blocks) + append(std::move(block)); + + { + Block to_return; + std::swap(to_return, accumulated_block); + return to_return; + } +} + +void NewSquashingBlockTransform::append(Block && input_block) +{ + if (input_block.columns() == 0) + return; + if (!accumulated_block) + { + for (size_t i = 0; i < input_block.columns(); ++ i) + { + LOG_TRACE(getLogger("Squashing"), "data_type: {}", input_block.getDataTypeNames()[i]); + ColumnWithTypeAndName col = ColumnWithTypeAndName(input_block.getColumns()[i], input_block.getDataTypes()[i], input_block.getNames()[i]); + accumulated_block.insert(accumulated_block.columns(), col); + } + return; + } + + for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) + { + const auto source_column = input_block.getColumns()[i]; + + auto mutable_column = IColumn::mutate(std::move(accumulated_block.getByPosition(i).column)); + mutable_column->insertRangeFrom(*source_column, 0, source_column->size()); + accumulated_block.getByPosition(i).column = std::move(mutable_column); + } +} + +BalanceBlockTransform::BalanceBlockTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) + : min_block_size_rows(min_block_size_rows_) + , min_block_size_bytes(min_block_size_bytes_) + , header(std::move(header_)) +{ + // Use query-level memory tracker + if (auto * memory_tracker_child = CurrentThread::getMemoryTracker()) + memory_tracker = memory_tracker_child->getParent(); +} + +Chunk BalanceBlockTransform::add(Block && input_block) +{ + return addImpl(std::move(input_block)); +} + +Chunk BalanceBlockTransform::addImpl(Block && input_block) +{ + Chunk input_chunk(input_block.getColumns(), input_block.rows()); + + if (!input_chunk) + { + Chunk res_chunk = convertToChunk(blocks_to_merge_vec); + return res_chunk; + } + + if (isEnoughSize(blocks_to_merge_vec)) + blocks_to_merge_vec.clear(); + + if (input_chunk) + blocks_to_merge_vec.push_back(std::move(input_block)); + + if (isEnoughSize(blocks_to_merge_vec)) + { + Chunk res_chunk = convertToChunk(blocks_to_merge_vec); + return res_chunk; + } + return input_chunk; +} + +Chunk BalanceBlockTransform::convertToChunk(std::vector &blocks) +{ + if (blocks.empty()) + return {}; + + auto info = std::make_shared(); + for (auto &block : blocks) + info->blocks.push_back(std::move(block)); + + blocks.clear(); // we can remove this + + return Chunk(header.cloneEmptyColumns(), 0, info); +} + +bool BalanceBlockTransform::isEnoughSize(const std::vector & blocks) +{ + size_t rows = 0; + size_t bytes = 0; + + for (const Block & block : blocks) + { + rows += block.rows(); + bytes += block.bytes(); + } + checkAndWaitMemoryAvailability(bytes); + + return isEnoughSize(rows, bytes); +} + +void BalanceBlockTransform::checkAndWaitMemoryAvailability(size_t bytes) +{ + // bytes_used += bytes; + if (const auto hard_limit = memory_tracker->getHardLimit() != 0) + { + auto free_memory = hard_limit - memory_tracker->get(); + while (Int64(bytes) >= free_memory) + free_memory = hard_limit - memory_tracker->get(); + } +} + +bool BalanceBlockTransform::isEnoughSize(size_t rows, size_t bytes) const +{ + return (!min_block_size_rows && !min_block_size_bytes) + || (min_block_size_rows && rows >= min_block_size_rows) + || (min_block_size_bytes && bytes >= min_block_size_bytes); +} } diff --git a/src/Interpreters/SquashingTransform.h b/src/Interpreters/SquashingTransform.h index b04d12b9bcd4..792a8c2efcf3 100644 --- a/src/Interpreters/SquashingTransform.h +++ b/src/Interpreters/SquashingTransform.h @@ -113,4 +113,61 @@ class BalanceTransform Chunk convertToChunk(std::vector &chunks); }; +class NewSquashingBlockTransform +{ +public: + NewSquashingBlockTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_); + + Block add(Chunk && input_chunk); + +private: + size_t min_block_size_rows; + size_t min_block_size_bytes; + + Block accumulated_block; + + Block addImpl(Chunk && chunk); + + void append(Block && input_block); + + bool isEnoughSize(const Block & block); + bool isEnoughSize(size_t rows, size_t bytes) const; +}; + +struct BlocksToSquash : public ChunkInfo +{ + mutable std::vector blocks = {}; +}; + +class BalanceBlockTransform +{ +public: + BalanceBlockTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); + + Chunk add(Block && input_block); + bool isDataLeft() + { + return !blocks_to_merge_vec.empty(); + } + +private: + std::vector blocks_to_merge_vec = {}; + size_t min_block_size_rows; + size_t min_block_size_bytes; + + Block accumulated_block; + const Block header; + + // template + Chunk addImpl(Block && input_block); + + bool isEnoughSize(const std::vector & blocks); + bool isEnoughSize(size_t rows, size_t bytes) const; + void checkAndWaitMemoryAvailability(size_t bytes); + + MemoryTracker * memory_tracker; + + Chunk convertToChunk(std::vector &blocks); +}; + } diff --git a/src/Processors/Transforms/BalancingTransform.h b/src/Processors/Transforms/BalancingTransform.h index 1b1d3ec6295b..a50b38c773f8 100644 --- a/src/Processors/Transforms/BalancingTransform.h +++ b/src/Processors/Transforms/BalancingTransform.h @@ -31,7 +31,7 @@ class BalancingChunksTransform : public IProcessor size_t CalculateBlockSize(const Block & block); Chunk chunk; size_t max_memory_usage; - BalanceTransform balance; + BalanceBlockTransform balance; bool has_data = false; std::vector was_output_processed; From 958b83a76a588c98b76f8c310b63cf2798fdbc1a Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 11 Apr 2024 18:36:54 +0000 Subject: [PATCH 025/254] changed Block -> Chunk and removed dataTypes, beautified --- src/Interpreters/InterpreterInsertQuery.cpp | 14 +- src/Interpreters/SquashingTransform.cpp | 239 +++--------------- src/Interpreters/SquashingTransform.h | 75 +----- .../Transforms/BalancingTransform.cpp | 10 +- .../Transforms/BalancingTransform.h | 2 +- .../Transforms/SquashingChunksTransform.cpp | 11 +- .../Transforms/SquashingChunksTransform.h | 4 +- 7 files changed, 57 insertions(+), 298 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 0041a0f08468..31fef267afce 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -669,20 +669,20 @@ BlockIO InterpreterInsertQuery::execute() { bool table_prefers_large_blocks = table->prefersLargeBlocks(); - auto squashing = std::make_shared( + auto squashing = std::make_shared( chain.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); chain.addSource(std::move(squashing)); - // auto balancing = std::make_shared( - // chain.getInputHeader(), - // table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - // table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, - // settings.max_memory_usage, true); + auto balancing = std::make_shared( + chain.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, + settings.max_memory_usage, true); - // chain.addSource(std::move(balancing)); + chain.addSource(std::move(balancing)); } auto context_ptr = getContext(); diff --git a/src/Interpreters/SquashingTransform.cpp b/src/Interpreters/SquashingTransform.cpp index ebd8a5f0c6e8..d4370b45119f 100644 --- a/src/Interpreters/SquashingTransform.cpp +++ b/src/Interpreters/SquashingTransform.cpp @@ -1,7 +1,5 @@ #include -#include "Common/logger_useful.h" #include -#include "IO/WriteHelpers.h" namespace DB @@ -129,28 +127,19 @@ bool SquashingTransform::isEnoughSize(size_t rows, size_t bytes) const || (min_block_size_bytes && bytes >= min_block_size_bytes); } - -NewSquashingTransform::NewSquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_) +NewSquashingTransform::NewSquashingTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) + , header(std::move(header_)) { } Block NewSquashingTransform::add(Chunk && input_chunk) { - return addImpl(std::move(input_chunk)); -} - -const ChunksToSquash * getInfoFromChunk(const Chunk & chunk) -{ - const auto& info = chunk.getChunkInfo(); - const auto * agg_info = typeid_cast(info.get()); - - return agg_info; + return addImpl(std::move(input_chunk)); } -template -Block NewSquashingTransform::addImpl(ReferenceType input_chunk) +Block NewSquashingTransform::addImpl(Chunk && input_chunk) { if (!input_chunk.hasChunkInfo()) { @@ -160,10 +149,8 @@ Block NewSquashingTransform::addImpl(ReferenceType input_chunk) } const auto *info = getInfoFromChunk(input_chunk); - for (size_t i = 0; i < info->chunks.size(); i++) - append(std::move(info->chunks[i]), info->data_types); - // for (auto & one : info->chunks) - // append(std::move(one), info->data_types); + for (auto & chunk : info->chunks) + append(chunk.clone()); { Block to_return; @@ -172,20 +159,15 @@ Block NewSquashingTransform::addImpl(ReferenceType input_chunk) } } -template -void NewSquashingTransform::append(ReferenceType input_chunk, DataTypes data_types) +void NewSquashingTransform::append(Chunk && input_chunk) { - // LOG_TRACE(getLogger("Squashing"), "data_type: {}", data_type->getName()); if (input_chunk.getNumColumns() == 0) return; if (!accumulated_block) { - // for (const ColumnPtr& column : input_chunk.getColumns()) for (size_t i = 0; i < input_chunk.getNumColumns(); ++ i) { - String name = data_types[i]->getName() + toString(i); - LOG_TRACE(getLogger("Squashing"), "data_type: {}", data_types[i]->getName()); - ColumnWithTypeAndName col = ColumnWithTypeAndName(input_chunk.getColumns()[i], data_types[i], name); + ColumnWithTypeAndName col = ColumnWithTypeAndName(input_chunk.getColumns()[i], header.getDataTypes()[i], header.getNames()[i]); accumulated_block.insert(accumulated_block.columns(), col); } return; @@ -201,167 +183,15 @@ void NewSquashingTransform::append(ReferenceType input_chunk, DataTypes data_typ } } -BalanceTransform::BalanceTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) - : min_block_size_rows(min_block_size_rows_) - , min_block_size_bytes(min_block_size_bytes_) - , header(std::move(header_)) -{ - // Use query-level memory tracker - if (auto * memory_tracker_child = CurrentThread::getMemoryTracker()) - memory_tracker = memory_tracker_child->getParent(); -} - -Chunk BalanceTransform::add(Block && input_block) -{ - return addImpl(std::move(input_block)); -} - -Chunk BalanceTransform::convertToChunk(std::vector &chunks) -{ - if (chunks.empty()) - return {}; - - auto info = std::make_shared(); - for (auto &chunk : chunks) - info->chunks.push_back(chunk.clone()); - info->data_types = data_types; - - chunks.clear(); - - return Chunk(header.cloneEmptyColumns(), 0, info); -} - - -template -Chunk BalanceTransform::addImpl(ReferenceType input_block) -{ - Chunk input_chunk(input_block.getColumns(), input_block.rows()); - if (!input_block.getDataTypes().empty()) - data_types = input_block.getDataTypes(); - if (!input_chunk) - { - Chunk res_chunk = convertToChunk(chunks_to_merge_vec); - return res_chunk; - } - - if (isEnoughSize(chunks_to_merge_vec)) - chunks_to_merge_vec.clear(); - - if (input_chunk) - chunks_to_merge_vec.push_back(input_chunk.clone()); - - if (isEnoughSize(chunks_to_merge_vec)) - { - Chunk res_chunk = convertToChunk(chunks_to_merge_vec); - return res_chunk; - } - return input_chunk; -} - -bool BalanceTransform::isEnoughSize(const std::vector & chunks) -{ - size_t rows = 0; - size_t bytes = 0; - - for (const Chunk & chunk : chunks) - { - rows += chunk.getNumRows(); - bytes += chunk.bytes(); - } - checkAndWaitMemoryAvailability(bytes); - - return isEnoughSize(rows, bytes); -} - -void BalanceTransform::checkAndWaitMemoryAvailability(size_t bytes) -{ - // bytes_used += bytes; - if (const auto hard_limit = memory_tracker->getHardLimit() != 0) - { - auto free_memory = hard_limit - memory_tracker->get(); - while (Int64(bytes) >= free_memory) - free_memory = hard_limit - memory_tracker->get(); - } -} - -bool BalanceTransform::isEnoughSize(const Chunk & chunk) -{ - return isEnoughSize(chunk.getNumRows(), chunk.bytes()); -} - - -bool BalanceTransform::isEnoughSize(size_t rows, size_t bytes) const -{ - return (!min_block_size_rows && !min_block_size_bytes) - || (min_block_size_rows && rows >= min_block_size_rows) - || (min_block_size_bytes && bytes >= min_block_size_bytes); -} - -NewSquashingBlockTransform::NewSquashingBlockTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_) - : min_block_size_rows(min_block_size_rows_) - , min_block_size_bytes(min_block_size_bytes_) -{ -} - -Block NewSquashingBlockTransform::add(Chunk && input_chunk) -{ - return addImpl(std::move(input_chunk)); -} - -const BlocksToSquash * getInfoFromChunkBlock(const Chunk & chunk) +const ChunksToSquash* NewSquashingTransform::getInfoFromChunk(const Chunk & chunk) { const auto& info = chunk.getChunkInfo(); - const auto * agg_info = typeid_cast(info.get()); + const auto * agg_info = typeid_cast(info.get()); return agg_info; } -Block NewSquashingBlockTransform::addImpl(Chunk && input_chunk) -{ - if (!input_chunk.hasChunkInfo()) - { - Block to_return; - std::swap(to_return, accumulated_block); - return to_return; - } - - const auto *info = getInfoFromChunkBlock(input_chunk); - for (auto & block : info->blocks) - append(std::move(block)); - - { - Block to_return; - std::swap(to_return, accumulated_block); - return to_return; - } -} - -void NewSquashingBlockTransform::append(Block && input_block) -{ - if (input_block.columns() == 0) - return; - if (!accumulated_block) - { - for (size_t i = 0; i < input_block.columns(); ++ i) - { - LOG_TRACE(getLogger("Squashing"), "data_type: {}", input_block.getDataTypeNames()[i]); - ColumnWithTypeAndName col = ColumnWithTypeAndName(input_block.getColumns()[i], input_block.getDataTypes()[i], input_block.getNames()[i]); - accumulated_block.insert(accumulated_block.columns(), col); - } - return; - } - - for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) - { - const auto source_column = input_block.getColumns()[i]; - - auto mutable_column = IColumn::mutate(std::move(accumulated_block.getByPosition(i).column)); - mutable_column->insertRangeFrom(*source_column, 0, source_column->size()); - accumulated_block.getByPosition(i).column = std::move(mutable_column); - } -} - -BalanceBlockTransform::BalanceBlockTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) +BalanceTransform::BalanceTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) , header(std::move(header_)) @@ -371,67 +201,70 @@ BalanceBlockTransform::BalanceBlockTransform(Block header_, size_t min_block_siz memory_tracker = memory_tracker_child->getParent(); } -Chunk BalanceBlockTransform::add(Block && input_block) +Chunk BalanceTransform::add(Block && input_block) { return addImpl(std::move(input_block)); } -Chunk BalanceBlockTransform::addImpl(Block && input_block) +Chunk BalanceTransform::addImpl(Block && input_block) { Chunk input_chunk(input_block.getColumns(), input_block.rows()); if (!input_chunk) { - Chunk res_chunk = convertToChunk(blocks_to_merge_vec); + Chunk res_chunk = convertToChunk(chunks_to_merge_vec); return res_chunk; } - if (isEnoughSize(blocks_to_merge_vec)) - blocks_to_merge_vec.clear(); + if (isEnoughSize(chunks_to_merge_vec)) + chunks_to_merge_vec.clear(); if (input_chunk) - blocks_to_merge_vec.push_back(std::move(input_block)); + chunks_to_merge_vec.push_back(std::move(input_chunk)); - if (isEnoughSize(blocks_to_merge_vec)) + if (isEnoughSize(chunks_to_merge_vec)) { - Chunk res_chunk = convertToChunk(blocks_to_merge_vec); + Chunk res_chunk = convertToChunk(chunks_to_merge_vec); return res_chunk; } return input_chunk; } -Chunk BalanceBlockTransform::convertToChunk(std::vector &blocks) +Chunk BalanceTransform::convertToChunk(std::vector &chunks) { - if (blocks.empty()) + if (chunks.empty()) return {}; - auto info = std::make_shared(); - for (auto &block : blocks) - info->blocks.push_back(std::move(block)); + auto info = std::make_shared(); + for (auto &chunk : chunks) + info->chunks.push_back(std::move(chunk)); - blocks.clear(); // we can remove this + chunks.clear(); // we can remove this return Chunk(header.cloneEmptyColumns(), 0, info); } -bool BalanceBlockTransform::isEnoughSize(const std::vector & blocks) +bool BalanceTransform::isEnoughSize(const std::vector & chunks) { size_t rows = 0; size_t bytes = 0; - for (const Block & block : blocks) + for (const Chunk & chunk : chunks) { - rows += block.rows(); - bytes += block.bytes(); + rows += chunk.getNumRows(); + bytes += chunk.bytes(); } + + if (!isEnoughSize(rows, bytes)) + return false; + checkAndWaitMemoryAvailability(bytes); - return isEnoughSize(rows, bytes); + return true; } -void BalanceBlockTransform::checkAndWaitMemoryAvailability(size_t bytes) +void BalanceTransform::checkAndWaitMemoryAvailability(size_t bytes) { - // bytes_used += bytes; if (const auto hard_limit = memory_tracker->getHardLimit() != 0) { auto free_memory = hard_limit - memory_tracker->get(); @@ -440,7 +273,7 @@ void BalanceBlockTransform::checkAndWaitMemoryAvailability(size_t bytes) } } -bool BalanceBlockTransform::isEnoughSize(size_t rows, size_t bytes) const +bool BalanceTransform::isEnoughSize(size_t rows, size_t bytes) const { return (!min_block_size_rows && !min_block_size_bytes) || (min_block_size_rows && rows >= min_block_size_rows) diff --git a/src/Interpreters/SquashingTransform.h b/src/Interpreters/SquashingTransform.h index 792a8c2efcf3..ce54c49e4417 100644 --- a/src/Interpreters/SquashingTransform.h +++ b/src/Interpreters/SquashingTransform.h @@ -14,7 +14,6 @@ namespace DB struct ChunksToSquash : public ChunkInfo { mutable std::vector chunks = {}; - DataTypes data_types = {}; }; /** Merging consecutive passed blocks to specified minimum size. @@ -60,7 +59,7 @@ class SquashingTransform class NewSquashingTransform { public: - NewSquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_); + NewSquashingTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); Block add(Chunk && input_chunk); @@ -69,12 +68,13 @@ class NewSquashingTransform size_t min_block_size_bytes; Block accumulated_block; + const Block header; - template - Block addImpl(ReferenceType chunk); + Block addImpl(Chunk && chunk); - template - void append(ReferenceType input_chunk, DataTypes data_types); + const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); + + void append(Chunk && input_chunk); bool isEnoughSize(const Block & block); bool isEnoughSize(size_t rows, size_t bytes) const; @@ -96,78 +96,17 @@ class BalanceTransform size_t min_block_size_rows; size_t min_block_size_bytes; - Chunk accumulated_block; const Block header; - template - Chunk addImpl(ReferenceType input_block); + Chunk addImpl(Block && input_block); - bool isEnoughSize(const Chunk & chunk); bool isEnoughSize(const std::vector & chunks); bool isEnoughSize(size_t rows, size_t bytes) const; void checkAndWaitMemoryAvailability(size_t bytes); - DataTypes data_types = {}; MemoryTracker * memory_tracker; Chunk convertToChunk(std::vector &chunks); }; -class NewSquashingBlockTransform -{ -public: - NewSquashingBlockTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_); - - Block add(Chunk && input_chunk); - -private: - size_t min_block_size_rows; - size_t min_block_size_bytes; - - Block accumulated_block; - - Block addImpl(Chunk && chunk); - - void append(Block && input_block); - - bool isEnoughSize(const Block & block); - bool isEnoughSize(size_t rows, size_t bytes) const; -}; - -struct BlocksToSquash : public ChunkInfo -{ - mutable std::vector blocks = {}; -}; - -class BalanceBlockTransform -{ -public: - BalanceBlockTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); - - Chunk add(Block && input_block); - bool isDataLeft() - { - return !blocks_to_merge_vec.empty(); - } - -private: - std::vector blocks_to_merge_vec = {}; - size_t min_block_size_rows; - size_t min_block_size_bytes; - - Block accumulated_block; - const Block header; - - // template - Chunk addImpl(Block && input_block); - - bool isEnoughSize(const std::vector & blocks); - bool isEnoughSize(size_t rows, size_t bytes) const; - void checkAndWaitMemoryAvailability(size_t bytes); - - MemoryTracker * memory_tracker; - - Chunk convertToChunk(std::vector &blocks); -}; - } diff --git a/src/Processors/Transforms/BalancingTransform.cpp b/src/Processors/Transforms/BalancingTransform.cpp index deb1abdb2fe1..7a9edbe5d865 100644 --- a/src/Processors/Transforms/BalancingTransform.cpp +++ b/src/Processors/Transforms/BalancingTransform.cpp @@ -1,7 +1,5 @@ #include -#include -#include -#include "Processors/IProcessor.h" +#include namespace DB { @@ -13,7 +11,6 @@ BalancingChunksTransform::BalancingChunksTransform(const Block & header, size_t IProcessor::Status BalancingChunksTransform::prepare() { - LOG_TRACE(getLogger("balancingProcessor"), "prepare"); Status status = Status::Ready; while (status == Status::Ready) @@ -27,7 +24,6 @@ IProcessor::Status BalancingChunksTransform::prepare() IProcessor::Status BalancingChunksTransform::prepareConsume() { - LOG_TRACE(getLogger("balancingProcessor"), "prepareConsume"); finished = false; while (!chunk.hasChunkInfo()) { @@ -75,7 +71,6 @@ IProcessor::Status BalancingChunksTransform::prepareConsume() was_output_processed.assign(outputs.size(), false); if (chunk.hasChunkInfo()) { - LOG_TRACE(getLogger("balancingProcessor"), "hasData"); has_data = true; return Status::Ready; } @@ -97,12 +92,10 @@ void BalancingChunksTransform::transform(Chunk & chunk_) Chunk res_chunk = balance.add({}); std::swap(res_chunk, chunk_); } - LOG_TRACE(getLogger("balancing"), "{}, BalancingTransform: struct of output chunk: {}, hasInfo: {}", reinterpret_cast(this), chunk_.dumpStructure(), chunk.hasChunkInfo()); } IProcessor::Status BalancingChunksTransform::prepareSend() { - LOG_TRACE(getLogger("balancingProcessor"), "prepareGenerate {}", chunk.dumpStructure()); bool all_outputs_processed = true; size_t chunk_number = 0; @@ -129,7 +122,6 @@ IProcessor::Status BalancingChunksTransform::prepareSend() continue; } - LOG_TRACE(getLogger("balancingProcessor"), "chunk struct: {}", chunk.dumpStructure()); output.push(std::move(chunk)); was_processed = true; break; diff --git a/src/Processors/Transforms/BalancingTransform.h b/src/Processors/Transforms/BalancingTransform.h index a50b38c773f8..1b1d3ec6295b 100644 --- a/src/Processors/Transforms/BalancingTransform.h +++ b/src/Processors/Transforms/BalancingTransform.h @@ -31,7 +31,7 @@ class BalancingChunksTransform : public IProcessor size_t CalculateBlockSize(const Block & block); Chunk chunk; size_t max_memory_usage; - BalanceBlockTransform balance; + BalanceTransform balance; bool has_data = false; std::vector was_output_processed; diff --git a/src/Processors/Transforms/SquashingChunksTransform.cpp b/src/Processors/Transforms/SquashingChunksTransform.cpp index 5b68df6b6c60..ec226a56548a 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.cpp +++ b/src/Processors/Transforms/SquashingChunksTransform.cpp @@ -7,13 +7,12 @@ namespace DB SquashingChunksTransform::SquashingChunksTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(min_block_size_rows, min_block_size_bytes) + , squashing(header, min_block_size_rows, min_block_size_bytes) { } void SquashingChunksTransform::onConsume(Chunk chunk) { - LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: !finished, hasInfo: {}", reinterpret_cast(this), chunk.hasChunkInfo()); if (auto block = squashing.add(std::move(chunk))) cur_chunk.setColumns(block.getColumns(), block.rows()); } @@ -29,9 +28,7 @@ SquashingChunksTransform::GenerateResult SquashingChunksTransform::onGenerate() void SquashingChunksTransform::onFinish() { auto block = squashing.add({}); - LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: finished, structure of block: {}", reinterpret_cast(this), block.dumpStructure()); finish_chunk.setColumns(block.getColumns(), block.rows()); - LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: finished, hasInfo: {}", reinterpret_cast(this), finish_chunk.hasChunkInfo()); } void SquashingChunksTransform::work() @@ -52,7 +49,7 @@ void SquashingChunksTransform::work() } SimpleSquashingChunksTransform::SimpleSquashingChunksTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, [[maybe_unused]] bool skip_empty_chunks_) + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ISimpleTransform(header, header, false), squashing(min_block_size_rows, min_block_size_bytes) { } @@ -61,13 +58,11 @@ void SimpleSquashingChunksTransform::transform(Chunk & chunk) { if (!finished) { - LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: !finished, hasInfo: {}", reinterpret_cast(this), chunk.hasChunkInfo()); - if (auto block = squashing.add(std::move(chunk))) + if (auto block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()))) chunk.setColumns(block.getColumns(), block.rows()); } else { - LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: finished, hasInfo: {}", reinterpret_cast(this), chunk.hasChunkInfo()); auto block = squashing.add({}); chunk.setColumns(block.getColumns(), block.rows()); } diff --git a/src/Processors/Transforms/SquashingChunksTransform.h b/src/Processors/Transforms/SquashingChunksTransform.h index 5c7ad12889f1..4bcf2216182f 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -32,7 +32,7 @@ class SquashingChunksTransform : public ExceptionKeepingTransform class SimpleSquashingChunksTransform : public ISimpleTransform { public: - explicit SimpleSquashingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, bool skip_empty_chunks_ = true); + explicit SimpleSquashingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); String getName() const override { return "SimpleSquashingTransform"; } @@ -42,7 +42,7 @@ class SimpleSquashingChunksTransform : public ISimpleTransform IProcessor::Status prepare() override; private: - NewSquashingTransform squashing; + SquashingTransform squashing; bool finished = false; }; From ba8af1fcd87c1d0bd419249cab37ccd7a0add564 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 12 Apr 2024 16:21:41 +0000 Subject: [PATCH 026/254] fix for projections --- src/Interpreters/InterpreterInsertQuery.cpp | 4 ++-- src/Processors/Transforms/BalancingTransform.cpp | 4 ++-- src/Processors/Transforms/BalancingTransform.h | 3 +-- src/Storages/ProjectionsDescription.cpp | 3 +++ 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 31fef267afce..412fba0c7630 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -609,7 +609,7 @@ BlockIO InterpreterInsertQuery::execute() header, table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, - settings.max_memory_usage, presink_chains.size())); + presink_chains.size())); pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { @@ -680,7 +680,7 @@ BlockIO InterpreterInsertQuery::execute() chain.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, - settings.max_memory_usage, true); + true); chain.addSource(std::move(balancing)); } diff --git a/src/Processors/Transforms/BalancingTransform.cpp b/src/Processors/Transforms/BalancingTransform.cpp index 7a9edbe5d865..dab73eae2c03 100644 --- a/src/Processors/Transforms/BalancingTransform.cpp +++ b/src/Processors/Transforms/BalancingTransform.cpp @@ -4,8 +4,8 @@ namespace DB { -BalancingChunksTransform::BalancingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage_, size_t num_ports) - : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), max_memory_usage(max_memory_usage_), balance(header, min_block_size_rows, min_block_size_bytes) +BalancingChunksTransform::BalancingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), balance(header, min_block_size_rows, min_block_size_bytes) { } diff --git a/src/Processors/Transforms/BalancingTransform.h b/src/Processors/Transforms/BalancingTransform.h index 1b1d3ec6295b..a8a8bc5cfab9 100644 --- a/src/Processors/Transforms/BalancingTransform.h +++ b/src/Processors/Transforms/BalancingTransform.h @@ -12,7 +12,7 @@ class BalancingChunksTransform : public IProcessor { public: BalancingChunksTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t max_memory_usage_, size_t num_ports); + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports); String getName() const override { return "BalancingChunksTransform"; } @@ -30,7 +30,6 @@ class BalancingChunksTransform : public IProcessor private: size_t CalculateBlockSize(const Block & block); Chunk chunk; - size_t max_memory_usage; BalanceTransform balance; bool has_data = false; std::vector was_output_processed; diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 08ebe3a10d01..55639641c95b 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -20,6 +20,8 @@ #include #include #include +#include "Interpreters/SquashingTransform.h" +#include "Processors/Transforms/BalancingTransform.h" namespace DB @@ -310,6 +312,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) builder.resize(1); // Generate aggregated blocks with rows less or equal than the original block. // There should be only one output block after this transformation. + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, true)); builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); From 24432f875abd5b5f77050f986e999bc15fda595d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 24 Apr 2024 16:15:47 +0200 Subject: [PATCH 027/254] empty commit From 0065a4cc6e8ac7eff3e72765f5ae0a7eb593ed2d Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 24 Apr 2024 14:55:01 +0000 Subject: [PATCH 028/254] fixing bugs by adding old transform to some parts --- src/Interpreters/InterpreterInsertQuery.cpp | 4 +- .../Transforms/SquashingChunksTransform.cpp | 50 ++++++++++++++++++- .../Transforms/SquashingChunksTransform.h | 23 ++++++++- .../Transforms/buildPushingToViewsChain.cpp | 9 +++- src/Storages/ProjectionsDescription.cpp | 2 +- 5 files changed, 81 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 412fba0c7630..06ae92f7f0b6 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -613,7 +613,7 @@ BlockIO InterpreterInsertQuery::execute() pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { - return std::make_shared( + return std::make_shared( in_header, table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); @@ -669,7 +669,7 @@ BlockIO InterpreterInsertQuery::execute() { bool table_prefers_large_blocks = table->prefersLargeBlocks(); - auto squashing = std::make_shared( + auto squashing = std::make_shared( chain.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); diff --git a/src/Processors/Transforms/SquashingChunksTransform.cpp b/src/Processors/Transforms/SquashingChunksTransform.cpp index ec226a56548a..00e430933beb 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.cpp +++ b/src/Processors/Transforms/SquashingChunksTransform.cpp @@ -7,13 +7,13 @@ namespace DB SquashingChunksTransform::SquashingChunksTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(header, min_block_size_rows, min_block_size_bytes) + , squashing(min_block_size_rows, min_block_size_bytes) { } void SquashingChunksTransform::onConsume(Chunk chunk) { - if (auto block = squashing.add(std::move(chunk))) + if (auto block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()))) cur_chunk.setColumns(block.getColumns(), block.rows()); } @@ -29,6 +29,7 @@ void SquashingChunksTransform::onFinish() { auto block = squashing.add({}); finish_chunk.setColumns(block.getColumns(), block.rows()); + LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: finished, hasInfo: {}", reinterpret_cast(this), finish_chunk.hasChunkInfo()); } void SquashingChunksTransform::work() @@ -48,6 +49,51 @@ void SquashingChunksTransform::work() } } +NewSquashingChunksTransform::NewSquashingChunksTransform( + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) + : ExceptionKeepingTransform(header, header, false) + , squashing(header, min_block_size_rows, min_block_size_bytes) +{ +} + +void NewSquashingChunksTransform::onConsume(Chunk chunk) +{ + if (auto block = squashing.add(std::move(chunk))) + cur_chunk.setColumns(block.getColumns(), block.rows()); +} + +NewSquashingChunksTransform::GenerateResult NewSquashingChunksTransform::onGenerate() +{ + GenerateResult res; + res.chunk = std::move(cur_chunk); + res.is_done = true; + return res; +} + +void NewSquashingChunksTransform::onFinish() +{ + auto block = squashing.add({}); + finish_chunk.setColumns(block.getColumns(), block.rows()); + LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: finished, hasInfo: {}", reinterpret_cast(this), finish_chunk.hasChunkInfo()); +} + +void NewSquashingChunksTransform::work() +{ + if (stage == Stage::Exception) + { + data.chunk.clear(); + ready_input = false; + return; + } + + ExceptionKeepingTransform::work(); + if (finish_chunk) + { + data.chunk = std::move(finish_chunk); + ready_output = true; + } +} + SimpleSquashingChunksTransform::SimpleSquashingChunksTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ISimpleTransform(header, header, false), squashing(min_block_size_rows, min_block_size_bytes) diff --git a/src/Processors/Transforms/SquashingChunksTransform.h b/src/Processors/Transforms/SquashingChunksTransform.h index 4bcf2216182f..1db3d46371c5 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -7,6 +7,27 @@ namespace DB { +class NewSquashingChunksTransform : public ExceptionKeepingTransform +{ +public: + explicit NewSquashingChunksTransform( + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); + + String getName() const override { return "SquashingTransform"; } + + void work() override; + +protected: + void onConsume(Chunk chunk) override; + GenerateResult onGenerate() override; + void onFinish() override; + +private: + NewSquashingTransform squashing; + Chunk cur_chunk; + Chunk finish_chunk; +}; + class SquashingChunksTransform : public ExceptionKeepingTransform { public: @@ -23,7 +44,7 @@ class SquashingChunksTransform : public ExceptionKeepingTransform void onFinish() override; private: - NewSquashingTransform squashing; + SquashingTransform squashing; Chunk cur_chunk; Chunk finish_chunk; }; diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index dd07d043599b..5c16cdbe84c9 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -368,10 +369,16 @@ std::optional generateViewChain( bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); const auto & settings = insert_context->getSettingsRef(); - out.addSource(std::make_shared( + out.addSource(std::make_shared( out.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + + out.addSource(std::make_shared( + out.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, + true)); } auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 55639641c95b..155c32e30ae5 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -313,7 +313,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) // Generate aggregated blocks with rows less or equal than the original block. // There should be only one output block after this transformation. builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, true)); - builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); From 7ce5e5a38c4e0eb6e16695aba4708106330a66c9 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 24 Apr 2024 15:36:11 +0000 Subject: [PATCH 029/254] SquashingChunksTranform -> (BalancingChunksTransform && SquashingChunksTranform) || SimpleSquashingChunksTranform --- programs/copier/Internals.cpp | 3 +- src/Interpreters/InterpreterInsertQuery.cpp | 4 +- .../Transforms/SquashingChunksTransform.cpp | 50 +------------------ .../Transforms/SquashingChunksTransform.h | 23 +-------- .../Transforms/buildPushingToViewsChain.cpp | 4 +- src/Storages/LiveView/StorageLiveView.cpp | 2 +- src/Storages/ProjectionsDescription.cpp | 4 +- src/Storages/WindowView/StorageWindowView.cpp | 4 +- 8 files changed, 14 insertions(+), 80 deletions(-) diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index 0cfff7e3f6c0..bb672f5cfbd1 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -62,7 +63,7 @@ std::shared_ptr createASTStorageDistributed( Block getBlockWithAllStreamData(QueryPipelineBuilder builder) { - builder.addTransform(std::make_shared( + builder.addTransform(std::make_shared( builder.getHeader(), std::numeric_limits::max(), std::numeric_limits::max())); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 06ae92f7f0b6..412fba0c7630 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -613,7 +613,7 @@ BlockIO InterpreterInsertQuery::execute() pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { - return std::make_shared( + return std::make_shared( in_header, table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); @@ -669,7 +669,7 @@ BlockIO InterpreterInsertQuery::execute() { bool table_prefers_large_blocks = table->prefersLargeBlocks(); - auto squashing = std::make_shared( + auto squashing = std::make_shared( chain.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); diff --git a/src/Processors/Transforms/SquashingChunksTransform.cpp b/src/Processors/Transforms/SquashingChunksTransform.cpp index 00e430933beb..ec226a56548a 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.cpp +++ b/src/Processors/Transforms/SquashingChunksTransform.cpp @@ -7,13 +7,13 @@ namespace DB SquashingChunksTransform::SquashingChunksTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(min_block_size_rows, min_block_size_bytes) + , squashing(header, min_block_size_rows, min_block_size_bytes) { } void SquashingChunksTransform::onConsume(Chunk chunk) { - if (auto block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()))) + if (auto block = squashing.add(std::move(chunk))) cur_chunk.setColumns(block.getColumns(), block.rows()); } @@ -29,7 +29,6 @@ void SquashingChunksTransform::onFinish() { auto block = squashing.add({}); finish_chunk.setColumns(block.getColumns(), block.rows()); - LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: finished, hasInfo: {}", reinterpret_cast(this), finish_chunk.hasChunkInfo()); } void SquashingChunksTransform::work() @@ -49,51 +48,6 @@ void SquashingChunksTransform::work() } } -NewSquashingChunksTransform::NewSquashingChunksTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) - : ExceptionKeepingTransform(header, header, false) - , squashing(header, min_block_size_rows, min_block_size_bytes) -{ -} - -void NewSquashingChunksTransform::onConsume(Chunk chunk) -{ - if (auto block = squashing.add(std::move(chunk))) - cur_chunk.setColumns(block.getColumns(), block.rows()); -} - -NewSquashingChunksTransform::GenerateResult NewSquashingChunksTransform::onGenerate() -{ - GenerateResult res; - res.chunk = std::move(cur_chunk); - res.is_done = true; - return res; -} - -void NewSquashingChunksTransform::onFinish() -{ - auto block = squashing.add({}); - finish_chunk.setColumns(block.getColumns(), block.rows()); - LOG_TRACE(getLogger("squashing"), "{}, SquashingTransform: finished, hasInfo: {}", reinterpret_cast(this), finish_chunk.hasChunkInfo()); -} - -void NewSquashingChunksTransform::work() -{ - if (stage == Stage::Exception) - { - data.chunk.clear(); - ready_input = false; - return; - } - - ExceptionKeepingTransform::work(); - if (finish_chunk) - { - data.chunk = std::move(finish_chunk); - ready_output = true; - } -} - SimpleSquashingChunksTransform::SimpleSquashingChunksTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ISimpleTransform(header, header, false), squashing(min_block_size_rows, min_block_size_bytes) diff --git a/src/Processors/Transforms/SquashingChunksTransform.h b/src/Processors/Transforms/SquashingChunksTransform.h index 1db3d46371c5..4bcf2216182f 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -7,27 +7,6 @@ namespace DB { -class NewSquashingChunksTransform : public ExceptionKeepingTransform -{ -public: - explicit NewSquashingChunksTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); - - String getName() const override { return "SquashingTransform"; } - - void work() override; - -protected: - void onConsume(Chunk chunk) override; - GenerateResult onGenerate() override; - void onFinish() override; - -private: - NewSquashingTransform squashing; - Chunk cur_chunk; - Chunk finish_chunk; -}; - class SquashingChunksTransform : public ExceptionKeepingTransform { public: @@ -44,7 +23,7 @@ class SquashingChunksTransform : public ExceptionKeepingTransform void onFinish() override; private: - SquashingTransform squashing; + NewSquashingTransform squashing; Chunk cur_chunk; Chunk finish_chunk; }; diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 5c16cdbe84c9..da7f10990e55 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -369,7 +369,7 @@ std::optional generateViewChain( bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); const auto & settings = insert_context->getSettingsRef(); - out.addSource(std::make_shared( + out.addSource(std::make_shared( out.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); @@ -625,7 +625,7 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat /// Squashing is needed here because the materialized view query can generate a lot of blocks /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY /// and two-level aggregation is triggered). - pipeline.addTransform(std::make_shared( + pipeline.addTransform(std::make_shared( pipeline.getHeader(), context->getSettingsRef().min_insert_block_size_rows, context->getSettingsRef().min_insert_block_size_bytes)); diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 2f011567b900..b019660cf5e1 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -667,7 +667,7 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes) /// and two-level aggregation is triggered). builder.addSimpleTransform([&](const Block & cur_header) { - return std::make_shared( + return std::make_shared( cur_header, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes); diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 155c32e30ae5..14713d151d73 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -16,12 +16,12 @@ #include #include #include +#include #include #include #include #include #include "Interpreters/SquashingTransform.h" -#include "Processors/Transforms/BalancingTransform.h" namespace DB @@ -313,7 +313,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) // Generate aggregated blocks with rows less or equal than the original block. // There should be only one output block after this transformation. builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, true)); - builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index b1984a947c8c..a3bc09f8d727 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -610,7 +610,7 @@ std::pair StorageWindowView::getNewBlocks(UInt32 watermark) }); builder.addSimpleTransform([&](const Block & current_header) { - return std::make_shared( + return std::make_shared( current_header, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes); @@ -1498,7 +1498,7 @@ void StorageWindowView::writeIntoWindowView( builder = select_block.buildQueryPipeline(); builder.addSimpleTransform([&](const Block & current_header) { - return std::make_shared( + return std::make_shared( current_header, local_context->getSettingsRef().min_insert_block_size_rows, local_context->getSettingsRef().min_insert_block_size_bytes); From 91b189ab534936fafdaf522b71d02fd4ba3e60cc Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 24 Apr 2024 17:40:15 +0200 Subject: [PATCH 030/254] Update Internals.cpp --- programs/copier/Internals.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index bb672f5cfbd1..650d606295f2 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include #include From 2eba133b536a7f22ba100792e6c594d42afb9278 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 24 Apr 2024 15:51:23 +0000 Subject: [PATCH 031/254] implement SquashingChunksTransformForBalancing --- programs/copier/Internals.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 4 +- .../Transforms/SquashingChunksTransform.cpp | 48 ++++++++++++++++++- .../Transforms/SquashingChunksTransform.h | 21 ++++++++ .../Transforms/buildPushingToViewsChain.cpp | 4 +- src/Storages/LiveView/StorageLiveView.cpp | 2 +- src/Storages/ProjectionsDescription.cpp | 3 +- src/Storages/WindowView/StorageWindowView.cpp | 4 +- 8 files changed, 76 insertions(+), 12 deletions(-) diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index 650d606295f2..0cfff7e3f6c0 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -62,7 +62,7 @@ std::shared_ptr createASTStorageDistributed( Block getBlockWithAllStreamData(QueryPipelineBuilder builder) { - builder.addTransform(std::make_shared( + builder.addTransform(std::make_shared( builder.getHeader(), std::numeric_limits::max(), std::numeric_limits::max())); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 412fba0c7630..50f7bc91d901 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -613,7 +613,7 @@ BlockIO InterpreterInsertQuery::execute() pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { - return std::make_shared( + return std::make_shared( in_header, table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); @@ -669,7 +669,7 @@ BlockIO InterpreterInsertQuery::execute() { bool table_prefers_large_blocks = table->prefersLargeBlocks(); - auto squashing = std::make_shared( + auto squashing = std::make_shared( chain.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); diff --git a/src/Processors/Transforms/SquashingChunksTransform.cpp b/src/Processors/Transforms/SquashingChunksTransform.cpp index ec226a56548a..62b87061344c 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.cpp +++ b/src/Processors/Transforms/SquashingChunksTransform.cpp @@ -7,13 +7,13 @@ namespace DB SquashingChunksTransform::SquashingChunksTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(header, min_block_size_rows, min_block_size_bytes) + , squashing(min_block_size_rows, min_block_size_bytes) { } void SquashingChunksTransform::onConsume(Chunk chunk) { - if (auto block = squashing.add(std::move(chunk))) + if (auto block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()))) cur_chunk.setColumns(block.getColumns(), block.rows()); } @@ -48,6 +48,50 @@ void SquashingChunksTransform::work() } } +SquashingChunksTransformForBalancing::SquashingChunksTransformForBalancing( + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) + : ExceptionKeepingTransform(header, header, false) + , squashing(header, min_block_size_rows, min_block_size_bytes) +{ +} + +void SquashingChunksTransformForBalancing::onConsume(Chunk chunk) +{ + if (auto block = squashing.add(std::move(chunk))) + cur_chunk.setColumns(block.getColumns(), block.rows()); +} + +SquashingChunksTransformForBalancing::GenerateResult SquashingChunksTransformForBalancing::onGenerate() +{ + GenerateResult res; + res.chunk = std::move(cur_chunk); + res.is_done = true; + return res; +} + +void SquashingChunksTransformForBalancing::onFinish() +{ + auto block = squashing.add({}); + finish_chunk.setColumns(block.getColumns(), block.rows()); +} + +void SquashingChunksTransformForBalancing::work() +{ + if (stage == Stage::Exception) + { + data.chunk.clear(); + ready_input = false; + return; + } + + ExceptionKeepingTransform::work(); + if (finish_chunk) + { + data.chunk = std::move(finish_chunk); + ready_output = true; + } +} + SimpleSquashingChunksTransform::SimpleSquashingChunksTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ISimpleTransform(header, header, false), squashing(min_block_size_rows, min_block_size_bytes) diff --git a/src/Processors/Transforms/SquashingChunksTransform.h b/src/Processors/Transforms/SquashingChunksTransform.h index 4bcf2216182f..7c7948d1af91 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -22,6 +22,27 @@ class SquashingChunksTransform : public ExceptionKeepingTransform GenerateResult onGenerate() override; void onFinish() override; +private: + SquashingTransform squashing; + Chunk cur_chunk; + Chunk finish_chunk; +}; + +class SquashingChunksTransformForBalancing : public ExceptionKeepingTransform +{ +public: + explicit SquashingChunksTransformForBalancing( + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); + + String getName() const override { return "SquashingTransform"; } + + void work() override; + +protected: + void onConsume(Chunk chunk) override; + GenerateResult onGenerate() override; + void onFinish() override; + private: NewSquashingTransform squashing; Chunk cur_chunk; diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index da7f10990e55..e05f36438744 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -369,7 +369,7 @@ std::optional generateViewChain( bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); const auto & settings = insert_context->getSettingsRef(); - out.addSource(std::make_shared( + out.addSource(std::make_shared( out.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); @@ -625,7 +625,7 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat /// Squashing is needed here because the materialized view query can generate a lot of blocks /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY /// and two-level aggregation is triggered). - pipeline.addTransform(std::make_shared( + pipeline.addTransform(std::make_shared( pipeline.getHeader(), context->getSettingsRef().min_insert_block_size_rows, context->getSettingsRef().min_insert_block_size_bytes)); diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index b019660cf5e1..2f011567b900 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -667,7 +667,7 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes) /// and two-level aggregation is triggered). builder.addSimpleTransform([&](const Block & cur_header) { - return std::make_shared( + return std::make_shared( cur_header, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes); diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 14713d151d73..f6922efc2720 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -21,7 +21,6 @@ #include #include #include -#include "Interpreters/SquashingTransform.h" namespace DB @@ -313,7 +312,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) // Generate aggregated blocks with rows less or equal than the original block. // There should be only one output block after this transformation. builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, true)); - builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index a3bc09f8d727..b1984a947c8c 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -610,7 +610,7 @@ std::pair StorageWindowView::getNewBlocks(UInt32 watermark) }); builder.addSimpleTransform([&](const Block & current_header) { - return std::make_shared( + return std::make_shared( current_header, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes); @@ -1498,7 +1498,7 @@ void StorageWindowView::writeIntoWindowView( builder = select_block.buildQueryPipeline(); builder.addSimpleTransform([&](const Block & current_header) { - return std::make_shared( + return std::make_shared( current_header, local_context->getSettingsRef().min_insert_block_size_rows, local_context->getSettingsRef().min_insert_block_size_bytes); From 6c8d7b866142842dd4b1a508c2b5af12b41d1f32 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 24 Apr 2024 17:04:26 +0000 Subject: [PATCH 032/254] mv balancing fix --- src/Processors/Transforms/buildPushingToViewsChain.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index e05f36438744..804af037c585 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -625,7 +625,12 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat /// Squashing is needed here because the materialized view query can generate a lot of blocks /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY /// and two-level aggregation is triggered). - pipeline.addTransform(std::make_shared( + pipeline.addTransform(std::make_shared( + pipeline.getHeader(), + context->getSettingsRef().min_insert_block_size_rows, + context->getSettingsRef().min_insert_block_size_bytes, + true)); + pipeline.addTransform(std::make_shared( pipeline.getHeader(), context->getSettingsRef().min_insert_block_size_rows, context->getSettingsRef().min_insert_block_size_bytes)); From 3501348e1fe4c6fe95bf3c9670be31e65f612458 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 29 Apr 2024 18:32:16 +0200 Subject: [PATCH 033/254] empty commit From 0bc664ac5bd0104219e061660cb6cd1cb0698b7c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 29 Apr 2024 18:21:19 +0000 Subject: [PATCH 034/254] added correct number of input ports for balancing --- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Processors/Transforms/buildPushingToViewsChain.cpp | 4 ++-- src/Storages/ProjectionsDescription.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 50f7bc91d901..a30616b0f3e3 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -680,7 +680,7 @@ BlockIO InterpreterInsertQuery::execute() chain.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, - true); + presink_chains.size()); chain.addSource(std::move(balancing)); } diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 804af037c585..3b4304dc39b7 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -378,7 +378,7 @@ std::optional generateViewChain( out.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, - true)); + out.getNumThreads())); } auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); @@ -629,7 +629,7 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat pipeline.getHeader(), context->getSettingsRef().min_insert_block_size_rows, context->getSettingsRef().min_insert_block_size_bytes, - true)); + pipeline.getNumStreams())); pipeline.addTransform(std::make_shared( pipeline.getHeader(), context->getSettingsRef().min_insert_block_size_rows, diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index f6922efc2720..731ac04a8c80 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -311,7 +311,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) builder.resize(1); // Generate aggregated blocks with rows less or equal than the original block. // There should be only one output block after this transformation. - builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, true)); + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, builder.getNumStreams())); builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); From 7dc4f1778bd8690f62e378ba3c26a013e6ae208b Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 6 May 2024 19:44:59 +0000 Subject: [PATCH 035/254] fixes + remove memory tracker --- src/Interpreters/SquashingTransform.cpp | 20 +------------------ src/Interpreters/SquashingTransform.h | 4 ---- .../Transforms/buildPushingToViewsChain.cpp | 2 +- 3 files changed, 2 insertions(+), 24 deletions(-) diff --git a/src/Interpreters/SquashingTransform.cpp b/src/Interpreters/SquashingTransform.cpp index d4370b45119f..dc2ce69ed2f1 100644 --- a/src/Interpreters/SquashingTransform.cpp +++ b/src/Interpreters/SquashingTransform.cpp @@ -196,9 +196,6 @@ BalanceTransform::BalanceTransform(Block header_, size_t min_block_size_rows_, s , min_block_size_bytes(min_block_size_bytes_) , header(std::move(header_)) { - // Use query-level memory tracker - if (auto * memory_tracker_child = CurrentThread::getMemoryTracker()) - memory_tracker = memory_tracker_child->getParent(); } Chunk BalanceTransform::add(Block && input_block) @@ -255,22 +252,7 @@ bool BalanceTransform::isEnoughSize(const std::vector & chunks) bytes += chunk.bytes(); } - if (!isEnoughSize(rows, bytes)) - return false; - - checkAndWaitMemoryAvailability(bytes); - - return true; -} - -void BalanceTransform::checkAndWaitMemoryAvailability(size_t bytes) -{ - if (const auto hard_limit = memory_tracker->getHardLimit() != 0) - { - auto free_memory = hard_limit - memory_tracker->get(); - while (Int64(bytes) >= free_memory) - free_memory = hard_limit - memory_tracker->get(); - } + return isEnoughSize(rows, bytes); } bool BalanceTransform::isEnoughSize(size_t rows, size_t bytes) const diff --git a/src/Interpreters/SquashingTransform.h b/src/Interpreters/SquashingTransform.h index ce54c49e4417..95e8c2a8fa12 100644 --- a/src/Interpreters/SquashingTransform.h +++ b/src/Interpreters/SquashingTransform.h @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB @@ -102,9 +101,6 @@ class BalanceTransform bool isEnoughSize(const std::vector & chunks); bool isEnoughSize(size_t rows, size_t bytes) const; - void checkAndWaitMemoryAvailability(size_t bytes); - - MemoryTracker * memory_tracker; Chunk convertToChunk(std::vector &chunks); }; diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 3b4304dc39b7..93cfc8f6d10f 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -378,7 +378,7 @@ std::optional generateViewChain( out.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, - out.getNumThreads())); + 1)); // Chain requires a single input } auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); From 0b939044087f9494cafab57ac1377ed58ed95971 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 8 May 2024 12:16:42 +0000 Subject: [PATCH 036/254] fix problems with ports --- .../Transforms/BalancingTransform.cpp | 68 ++++++++++++------- 1 file changed, 45 insertions(+), 23 deletions(-) diff --git a/src/Processors/Transforms/BalancingTransform.cpp b/src/Processors/Transforms/BalancingTransform.cpp index dab73eae2c03..def124070198 100644 --- a/src/Processors/Transforms/BalancingTransform.cpp +++ b/src/Processors/Transforms/BalancingTransform.cpp @@ -25,41 +25,63 @@ IProcessor::Status BalancingChunksTransform::prepare() IProcessor::Status BalancingChunksTransform::prepareConsume() { finished = false; - while (!chunk.hasChunkInfo()) + bool all_finished = true; + for (auto & output : outputs) { - for (auto & input : inputs) - { - bool all_finished = true; - for (auto & output : outputs) - { - if (output.isFinished()) - continue; + if (output.isFinished()) + continue; - all_finished = false; - } + all_finished = false; + } - if (all_finished) - { - input.close(); - return Status::Finished; - } + if (all_finished) /// If all outputs are closed, we close inputs (just in case) + { + for (auto & in : inputs) + in.close(); + return Status::Finished; + } - if (input.isFinished() && !balance.isDataLeft()) - { - for (auto & output : outputs) - output.finish(); + all_finished = true; + for (auto & input : inputs) + { + if (input.isFinished()) + continue; - return Status::Finished; - } + all_finished = false; + } + + if (all_finished) /// If all inputs are closed, we check if we have data in balancing + { + if (balance.isDataLeft()) /// If we have data in balancing, we process this data + { + finished = false; + transform(chunk); + has_data = true; + } + else /// If we don't have data, We send FINISHED + { + for (auto & output : outputs) + output.finish(); + + return Status::Finished; + } + } + + while (!chunk.hasChunkInfo()) + { + for (auto & input : inputs) + { + if (input.isFinished()) + continue; input.setNeeded(); if (!input.hasData()) { - finished = true; if (!balance.isDataLeft()) return Status::NeedData; else { + finished = true; transform(chunk); has_data = true; return Status::Ready; @@ -68,7 +90,7 @@ IProcessor::Status BalancingChunksTransform::prepareConsume() chunk = input.pull(); transform(chunk); - was_output_processed.assign(outputs.size(), false); + was_output_processed.assign(inputs.size(), false); if (chunk.hasChunkInfo()) { has_data = true; From e1ed0af3d2598f6511a8d804ed52f0822d06b5b5 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 8 May 2024 14:28:09 +0000 Subject: [PATCH 037/254] Added pointer check, Chunk as argument to PlanSquashingTransform, fully refactored names of functions --- programs/copier/Internals.cpp | 4 +- src/Interpreters/InterpreterInsertQuery.cpp | 12 +- .../{SquashingTransform.cpp => Squashing.cpp} | 45 +++---- .../{SquashingTransform.h => Squashing.h} | 16 +-- .../Transforms/AggregatingTransform.cpp | 4 +- .../Transforms/ApplySquashingTransform.h | 63 +++++++++ ...ansform.cpp => PlanSquashingTransform.cpp} | 16 +-- ...ngTransform.h => PlanSquashingTransform.h} | 10 +- .../Transforms/SquashingChunksTransform.cpp | 124 ------------------ .../Transforms/SquashingChunksTransform.h | 70 ---------- .../Transforms/SquashingTransform.cpp | 80 +++++++++++ .../Transforms/SquashingTransform.h | 50 +++++++ .../Transforms/buildPushingToViewsChain.cpp | 12 +- src/Server/TCPHandler.cpp | 8 +- src/Storages/LiveView/StorageLiveView.cpp | 4 +- src/Storages/MergeTree/MutateTask.cpp | 4 +- src/Storages/ProjectionsDescription.cpp | 8 +- src/Storages/WindowView/StorageWindowView.cpp | 6 +- src/Storages/buildQueryTreeForShard.cpp | 4 +- 19 files changed, 270 insertions(+), 270 deletions(-) rename src/Interpreters/{SquashingTransform.cpp => Squashing.cpp} (80%) rename src/Interpreters/{SquashingTransform.h => Squashing.h} (84%) create mode 100644 src/Processors/Transforms/ApplySquashingTransform.h rename src/Processors/Transforms/{BalancingTransform.cpp => PlanSquashingTransform.cpp} (83%) rename src/Processors/Transforms/{BalancingTransform.h => PlanSquashingTransform.h} (76%) delete mode 100644 src/Processors/Transforms/SquashingChunksTransform.cpp delete mode 100644 src/Processors/Transforms/SquashingChunksTransform.h create mode 100644 src/Processors/Transforms/SquashingTransform.cpp create mode 100644 src/Processors/Transforms/SquashingTransform.h diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index 0cfff7e3f6c0..1e118bd6a32c 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include @@ -62,7 +62,7 @@ std::shared_ptr createASTStorageDistributed( Block getBlockWithAllStreamData(QueryPipelineBuilder builder) { - builder.addTransform(std::make_shared( + builder.addTransform(std::make_shared( builder.getHeader(), std::numeric_limits::max(), std::numeric_limits::max())); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index a30616b0f3e3..e39af9e2804d 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -25,8 +25,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include @@ -605,7 +605,7 @@ BlockIO InterpreterInsertQuery::execute() { bool table_prefers_large_blocks = table->prefersLargeBlocks(); - pipeline.addTransform(std::make_shared( + pipeline.addTransform(std::make_shared( header, table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, @@ -613,7 +613,7 @@ BlockIO InterpreterInsertQuery::execute() pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { - return std::make_shared( + return std::make_shared( in_header, table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); @@ -669,14 +669,14 @@ BlockIO InterpreterInsertQuery::execute() { bool table_prefers_large_blocks = table->prefersLargeBlocks(); - auto squashing = std::make_shared( + auto squashing = std::make_shared( chain.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); chain.addSource(std::move(squashing)); - auto balancing = std::make_shared( + auto balancing = std::make_shared( chain.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, diff --git a/src/Interpreters/SquashingTransform.cpp b/src/Interpreters/Squashing.cpp similarity index 80% rename from src/Interpreters/SquashingTransform.cpp rename to src/Interpreters/Squashing.cpp index dc2ce69ed2f1..6063714e8dbb 100644 --- a/src/Interpreters/SquashingTransform.cpp +++ b/src/Interpreters/Squashing.cpp @@ -1,4 +1,4 @@ -#include +#include #include @@ -9,18 +9,18 @@ namespace ErrorCodes extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; } -SquashingTransform::SquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_) +Squashing::Squashing(size_t min_block_size_rows_, size_t min_block_size_bytes_) : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) { } -Block SquashingTransform::add(Block && input_block) +Block Squashing::add(Block && input_block) { return addImpl(std::move(input_block)); } -Block SquashingTransform::add(const Block & input_block) +Block Squashing::add(const Block & input_block) { return addImpl(input_block); } @@ -32,7 +32,7 @@ Block SquashingTransform::add(const Block & input_block) * have to. */ template -Block SquashingTransform::addImpl(ReferenceType input_block) +Block Squashing::addImpl(ReferenceType input_block) { /// End of input stream. if (!input_block) @@ -80,7 +80,7 @@ Block SquashingTransform::addImpl(ReferenceType input_block) template -void SquashingTransform::append(ReferenceType input_block) +void Squashing::append(ReferenceType input_block) { if (!accumulated_block) { @@ -101,7 +101,7 @@ void SquashingTransform::append(ReferenceType input_block) } -bool SquashingTransform::isEnoughSize(const Block & block) +bool Squashing::isEnoughSize(const Block & block) { size_t rows = 0; size_t bytes = 0; @@ -120,26 +120,26 @@ bool SquashingTransform::isEnoughSize(const Block & block) } -bool SquashingTransform::isEnoughSize(size_t rows, size_t bytes) const +bool Squashing::isEnoughSize(size_t rows, size_t bytes) const { return (!min_block_size_rows && !min_block_size_bytes) || (min_block_size_rows && rows >= min_block_size_rows) || (min_block_size_bytes && bytes >= min_block_size_bytes); } -NewSquashingTransform::NewSquashingTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) +ApplySquashing::ApplySquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) , header(std::move(header_)) { } -Block NewSquashingTransform::add(Chunk && input_chunk) +Block ApplySquashing::add(Chunk && input_chunk) { return addImpl(std::move(input_chunk)); } -Block NewSquashingTransform::addImpl(Chunk && input_chunk) +Block ApplySquashing::addImpl(Chunk && input_chunk) { if (!input_chunk.hasChunkInfo()) { @@ -159,7 +159,7 @@ Block NewSquashingTransform::addImpl(Chunk && input_chunk) } } -void NewSquashingTransform::append(Chunk && input_chunk) +void ApplySquashing::append(Chunk && input_chunk) { if (input_chunk.getNumColumns() == 0) return; @@ -183,30 +183,31 @@ void NewSquashingTransform::append(Chunk && input_chunk) } } -const ChunksToSquash* NewSquashingTransform::getInfoFromChunk(const Chunk & chunk) +const ChunksToSquash* ApplySquashing::getInfoFromChunk(const Chunk & chunk) { const auto& info = chunk.getChunkInfo(); const auto * agg_info = typeid_cast(info.get()); + if (!agg_info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no ChunksToSquash in ChunkInfoPtr"); + return agg_info; } -BalanceTransform::BalanceTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) +PlanSquashing::PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) , header(std::move(header_)) { } -Chunk BalanceTransform::add(Block && input_block) +Chunk PlanSquashing::add(Chunk && input_chunk) { - return addImpl(std::move(input_block)); + return addImpl(std::move(input_chunk)); } -Chunk BalanceTransform::addImpl(Block && input_block) +Chunk PlanSquashing::addImpl(Chunk && input_chunk) { - Chunk input_chunk(input_block.getColumns(), input_block.rows()); - if (!input_chunk) { Chunk res_chunk = convertToChunk(chunks_to_merge_vec); @@ -227,7 +228,7 @@ Chunk BalanceTransform::addImpl(Block && input_block) return input_chunk; } -Chunk BalanceTransform::convertToChunk(std::vector &chunks) +Chunk PlanSquashing::convertToChunk(std::vector &chunks) { if (chunks.empty()) return {}; @@ -241,7 +242,7 @@ Chunk BalanceTransform::convertToChunk(std::vector &chunks) return Chunk(header.cloneEmptyColumns(), 0, info); } -bool BalanceTransform::isEnoughSize(const std::vector & chunks) +bool PlanSquashing::isEnoughSize(const std::vector & chunks) { size_t rows = 0; size_t bytes = 0; @@ -255,7 +256,7 @@ bool BalanceTransform::isEnoughSize(const std::vector & chunks) return isEnoughSize(rows, bytes); } -bool BalanceTransform::isEnoughSize(size_t rows, size_t bytes) const +bool PlanSquashing::isEnoughSize(size_t rows, size_t bytes) const { return (!min_block_size_rows && !min_block_size_bytes) || (min_block_size_rows && rows >= min_block_size_rows) diff --git a/src/Interpreters/SquashingTransform.h b/src/Interpreters/Squashing.h similarity index 84% rename from src/Interpreters/SquashingTransform.h rename to src/Interpreters/Squashing.h index 95e8c2a8fa12..82d7fe616f6b 100644 --- a/src/Interpreters/SquashingTransform.h +++ b/src/Interpreters/Squashing.h @@ -27,11 +27,11 @@ struct ChunksToSquash : public ChunkInfo * * Order of data is kept. */ -class SquashingTransform +class Squashing { public: /// Conditions on rows and bytes are OR-ed. If one of them is zero, then corresponding condition is ignored. - SquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_); + Squashing(size_t min_block_size_rows_, size_t min_block_size_bytes_); /** Add next block and possibly returns squashed block. * At end, you need to pass empty block. As the result for last (empty) block, you will get last Result with ready = true. @@ -55,10 +55,10 @@ class SquashingTransform bool isEnoughSize(size_t rows, size_t bytes) const; }; -class NewSquashingTransform +class ApplySquashing { public: - NewSquashingTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); + ApplySquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); Block add(Chunk && input_chunk); @@ -79,12 +79,12 @@ class NewSquashingTransform bool isEnoughSize(size_t rows, size_t bytes) const; }; -class BalanceTransform +class PlanSquashing { public: - BalanceTransform(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); + PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); - Chunk add(Block && input_block); + Chunk add(Chunk && input_chunk); bool isDataLeft() { return !chunks_to_merge_vec.empty(); @@ -97,7 +97,7 @@ class BalanceTransform const Block header; - Chunk addImpl(Block && input_block); + Chunk addImpl(Chunk && input_chunk); bool isEnoughSize(const std::vector & chunks); bool isEnoughSize(size_t rows, size_t bytes) const; diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 74da97f21990..7ca9cd754b2f 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include namespace ProfileEvents @@ -773,7 +773,7 @@ void AggregatingTransform::initGenerate() { /// Just a reasonable constant, matches default value for the setting `preferred_block_size_bytes` static constexpr size_t oneMB = 1024 * 1024; - return std::make_shared(header, params->params.max_block_size, oneMB); + return std::make_shared(header, params->params.max_block_size, oneMB); }); } /// AggregatingTransform::expandPipeline expects single output port. diff --git a/src/Processors/Transforms/ApplySquashingTransform.h b/src/Processors/Transforms/ApplySquashingTransform.h new file mode 100644 index 000000000000..26507d9c4966 --- /dev/null +++ b/src/Processors/Transforms/ApplySquashingTransform.h @@ -0,0 +1,63 @@ +#include +#include +#include + +namespace DB +{ + +class ApplySquashingTransform : public ExceptionKeepingTransform +{ +public: + ApplySquashingTransform( + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) + : ExceptionKeepingTransform(header, header, false) + , squashing(header, min_block_size_rows, min_block_size_bytes) + { + } + + String getName() const override { return "ApplySquashingTransform"; } + + void work() override + { + if (stage == Stage::Exception) + { + data.chunk.clear(); + ready_input = false; + return; + } + + ExceptionKeepingTransform::work(); + if (finish_chunk) + { + data.chunk = std::move(finish_chunk); + ready_output = true; + } + } + +protected: + void onConsume(Chunk chunk) override + { + if (auto block = squashing.add(std::move(chunk))) + cur_chunk.setColumns(block.getColumns(), block.rows()); + } + + GenerateResult onGenerate() override + { + GenerateResult res; + res.chunk = std::move(cur_chunk); + res.is_done = true; + return res; + } + void onFinish() override + { + auto block = squashing.add({}); + finish_chunk.setColumns(block.getColumns(), block.rows()); + } + +private: + ApplySquashing squashing; + Chunk cur_chunk; + Chunk finish_chunk; +}; + +} diff --git a/src/Processors/Transforms/BalancingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp similarity index 83% rename from src/Processors/Transforms/BalancingTransform.cpp rename to src/Processors/Transforms/PlanSquashingTransform.cpp index def124070198..62ff3a0bf39e 100644 --- a/src/Processors/Transforms/BalancingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -1,15 +1,15 @@ -#include +#include #include namespace DB { -BalancingChunksTransform::BalancingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) +PlanSquashingTransform::PlanSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), balance(header, min_block_size_rows, min_block_size_bytes) { } -IProcessor::Status BalancingChunksTransform::prepare() +IProcessor::Status PlanSquashingTransform::prepare() { Status status = Status::Ready; @@ -22,7 +22,7 @@ IProcessor::Status BalancingChunksTransform::prepare() return status; } -IProcessor::Status BalancingChunksTransform::prepareConsume() +IProcessor::Status PlanSquashingTransform::prepareConsume() { finished = false; bool all_finished = true; @@ -90,7 +90,7 @@ IProcessor::Status BalancingChunksTransform::prepareConsume() chunk = input.pull(); transform(chunk); - was_output_processed.assign(inputs.size(), false); + was_output_processed.assign(outputs.size(), false); if (chunk.hasChunkInfo()) { has_data = true; @@ -102,11 +102,11 @@ IProcessor::Status BalancingChunksTransform::prepareConsume() return Status::Ready; } -void BalancingChunksTransform::transform(Chunk & chunk_) +void PlanSquashingTransform::transform(Chunk & chunk_) { if (!finished) { - Chunk res_chunk = balance.add(getInputPorts().front().getHeader().cloneWithColumns(chunk_.detachColumns())); + Chunk res_chunk = balance.add(std::move(chunk_)); std::swap(res_chunk, chunk_); } else @@ -116,7 +116,7 @@ void BalancingChunksTransform::transform(Chunk & chunk_) } } -IProcessor::Status BalancingChunksTransform::prepareSend() +IProcessor::Status PlanSquashingTransform::prepareSend() { bool all_outputs_processed = true; diff --git a/src/Processors/Transforms/BalancingTransform.h b/src/Processors/Transforms/PlanSquashingTransform.h similarity index 76% rename from src/Processors/Transforms/BalancingTransform.h rename to src/Processors/Transforms/PlanSquashingTransform.h index a8a8bc5cfab9..c30569fffb59 100644 --- a/src/Processors/Transforms/BalancingTransform.h +++ b/src/Processors/Transforms/PlanSquashingTransform.h @@ -2,19 +2,19 @@ #include #include -#include +#include namespace DB { -class BalancingChunksTransform : public IProcessor +class PlanSquashingTransform : public IProcessor { public: - BalancingChunksTransform( + PlanSquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports); - String getName() const override { return "BalancingChunksTransform"; } + String getName() const override { return "PlanSquashingTransform"; } InputPorts & getInputPorts() { return inputs; } OutputPorts & getOutputPorts() { return outputs; } @@ -30,7 +30,7 @@ class BalancingChunksTransform : public IProcessor private: size_t CalculateBlockSize(const Block & block); Chunk chunk; - BalanceTransform balance; + PlanSquashing balance; bool has_data = false; std::vector was_output_processed; diff --git a/src/Processors/Transforms/SquashingChunksTransform.cpp b/src/Processors/Transforms/SquashingChunksTransform.cpp deleted file mode 100644 index 62b87061344c..000000000000 --- a/src/Processors/Transforms/SquashingChunksTransform.cpp +++ /dev/null @@ -1,124 +0,0 @@ -#include -#include - -namespace DB -{ - -SquashingChunksTransform::SquashingChunksTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) - : ExceptionKeepingTransform(header, header, false) - , squashing(min_block_size_rows, min_block_size_bytes) -{ -} - -void SquashingChunksTransform::onConsume(Chunk chunk) -{ - if (auto block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()))) - cur_chunk.setColumns(block.getColumns(), block.rows()); -} - -SquashingChunksTransform::GenerateResult SquashingChunksTransform::onGenerate() -{ - GenerateResult res; - res.chunk = std::move(cur_chunk); - res.is_done = true; - return res; -} - -void SquashingChunksTransform::onFinish() -{ - auto block = squashing.add({}); - finish_chunk.setColumns(block.getColumns(), block.rows()); -} - -void SquashingChunksTransform::work() -{ - if (stage == Stage::Exception) - { - data.chunk.clear(); - ready_input = false; - return; - } - - ExceptionKeepingTransform::work(); - if (finish_chunk) - { - data.chunk = std::move(finish_chunk); - ready_output = true; - } -} - -SquashingChunksTransformForBalancing::SquashingChunksTransformForBalancing( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) - : ExceptionKeepingTransform(header, header, false) - , squashing(header, min_block_size_rows, min_block_size_bytes) -{ -} - -void SquashingChunksTransformForBalancing::onConsume(Chunk chunk) -{ - if (auto block = squashing.add(std::move(chunk))) - cur_chunk.setColumns(block.getColumns(), block.rows()); -} - -SquashingChunksTransformForBalancing::GenerateResult SquashingChunksTransformForBalancing::onGenerate() -{ - GenerateResult res; - res.chunk = std::move(cur_chunk); - res.is_done = true; - return res; -} - -void SquashingChunksTransformForBalancing::onFinish() -{ - auto block = squashing.add({}); - finish_chunk.setColumns(block.getColumns(), block.rows()); -} - -void SquashingChunksTransformForBalancing::work() -{ - if (stage == Stage::Exception) - { - data.chunk.clear(); - ready_input = false; - return; - } - - ExceptionKeepingTransform::work(); - if (finish_chunk) - { - data.chunk = std::move(finish_chunk); - ready_output = true; - } -} - -SimpleSquashingChunksTransform::SimpleSquashingChunksTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) - : ISimpleTransform(header, header, false), squashing(min_block_size_rows, min_block_size_bytes) -{ -} - -void SimpleSquashingChunksTransform::transform(Chunk & chunk) -{ - if (!finished) - { - if (auto block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()))) - chunk.setColumns(block.getColumns(), block.rows()); - } - else - { - auto block = squashing.add({}); - chunk.setColumns(block.getColumns(), block.rows()); - } -} - -IProcessor::Status SimpleSquashingChunksTransform::prepare() -{ - if (!finished && input.isFinished()) - { - finished = true; - return Status::Ready; - } - return ISimpleTransform::prepare(); -} -} diff --git a/src/Processors/Transforms/SquashingChunksTransform.h b/src/Processors/Transforms/SquashingChunksTransform.h deleted file mode 100644 index 7c7948d1af91..000000000000 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ /dev/null @@ -1,70 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -class SquashingChunksTransform : public ExceptionKeepingTransform -{ -public: - explicit SquashingChunksTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); - - String getName() const override { return "SquashingTransform"; } - - void work() override; - -protected: - void onConsume(Chunk chunk) override; - GenerateResult onGenerate() override; - void onFinish() override; - -private: - SquashingTransform squashing; - Chunk cur_chunk; - Chunk finish_chunk; -}; - -class SquashingChunksTransformForBalancing : public ExceptionKeepingTransform -{ -public: - explicit SquashingChunksTransformForBalancing( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); - - String getName() const override { return "SquashingTransform"; } - - void work() override; - -protected: - void onConsume(Chunk chunk) override; - GenerateResult onGenerate() override; - void onFinish() override; - -private: - NewSquashingTransform squashing; - Chunk cur_chunk; - Chunk finish_chunk; -}; - -/// Doesn't care about propagating exceptions and thus doesn't throw LOGICAL_ERROR if the following transform closes its input port. -class SimpleSquashingChunksTransform : public ISimpleTransform -{ -public: - explicit SimpleSquashingChunksTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); - - String getName() const override { return "SimpleSquashingTransform"; } - -protected: - void transform(Chunk &) override; - - IProcessor::Status prepare() override; - -private: - SquashingTransform squashing; - - bool finished = false; -}; -} diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp new file mode 100644 index 000000000000..43f72262846d --- /dev/null +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -0,0 +1,80 @@ +#include +#include + +namespace DB +{ + +SquashingTransform::SquashingTransform( + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) + : ExceptionKeepingTransform(header, header, false) + , squashing(min_block_size_rows, min_block_size_bytes) +{ +} + +void SquashingTransform::onConsume(Chunk chunk) +{ + if (auto block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()))) + cur_chunk.setColumns(block.getColumns(), block.rows()); +} + +SquashingTransform::GenerateResult SquashingTransform::onGenerate() +{ + GenerateResult res; + res.chunk = std::move(cur_chunk); + res.is_done = true; + return res; +} + +void SquashingTransform::onFinish() +{ + auto block = squashing.add({}); + finish_chunk.setColumns(block.getColumns(), block.rows()); +} + +void SquashingTransform::work() +{ + if (stage == Stage::Exception) + { + data.chunk.clear(); + ready_input = false; + return; + } + + ExceptionKeepingTransform::work(); + if (finish_chunk) + { + data.chunk = std::move(finish_chunk); + ready_output = true; + } +} + +SimpleSquashingTransform::SimpleSquashingTransform( + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) + : ISimpleTransform(header, header, false), squashing(min_block_size_rows, min_block_size_bytes) +{ +} + +void SimpleSquashingTransform::transform(Chunk & chunk) +{ + if (!finished) + { + if (auto block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()))) + chunk.setColumns(block.getColumns(), block.rows()); + } + else + { + auto block = squashing.add({}); + chunk.setColumns(block.getColumns(), block.rows()); + } +} + +IProcessor::Status SimpleSquashingTransform::prepare() +{ + if (!finished && input.isFinished()) + { + finished = true; + return Status::Ready; + } + return ISimpleTransform::prepare(); +} +} diff --git a/src/Processors/Transforms/SquashingTransform.h b/src/Processors/Transforms/SquashingTransform.h new file mode 100644 index 000000000000..c5b727ac6ec7 --- /dev/null +++ b/src/Processors/Transforms/SquashingTransform.h @@ -0,0 +1,50 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class SquashingTransform : public ExceptionKeepingTransform +{ +public: + explicit SquashingTransform( + const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); + + String getName() const override { return "SquashingTransform"; } + + void work() override; + +protected: + void onConsume(Chunk chunk) override; + GenerateResult onGenerate() override; + void onFinish() override; + +private: + Squashing squashing; + Chunk cur_chunk; + Chunk finish_chunk; +}; + +/// Doesn't care about propagating exceptions and thus doesn't throw LOGICAL_ERROR if the following transform closes its input port. +class SimpleSquashingTransform : public ISimpleTransform +{ +public: + explicit SimpleSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes); + + String getName() const override { return "SimpleSquashingTransform"; } + +protected: + void transform(Chunk &) override; + + IProcessor::Status prepare() override; + +private: + Squashing squashing; + + bool finished = false; +}; +} diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 93cfc8f6d10f..e106cbf8d425 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -6,8 +6,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include @@ -369,12 +369,12 @@ std::optional generateViewChain( bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); const auto & settings = insert_context->getSettingsRef(); - out.addSource(std::make_shared( + out.addSource(std::make_shared( out.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); - out.addSource(std::make_shared( + out.addSource(std::make_shared( out.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, @@ -625,12 +625,12 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat /// Squashing is needed here because the materialized view query can generate a lot of blocks /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY /// and two-level aggregation is triggered). - pipeline.addTransform(std::make_shared( + pipeline.addTransform(std::make_shared( pipeline.getHeader(), context->getSettingsRef().min_insert_block_size_rows, context->getSettingsRef().min_insert_block_size_bytes, pipeline.getNumStreams())); - pipeline.addTransform(std::make_shared( + pipeline.addTransform(std::make_shared( pipeline.getHeader(), context->getSettingsRef().min_insert_block_size_rows, context->getSettingsRef().min_insert_block_size_bytes)); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d883029408c5..08d8b1b9fbc5 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1,6 +1,6 @@ -#include "Interpreters/AsynchronousInsertQueue.h" -#include "Interpreters/SquashingTransform.h" -#include "Parsers/ASTInsertQuery.h" +#include +#include +#include #include #include #include @@ -876,7 +876,7 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro using PushResult = AsynchronousInsertQueue::PushResult; startInsertQuery(); - SquashingTransform squashing(0, query_context->getSettingsRef().async_insert_max_data_size); + Squashing squashing(0, query_context->getSettingsRef().async_insert_max_data_size); while (readDataNext()) { diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 2f011567b900..b274518e248b 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -21,7 +21,7 @@ limitations under the License. */ #include #include #include -#include +#include #include #include #include @@ -667,7 +667,7 @@ QueryPipelineBuilder StorageLiveView::completeQuery(Pipes pipes) /// and two-level aggregation is triggered). builder.addSimpleTransform([&](const Block & cur_header) { - return std::make_shared( + return std::make_shared( cur_header, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 1c33f018a5d4..514e7b8299b8 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include @@ -1223,7 +1223,7 @@ class PartMergerWriter ProjectionNameToItsBlocks projection_parts; std::move_iterator projection_parts_iterator; - std::vector projection_squashes; + std::vector projection_squashes; const ProjectionsDescription & projections; ExecutableTaskPtr merge_projection_parts_task_ptr; diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 731ac04a8c80..d1bcc89cbe0e 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -16,8 +16,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include @@ -311,8 +311,8 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) builder.resize(1); // Generate aggregated blocks with rows less or equal than the original block. // There should be only one output block after this transformation. - builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, builder.getNumStreams())); - builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, builder.getNumStreams())); + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index b1984a947c8c..f6ffaf679edd 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -35,7 +35,7 @@ #include #include #include -#include +#include #include #include #include @@ -610,7 +610,7 @@ std::pair StorageWindowView::getNewBlocks(UInt32 watermark) }); builder.addSimpleTransform([&](const Block & current_header) { - return std::make_shared( + return std::make_shared( current_header, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes); @@ -1498,7 +1498,7 @@ void StorageWindowView::writeIntoWindowView( builder = select_block.buildQueryPipeline(); builder.addSimpleTransform([&](const Block & current_header) { - return std::make_shared( + return std::make_shared( current_header, local_context->getSettingsRef().min_insert_block_size_rows, local_context->getSettingsRef().min_insert_block_size_bytes); diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index c87a1b216ca1..977a803bd283 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -16,7 +16,7 @@ #include #include #include -#include +#include #include #include #include @@ -288,7 +288,7 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, size_t min_block_size_rows = mutable_context->getSettingsRef().min_external_table_block_size_rows; size_t min_block_size_bytes = mutable_context->getSettingsRef().min_external_table_block_size_bytes; - auto squashing = std::make_shared(builder->getHeader(), min_block_size_rows, min_block_size_bytes); + auto squashing = std::make_shared(builder->getHeader(), min_block_size_rows, min_block_size_bytes); builder->resize(1); builder->addTransform(std::move(squashing)); From 8c0786bd80a2aad2934395124d9c1213fe79e0cc Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 8 May 2024 19:43:22 +0000 Subject: [PATCH 038/254] fix for projections --- src/Interpreters/Squashing.cpp | 10 ++-- .../Transforms/PlanSquashingTransform.cpp | 50 ++++++++----------- src/Storages/ProjectionsDescription.cpp | 2 +- 3 files changed, 26 insertions(+), 36 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 6063714e8dbb..ece124e8a15a 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -142,11 +142,7 @@ Block ApplySquashing::add(Chunk && input_chunk) Block ApplySquashing::addImpl(Chunk && input_chunk) { if (!input_chunk.hasChunkInfo()) - { - Block to_return; - std::swap(to_return, accumulated_block); - return to_return; - } + return Block(); const auto *info = getInfoFromChunk(input_chunk); for (auto & chunk : info->chunks) @@ -225,7 +221,7 @@ Chunk PlanSquashing::addImpl(Chunk && input_chunk) Chunk res_chunk = convertToChunk(chunks_to_merge_vec); return res_chunk; } - return input_chunk; + return {}; } Chunk PlanSquashing::convertToChunk(std::vector &chunks) @@ -237,7 +233,7 @@ Chunk PlanSquashing::convertToChunk(std::vector &chunks) for (auto &chunk : chunks) info->chunks.push_back(std::move(chunk)); - chunks.clear(); // we can remove this + chunks.clear(); return Chunk(header.cloneEmptyColumns(), 0, info); } diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 62ff3a0bf39e..fe0f6ed39f5d 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -41,34 +41,32 @@ IProcessor::Status PlanSquashingTransform::prepareConsume() return Status::Finished; } - all_finished = true; - for (auto & input : inputs) - { - if (input.isFinished()) - continue; - - all_finished = false; - } - - if (all_finished) /// If all inputs are closed, we check if we have data in balancing + while (!chunk.hasChunkInfo()) { - if (balance.isDataLeft()) /// If we have data in balancing, we process this data + all_finished = true; + for (auto & input : inputs) { - finished = false; - transform(chunk); - has_data = true; + if (!input.isFinished()) + all_finished = false; } - else /// If we don't have data, We send FINISHED + + if (all_finished) /// If all inputs are closed, we check if we have data in balancing { - for (auto & output : outputs) - output.finish(); + if (balance.isDataLeft()) /// If we have data in balancing, we process this data + { + finished = false; + transform(chunk); + has_data = true; + } + else /// If we don't have data, We send FINISHED + { + for (auto & output : outputs) + output.finish(); - return Status::Finished; + return Status::Finished; + } } - } - while (!chunk.hasChunkInfo()) - { for (auto & input : inputs) { if (input.isFinished()) @@ -80,12 +78,7 @@ IProcessor::Status PlanSquashingTransform::prepareConsume() if (!balance.isDataLeft()) return Status::NeedData; else - { - finished = true; - transform(chunk); - has_data = true; - return Status::Ready; - } + continue; } chunk = input.pull(); @@ -96,7 +89,8 @@ IProcessor::Status PlanSquashingTransform::prepareConsume() has_data = true; return Status::Ready; } - + else + return Status::NeedData; } } return Status::Ready; diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index d1bcc89cbe0e..87e203e8665a 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -311,7 +311,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) builder.resize(1); // Generate aggregated blocks with rows less or equal than the original block. // There should be only one output block after this transformation. - builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, builder.getNumStreams())); + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, 1)); builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); From e39213a8879abbb54ed286f954dc3de6702c61db Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 9 May 2024 00:45:16 +0200 Subject: [PATCH 039/254] empty commit From 37c67aba9f933b949c3cf27f246e71174ed0d8a6 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 10 May 2024 13:32:34 +0200 Subject: [PATCH 040/254] remove squashingTransform.cpp --- src/Interpreters/SquashingTransform.cpp | 145 ------------------------ 1 file changed, 145 deletions(-) delete mode 100644 src/Interpreters/SquashingTransform.cpp diff --git a/src/Interpreters/SquashingTransform.cpp b/src/Interpreters/SquashingTransform.cpp deleted file mode 100644 index 41f024df7a72..000000000000 --- a/src/Interpreters/SquashingTransform.cpp +++ /dev/null @@ -1,145 +0,0 @@ -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; - extern const int LOGICAL_ERROR; -} - -SquashingTransform::SquashingTransform(size_t min_block_size_rows_, size_t min_block_size_bytes_) - : min_block_size_rows(min_block_size_rows_) - , min_block_size_bytes(min_block_size_bytes_) -{ -} - -Block SquashingTransform::add(Block && input_block) -{ - return addImpl(std::move(input_block)); -} - -Block SquashingTransform::add(const Block & input_block) -{ - return addImpl(input_block); -} - -/* - * To minimize copying, accept two types of argument: const reference for output - * stream, and rvalue reference for input stream, and decide whether to copy - * inside this function. This allows us not to copy Block unless we absolutely - * have to. - */ -template -Block SquashingTransform::addImpl(ReferenceType input_block) -{ - /// End of input stream. - if (!input_block) - { - Block to_return; - std::swap(to_return, accumulated_block); - return to_return; - } - - /// Just read block is already enough. - if (isEnoughSize(input_block)) - { - /// If no accumulated data, return just read block. - if (!accumulated_block) - { - return std::move(input_block); - } - - /// Return accumulated data (maybe it has small size) and place new block to accumulated data. - Block to_return = std::move(input_block); - std::swap(to_return, accumulated_block); - return to_return; - } - - /// Accumulated block is already enough. - if (isEnoughSize(accumulated_block)) - { - /// Return accumulated data and place new block to accumulated data. - Block to_return = std::move(input_block); - std::swap(to_return, accumulated_block); - return to_return; - } - - append(std::move(input_block)); - if (isEnoughSize(accumulated_block)) - { - Block to_return; - std::swap(to_return, accumulated_block); - return to_return; - } - - /// Squashed block is not ready. - return {}; -} - - -template -void SquashingTransform::append(ReferenceType input_block) -{ - if (!accumulated_block) - { - accumulated_block = std::move(input_block); - return; - } - - assert(blocksHaveEqualStructure(input_block, accumulated_block)); - - try - { - for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) - { - const auto source_column = input_block.getByPosition(i).column; - - auto mutable_column = IColumn::mutate(std::move(accumulated_block.getByPosition(i).column)); - mutable_column->insertRangeFrom(*source_column, 0, source_column->size()); - accumulated_block.getByPosition(i).column = std::move(mutable_column); - } - } - catch (...) - { - /// add() may be called again even after a previous add() threw an exception. - /// Keep accumulated_block in a valid state. - /// Seems ok to discard accumulated data because we're throwing an exception, which the caller will - /// hopefully interpret to mean "this block and all *previous* blocks are potentially lost". - accumulated_block.clear(); - throw; - } -} - - -bool SquashingTransform::isEnoughSize(const Block & block) -{ - size_t rows = 0; - size_t bytes = 0; - - for (const auto & [column, type, name] : block) - { - if (!column) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid column in block."); - - if (!rows) - rows = column->size(); - else if (rows != column->size()) - throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Sizes of columns doesn't match"); - - bytes += column->byteSize(); - } - - return isEnoughSize(rows, bytes); -} - - -bool SquashingTransform::isEnoughSize(size_t rows, size_t bytes) const -{ - return (!min_block_size_rows && !min_block_size_bytes) - || (min_block_size_rows && rows >= min_block_size_rows) - || (min_block_size_bytes && bytes >= min_block_size_bytes); -} - -} From bcd5482c5b73743ec958a2f818c6e72dfd784832 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 10 May 2024 13:34:09 +0200 Subject: [PATCH 041/254] remove internals.cpp --- programs/copier/Internals.cpp | 280 ---------------------------------- 1 file changed, 280 deletions(-) delete mode 100644 programs/copier/Internals.cpp diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp deleted file mode 100644 index 1e118bd6a32c..000000000000 --- a/programs/copier/Internals.cpp +++ /dev/null @@ -1,280 +0,0 @@ -#include "Internals.h" -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - -using ConfigurationPtr = Poco::AutoPtr; - -ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) -{ - std::stringstream ss(xml_data); // STYLE_CHECK_ALLOW_STD_STRING_STREAM - Poco::XML::InputSource input_source{ss}; - return {new Poco::Util::XMLConfiguration{&input_source}}; -} - -String getQuotedTable(const String & database, const String & table) -{ - if (database.empty()) - return backQuoteIfNeed(table); - - return backQuoteIfNeed(database) + "." + backQuoteIfNeed(table); -} - -String getQuotedTable(const DatabaseAndTableName & db_and_table) -{ - return getQuotedTable(db_and_table.first, db_and_table.second); -} - - -// Creates AST representing 'ENGINE = Distributed(cluster, db, table, [sharding_key]) -std::shared_ptr createASTStorageDistributed( - const String & cluster_name, const String & database, const String & table, - const ASTPtr & sharding_key_ast) -{ - auto args = std::make_shared(); - args->children.emplace_back(std::make_shared(cluster_name)); - args->children.emplace_back(std::make_shared(database)); - args->children.emplace_back(std::make_shared(table)); - if (sharding_key_ast) - args->children.emplace_back(sharding_key_ast); - - auto engine = std::make_shared(); - engine->name = "Distributed"; - engine->arguments = args; - - auto storage = std::make_shared(); - storage->set(storage->engine, engine); - - return storage; -} - - -Block getBlockWithAllStreamData(QueryPipelineBuilder builder) -{ - builder.addTransform(std::make_shared( - builder.getHeader(), - std::numeric_limits::max(), - std::numeric_limits::max())); - - auto cur_pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); - Block block; - PullingPipelineExecutor executor(cur_pipeline); - executor.pull(block); - - return block; -} - -bool isExtendedDefinitionStorage(const ASTPtr & storage_ast) -{ - const auto & storage = storage_ast->as(); - return storage.partition_by || storage.order_by || storage.sample_by; -} - -ASTPtr extractPartitionKey(const ASTPtr & storage_ast) -{ - String storage_str = queryToString(storage_ast); - - const auto & storage = storage_ast->as(); - const auto & engine = storage.engine->as(); - - if (!endsWith(engine.name, "MergeTree")) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported engine was specified in {}, only *MergeTree engines are supported", storage_str); - } - - if (isExtendedDefinitionStorage(storage_ast)) - { - if (storage.partition_by) - return storage.partition_by->clone(); - - static const char * all = "all"; - return std::make_shared(Field(all, strlen(all))); - } - else - { - bool is_replicated = startsWith(engine.name, "Replicated"); - size_t min_args = is_replicated ? 3 : 1; - - if (!engine.arguments) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected arguments in {}", storage_str); - - ASTPtr arguments_ast = engine.arguments->clone(); - ASTs & arguments = arguments_ast->children; - - if (arguments.size() < min_args) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected at least {} arguments in {}", min_args, storage_str); - - ASTPtr & month_arg = is_replicated ? arguments[2] : arguments[1]; - return makeASTFunction("toYYYYMM", month_arg->clone()); - } -} - -ASTPtr extractPrimaryKey(const ASTPtr & storage_ast) -{ - String storage_str = queryToString(storage_ast); - - const auto & storage = storage_ast->as(); - const auto & engine = storage.engine->as(); - - if (!endsWith(engine.name, "MergeTree")) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported engine was specified in {}, only *MergeTree engines are supported", storage_str); - } - - if (!isExtendedDefinitionStorage(storage_ast)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Is not extended deginition storage {} Will be fixed later.", storage_str); - } - - if (storage.primary_key) - return storage.primary_key->clone(); - - return nullptr; -} - - -ASTPtr extractOrderBy(const ASTPtr & storage_ast) -{ - String storage_str = queryToString(storage_ast); - - const auto & storage = storage_ast->as(); - const auto & engine = storage.engine->as(); - - if (!endsWith(engine.name, "MergeTree")) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported engine was specified in {}, only *MergeTree engines are supported", storage_str); - } - - if (!isExtendedDefinitionStorage(storage_ast)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Is not extended deginition storage {} Will be fixed later.", storage_str); - } - - if (storage.order_by) - return storage.order_by->clone(); - - throw Exception(ErrorCodes::BAD_ARGUMENTS, "ORDER BY cannot be empty"); -} - -/// Wraps only identifiers with backticks. -std::string wrapIdentifiersWithBackticks(const ASTPtr & root) -{ - if (auto identifier = std::dynamic_pointer_cast(root)) - return backQuote(identifier->name()); - - if (auto function = std::dynamic_pointer_cast(root)) - return function->name + '(' + wrapIdentifiersWithBackticks(function->arguments) + ')'; - - if (auto expression_list = std::dynamic_pointer_cast(root)) - { - Names function_arguments(expression_list->children.size()); - for (size_t i = 0; i < expression_list->children.size(); ++i) - function_arguments[i] = wrapIdentifiersWithBackticks(expression_list->children[0]); - return boost::algorithm::join(function_arguments, ", "); - } - - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key could be represented only as columns or functions from columns."); -} - - -Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast) -{ - const auto sorting_key_ast = extractOrderBy(storage_ast); - const auto primary_key_ast = extractPrimaryKey(storage_ast); - - const auto sorting_key_expr_list = extractKeyExpressionList(sorting_key_ast); - const auto primary_key_expr_list = primary_key_ast - ? extractKeyExpressionList(primary_key_ast) : sorting_key_expr_list->clone(); - - /// Maybe we have to handle VersionedCollapsing engine separately. But in our case in looks pointless. - - size_t primary_key_size = primary_key_expr_list->children.size(); - size_t sorting_key_size = sorting_key_expr_list->children.size(); - - if (primary_key_size > sorting_key_size) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key must be a prefix of the sorting key, but its length: " - "{} is greater than the sorting key length: {}", - primary_key_size, sorting_key_size); - - Names primary_key_columns; - NameSet primary_key_columns_set; - - for (size_t i = 0; i < sorting_key_size; ++i) - { - /// Column name could be represented as a f_1(f_2(...f_n(column_name))). - /// Each f_i could take one or more parameters. - /// We will wrap identifiers with backticks to allow non-standard identifier names. - String sorting_key_column = sorting_key_expr_list->children[i]->getColumnName(); - - if (i < primary_key_size) - { - String pk_column = primary_key_expr_list->children[i]->getColumnName(); - if (pk_column != sorting_key_column) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Primary key must be a prefix of the sorting key, " - "but the column in the position {} is {}, not {}", i, sorting_key_column, pk_column); - - if (!primary_key_columns_set.emplace(pk_column).second) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Primary key contains duplicate columns"); - - primary_key_columns.push_back(wrapIdentifiersWithBackticks(primary_key_expr_list->children[i])); - } - } - - return primary_key_columns; -} - -bool isReplicatedTableEngine(const ASTPtr & storage_ast) -{ - const auto & storage = storage_ast->as(); - const auto & engine = storage.engine->as(); - - if (!endsWith(engine.name, "MergeTree")) - { - String storage_str = queryToString(storage_ast); - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported engine was specified in {}, only *MergeTree engines are supported", storage_str); - } - - return startsWith(engine.name, "Replicated"); -} - -ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) -{ - ShardPriority res; - - if (replicas.empty()) - return res; - - res.is_remote = 1; - for (const auto & replica : replicas) - { - if (isLocalAddress(DNSResolver::instance().resolveHost(replica.host_name))) - { - res.is_remote = 0; - break; - } - } - - res.hostname_difference = std::numeric_limits::max(); - for (const auto & replica : replicas) - { - size_t difference = getHostNamePrefixDistance(local_hostname, replica.host_name); - res.hostname_difference = std::min(difference, res.hostname_difference); - } - - res.random = random; - return res; -} - -} From 2a5671d8819787d4d675a9131c9e3c491110c409 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 10 May 2024 13:41:30 +0200 Subject: [PATCH 042/254] fix style --- src/Interpreters/Squashing.cpp | 1 + src/Processors/Transforms/ApplySquashingTransform.h | 1 + 2 files changed, 2 insertions(+) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index ece124e8a15a..78d1b9fc6437 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -7,6 +7,7 @@ namespace DB namespace ErrorCodes { extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } Squashing::Squashing(size_t min_block_size_rows_, size_t min_block_size_bytes_) diff --git a/src/Processors/Transforms/ApplySquashingTransform.h b/src/Processors/Transforms/ApplySquashingTransform.h index 26507d9c4966..584fb72cccb0 100644 --- a/src/Processors/Transforms/ApplySquashingTransform.h +++ b/src/Processors/Transforms/ApplySquashingTransform.h @@ -1,3 +1,4 @@ +#pragma once #include #include #include From 568c6dfd8039dc389760f3060106e15f96c72d46 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 10 May 2024 14:35:32 +0200 Subject: [PATCH 043/254] fix link in RecursiveCTESource --- src/Processors/Sources/RecursiveCTESource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Sources/RecursiveCTESource.cpp b/src/Processors/Sources/RecursiveCTESource.cpp index b94cb1880867..2f7568c2bb0e 100644 --- a/src/Processors/Sources/RecursiveCTESource.cpp +++ b/src/Processors/Sources/RecursiveCTESource.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include From d833c9cce05cf508596ef5191a9ee179c59a1c6f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 13 May 2024 16:45:38 +0000 Subject: [PATCH 044/254] full refactoring of planning --- .../Transforms/PlanSquashingTransform.cpp | 246 ++++++++++++------ .../Transforms/PlanSquashingTransform.h | 26 +- 2 files changed, 191 insertions(+), 81 deletions(-) diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index fe0f6ed39f5d..89b59354722c 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { @@ -15,85 +16,172 @@ IProcessor::Status PlanSquashingTransform::prepare() while (status == Status::Ready) { - status = !has_data ? prepareConsume() - : prepareSend(); + switch (planning_status) + { + case PlanningStatus::INIT: + { + status = init(); + break; + } + case PlanningStatus::READ_IF_CAN: + { + status = prepareConsume(); + break; + } + case PlanningStatus::WAIT_IN: + { + status = waitForDataIn(); + break; + } + case PlanningStatus::WAIT_OUT_AND_PUSH: + { + status = prepareSend(); + break; + } + case PlanningStatus::PUSH: + { + status = prepareSend(); + break; + } + case PlanningStatus::WAIT_OUT_FLUSH: + { + status = prepareSendFlush(); + break; + } + case FINISH: + { + status = finish(); + break; + } + } } return status; } -IProcessor::Status PlanSquashingTransform::prepareConsume() +IProcessor::Status PlanSquashingTransform::init() { - finished = false; - bool all_finished = true; - for (auto & output : outputs) + for (auto input : inputs) { - if (output.isFinished()) - continue; - - all_finished = false; + input.setNeeded(); + if (input.hasData()) + available_inputs++; } - if (all_finished) /// If all outputs are closed, we close inputs (just in case) + planning_status = PlanningStatus::READ_IF_CAN; + return Status::Ready; +} + +IProcessor::Status PlanSquashingTransform::prepareConsume() +{ + if (available_inputs == 0) { - for (auto & in : inputs) - in.close(); - return Status::Finished; + planning_status = PlanningStatus::WAIT_IN; + return Status::NeedData; } + finished = false; - while (!chunk.hasChunkInfo()) + bool inputs_have_no_data = true; + for (auto & input : inputs) { - all_finished = true; - for (auto & input : inputs) + if (input.hasData()) { - if (!input.isFinished()) - all_finished = false; - } + inputs_have_no_data = false; + chunk = input.pull(); + transform(chunk); - if (all_finished) /// If all inputs are closed, we check if we have data in balancing - { - if (balance.isDataLeft()) /// If we have data in balancing, we process this data - { - finished = false; - transform(chunk); - has_data = true; - } - else /// If we don't have data, We send FINISHED + available_inputs--; + if (chunk.hasChunkInfo()) { - for (auto & output : outputs) - output.finish(); - - return Status::Finished; + planning_status = PlanningStatus::WAIT_OUT_AND_PUSH; + return Status::Ready; } } - for (auto & input : inputs) + if (available_inputs == 0) { - if (input.isFinished()) - continue; + planning_status = PlanningStatus::WAIT_IN; + return Status::NeedData; + } + } - input.setNeeded(); - if (!input.hasData()) - { - if (!balance.isDataLeft()) - return Status::NeedData; - else - continue; - } + if (inputs_have_no_data) + { + if (checkInputs()) + return Status::Ready; - chunk = input.pull(); + planning_status = PlanningStatus::WAIT_IN; + return Status::NeedData; + } + return Status::Ready; +} + +bool PlanSquashingTransform::checkInputs() +{ + bool all_finished = true; + + for (auto & output : outputs) + { + if (!output.isFinished()) + all_finished = false; + } + if (all_finished) /// If all outputs are closed, we close inputs (just in case) + { + planning_status = PlanningStatus::FINISH; + return true; + } + + all_finished = true; + for (auto & input : inputs) + { + + if (!input.isFinished()) + all_finished = false; + } + + if (all_finished) /// If all inputs are closed, we check if we have data in balancing + { + if (balance.isDataLeft()) /// If we have data in balancing, we process this data + { + planning_status = PlanningStatus::WAIT_OUT_FLUSH; + finished = false; transform(chunk); - was_output_processed.assign(outputs.size(), false); - if (chunk.hasChunkInfo()) - { - has_data = true; - return Status::Ready; - } - else - return Status::NeedData; } + else /// If we don't have data, We send FINISHED + planning_status = PlanningStatus::FINISH; + return true; } - return Status::Ready; + return false; +} + +IProcessor::Status PlanSquashingTransform::waitForDataIn() +{ + bool all_finished = true; + for (auto & input : inputs) + { + if (input.isFinished()) + continue; + + all_finished = false; + + if (!input.hasData()) + continue; + + available_inputs++; + } + if (all_finished) + { + checkInputs(); + return Status::Ready; + } + + if (available_inputs > 0) + { + planning_status = PlanningStatus::READ_IF_CAN; + return Status::Ready; + } + + return Status::NeedData; } void PlanSquashingTransform::transform(Chunk & chunk_) @@ -112,43 +200,47 @@ void PlanSquashingTransform::transform(Chunk & chunk_) IProcessor::Status PlanSquashingTransform::prepareSend() { - bool all_outputs_processed = true; + if (!chunk) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should be available in prepareSend"); - size_t chunk_number = 0; for (auto &output : outputs) { - auto & was_processed = was_output_processed[chunk_number]; - ++chunk_number; - if (!chunk.hasChunkInfo()) + if (output.canPush()) { - has_data = false; + planning_status = PlanningStatus::READ_IF_CAN; + output.push(std::move(chunk)); return Status::Ready; } + } + return Status::PortFull; +} - if (was_processed) - continue; +IProcessor::Status PlanSquashingTransform::prepareSendFlush() +{ + if (!chunk) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should be available in prepareSendFlush"); - if (output.isFinished()) - continue; + for (auto &output : outputs) + { - if (!output.canPush()) + if (output.canPush()) { - all_outputs_processed = false; - continue; + planning_status = PlanningStatus::FINISH; + output.push(std::move(chunk)); + return Status::Ready; } - - output.push(std::move(chunk)); - was_processed = true; - break; } + return Status::PortFull; +} - if (all_outputs_processed) - { - has_data = false; - return Status::Ready; - } +IProcessor::Status PlanSquashingTransform::finish() +{ + for (auto & in : inputs) + in.close(); + for (auto & output : outputs) + output.finish(); - return Status::PortFull; + return Status::Finished; } } diff --git a/src/Processors/Transforms/PlanSquashingTransform.h b/src/Processors/Transforms/PlanSquashingTransform.h index c30569fffb59..a500787ad0c2 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.h +++ b/src/Processors/Transforms/PlanSquashingTransform.h @@ -4,9 +4,23 @@ #include #include -namespace DB +enum PlanningStatus { + INIT, + READ_IF_CAN, + WAIT_IN, + PUSH, + WAIT_OUT_AND_PUSH, + WAIT_OUT_FLUSH, + FINISH +}; +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} class PlanSquashingTransform : public IProcessor { @@ -20,19 +34,23 @@ class PlanSquashingTransform : public IProcessor OutputPorts & getOutputPorts() { return outputs; } Status prepare() override; + Status init(); Status prepareConsume(); Status prepareSend(); + Status prepareSendFlush(); + Status waitForDataIn(); + Status finish(); + bool checkInputs(); void transform(Chunk & chunk); protected: private: - size_t CalculateBlockSize(const Block & block); Chunk chunk; PlanSquashing balance; - bool has_data = false; - std::vector was_output_processed; + PlanningStatus planning_status = PlanningStatus::INIT; + size_t available_inputs = 0; /// When consumption is finished we need to release the final chunk regardless of its size. bool finished = false; From 00f9355ede76b5cf5a207d5043201b0d6473f64e Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 13 May 2024 16:57:46 +0000 Subject: [PATCH 045/254] fix style + PUSH removal --- .../Transforms/PlanSquashingTransform.cpp | 22 ++++++++----------- .../Transforms/PlanSquashingTransform.h | 5 ----- 2 files changed, 9 insertions(+), 18 deletions(-) diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 89b59354722c..5125c28fb06c 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -5,6 +5,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + PlanSquashingTransform::PlanSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), balance(header, min_block_size_rows, min_block_size_bytes) { @@ -38,11 +43,6 @@ IProcessor::Status PlanSquashingTransform::prepare() status = prepareSend(); break; } - case PlanningStatus::PUSH: - { - status = prepareSend(); - break; - } case PlanningStatus::WAIT_OUT_FLUSH: { status = prepareSendFlush(); @@ -121,10 +121,9 @@ bool PlanSquashingTransform::checkInputs() bool all_finished = true; for (auto & output : outputs) - { if (!output.isFinished()) all_finished = false; - } + if (all_finished) /// If all outputs are closed, we close inputs (just in case) { planning_status = PlanningStatus::FINISH; @@ -133,11 +132,8 @@ bool PlanSquashingTransform::checkInputs() all_finished = true; for (auto & input : inputs) - { - if (!input.isFinished()) all_finished = false; - } if (all_finished) /// If all inputs are closed, we check if we have data in balancing { @@ -166,7 +162,7 @@ IProcessor::Status PlanSquashingTransform::waitForDataIn() if (!input.hasData()) continue; - + available_inputs++; } if (all_finished) @@ -174,13 +170,13 @@ IProcessor::Status PlanSquashingTransform::waitForDataIn() checkInputs(); return Status::Ready; } - + if (available_inputs > 0) { planning_status = PlanningStatus::READ_IF_CAN; return Status::Ready; } - + return Status::NeedData; } diff --git a/src/Processors/Transforms/PlanSquashingTransform.h b/src/Processors/Transforms/PlanSquashingTransform.h index a500787ad0c2..39f3a70a4a2d 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.h +++ b/src/Processors/Transforms/PlanSquashingTransform.h @@ -9,7 +9,6 @@ enum PlanningStatus INIT, READ_IF_CAN, WAIT_IN, - PUSH, WAIT_OUT_AND_PUSH, WAIT_OUT_FLUSH, FINISH @@ -17,10 +16,6 @@ enum PlanningStatus namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} class PlanSquashingTransform : public IProcessor { From 38e71274d58be6356e03d76189076ba5dc7a556a Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 14 May 2024 13:58:14 +0000 Subject: [PATCH 046/254] improve automata state transitions --- .../Transforms/PlanSquashingTransform.cpp | 46 +++++++++++-------- .../Transforms/PlanSquashingTransform.h | 1 + 2 files changed, 29 insertions(+), 18 deletions(-) diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 5125c28fb06c..5600c30b1ba7 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -19,7 +19,7 @@ IProcessor::Status PlanSquashingTransform::prepare() { Status status = Status::Ready; - while (status == Status::Ready) + while (planning_status != PlanningStatus::FINISH) { switch (planning_status) { @@ -119,18 +119,6 @@ IProcessor::Status PlanSquashingTransform::prepareConsume() bool PlanSquashingTransform::checkInputs() { bool all_finished = true; - - for (auto & output : outputs) - if (!output.isFinished()) - all_finished = false; - - if (all_finished) /// If all outputs are closed, we close inputs (just in case) - { - planning_status = PlanningStatus::FINISH; - return true; - } - - all_finished = true; for (auto & input : inputs) if (!input.isFinished()) all_finished = false; @@ -140,11 +128,27 @@ bool PlanSquashingTransform::checkInputs() if (balance.isDataLeft()) /// If we have data in balancing, we process this data { planning_status = PlanningStatus::WAIT_OUT_FLUSH; - finished = false; + finished = true; transform(chunk); } - else /// If we don't have data, We send FINISHED - planning_status = PlanningStatus::FINISH; + // else /// If we don't have data, We send FINISHED + // planning_status = PlanningStatus::FINISH; + return true; + } + return false; +} + +bool PlanSquashingTransform::checkOutputs() +{ + bool all_finished = true; + + for (auto & output : outputs) + if (!output.isFinished()) + all_finished = false; + + if (all_finished) /// If all outputs are closed, we close inputs (just in case) + { + planning_status = PlanningStatus::FINISH; return true; } return false; @@ -197,7 +201,10 @@ void PlanSquashingTransform::transform(Chunk & chunk_) IProcessor::Status PlanSquashingTransform::prepareSend() { if (!chunk) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should be available in prepareSend"); + { + planning_status = PlanningStatus::FINISH; + return Status::Ready; + } for (auto &output : outputs) { @@ -215,7 +222,10 @@ IProcessor::Status PlanSquashingTransform::prepareSend() IProcessor::Status PlanSquashingTransform::prepareSendFlush() { if (!chunk) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should be available in prepareSendFlush"); + { + planning_status = PlanningStatus::FINISH; + return Status::Ready; + } for (auto &output : outputs) { diff --git a/src/Processors/Transforms/PlanSquashingTransform.h b/src/Processors/Transforms/PlanSquashingTransform.h index 39f3a70a4a2d..57c772748632 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.h +++ b/src/Processors/Transforms/PlanSquashingTransform.h @@ -37,6 +37,7 @@ class PlanSquashingTransform : public IProcessor Status finish(); bool checkInputs(); + bool checkOutputs(); void transform(Chunk & chunk); protected: From a8a2aa21b289bc6467f45f92ba3b7b76ebd172cc Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 14 May 2024 16:03:05 +0000 Subject: [PATCH 047/254] change automata algorithm --- .../Transforms/PlanSquashingTransform.cpp | 61 +++++++++++-------- .../Transforms/PlanSquashingTransform.h | 7 ++- 2 files changed, 43 insertions(+), 25 deletions(-) diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 5600c30b1ba7..95d3b454a4cb 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -23,42 +23,41 @@ IProcessor::Status PlanSquashingTransform::prepare() { switch (planning_status) { - case PlanningStatus::INIT: + case INIT: { status = init(); break; } - case PlanningStatus::READ_IF_CAN: + case READ_IF_CAN: { status = prepareConsume(); break; } - case PlanningStatus::WAIT_IN: + case PUSH: { - status = waitForDataIn(); - break; - } - case PlanningStatus::WAIT_OUT_AND_PUSH: - { - status = prepareSend(); - break; - } - case PlanningStatus::WAIT_OUT_FLUSH: - { - status = prepareSendFlush(); + status = push(); break; } + case WAIT_IN: + return waitForDataIn(); + case WAIT_OUT: + return prepareSend(); + case WAIT_OUT_FLUSH: + return prepareSendFlush(); case FINISH: - { - status = finish(); - break; - } + break; /// never reached } } + status = finish(); return status; } +void PlanSquashingTransform::work() +{ + prepare(); +} + IProcessor::Status PlanSquashingTransform::init() { for (auto input : inputs) @@ -93,7 +92,7 @@ IProcessor::Status PlanSquashingTransform::prepareConsume() available_inputs--; if (chunk.hasChunkInfo()) { - planning_status = PlanningStatus::WAIT_OUT_AND_PUSH; + planning_status = PlanningStatus::WAIT_OUT; return Status::Ready; } } @@ -198,6 +197,21 @@ void PlanSquashingTransform::transform(Chunk & chunk_) } } +IProcessor::Status PlanSquashingTransform::push() +{ + if (!free_output) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There should be a free output in push()"); + + if (finished) + planning_status = PlanningStatus::FINISH; + else + planning_status = PlanningStatus::READ_IF_CAN; + + free_output->push(std::move(chunk)); + free_output = nullptr; + return Status::Ready; +} + IProcessor::Status PlanSquashingTransform::prepareSend() { if (!chunk) @@ -208,11 +222,10 @@ IProcessor::Status PlanSquashingTransform::prepareSend() for (auto &output : outputs) { - if (output.canPush()) { - planning_status = PlanningStatus::READ_IF_CAN; - output.push(std::move(chunk)); + planning_status = PlanningStatus::PUSH; + free_output = &output; return Status::Ready; } } @@ -232,8 +245,8 @@ IProcessor::Status PlanSquashingTransform::prepareSendFlush() if (output.canPush()) { - planning_status = PlanningStatus::FINISH; - output.push(std::move(chunk)); + planning_status = PlanningStatus::PUSH; + free_output = &output; return Status::Ready; } } diff --git a/src/Processors/Transforms/PlanSquashingTransform.h b/src/Processors/Transforms/PlanSquashingTransform.h index 57c772748632..55685b0c532f 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.h +++ b/src/Processors/Transforms/PlanSquashingTransform.h @@ -3,13 +3,15 @@ #include #include #include +#include "Processors/Port.h" enum PlanningStatus { INIT, READ_IF_CAN, WAIT_IN, - WAIT_OUT_AND_PUSH, + WAIT_OUT, + PUSH, WAIT_OUT_FLUSH, FINISH }; @@ -29,9 +31,11 @@ class PlanSquashingTransform : public IProcessor OutputPorts & getOutputPorts() { return outputs; } Status prepare() override; + void work() override; Status init(); Status prepareConsume(); Status prepareSend(); + Status push(); Status prepareSendFlush(); Status waitForDataIn(); Status finish(); @@ -47,6 +51,7 @@ class PlanSquashingTransform : public IProcessor PlanSquashing balance; PlanningStatus planning_status = PlanningStatus::INIT; size_t available_inputs = 0; + OutputPort* free_output = nullptr; /// When consumption is finished we need to release the final chunk regardless of its size. bool finished = false; From 0619b0921f195951b8e72c02dcc0ad06094811b2 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 15 May 2024 15:56:24 +0000 Subject: [PATCH 048/254] removed memory from automata, refactored the code --- src/Interpreters/Squashing.cpp | 10 +- src/Interpreters/Squashing.h | 1 + .../Transforms/PlanSquashingTransform.cpp | 166 ++++-------------- .../Transforms/PlanSquashingTransform.h | 19 +- 4 files changed, 49 insertions(+), 147 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 78d1b9fc6437..2d87b47798c7 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -198,6 +198,11 @@ PlanSquashing::PlanSquashing(Block header_, size_t min_block_size_rows_, size_t { } +Chunk PlanSquashing::flush() +{ + return convertToChunk(chunks_to_merge_vec); +} + Chunk PlanSquashing::add(Chunk && input_chunk) { return addImpl(std::move(input_chunk)); @@ -206,10 +211,7 @@ Chunk PlanSquashing::add(Chunk && input_chunk) Chunk PlanSquashing::addImpl(Chunk && input_chunk) { if (!input_chunk) - { - Chunk res_chunk = convertToChunk(chunks_to_merge_vec); - return res_chunk; - } + return {}; if (isEnoughSize(chunks_to_merge_vec)) chunks_to_merge_vec.clear(); diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index 82d7fe616f6b..0bb6acf93724 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -85,6 +85,7 @@ class PlanSquashing PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); Chunk add(Chunk && input_chunk); + Chunk flush(); bool isDataLeft() { return !chunks_to_merge_vec.empty(); diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 95d3b454a4cb..9c42b846a7b0 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -1,5 +1,6 @@ #include #include +#include "Common/logger_useful.h" #include namespace DB @@ -24,26 +25,18 @@ IProcessor::Status PlanSquashingTransform::prepare() switch (planning_status) { case INIT: - { - status = init(); + init(); break; - } case READ_IF_CAN: - { status = prepareConsume(); break; - } - case PUSH: - { - status = push(); - break; - } case WAIT_IN: - return waitForDataIn(); - case WAIT_OUT: - return prepareSend(); - case WAIT_OUT_FLUSH: - return prepareSendFlush(); + planning_status = PlanningStatus::READ_IF_CAN; + return Status::NeedData; + case PUSH: + return sendOrFlush(); + case FLUSH: + return sendOrFlush(); case FINISH: break; /// never reached } @@ -58,104 +51,58 @@ void PlanSquashingTransform::work() prepare(); } -IProcessor::Status PlanSquashingTransform::init() +void PlanSquashingTransform::init() { - for (auto input : inputs) - { - input.setNeeded(); - if (input.hasData()) - available_inputs++; - } + for (auto input: inputs) + if (!input.isFinished()) + input.setNeeded(); planning_status = PlanningStatus::READ_IF_CAN; - return Status::Ready; } IProcessor::Status PlanSquashingTransform::prepareConsume() { - if (available_inputs == 0) - { - planning_status = PlanningStatus::WAIT_IN; - return Status::NeedData; - } - finished = false; - - bool inputs_have_no_data = true; + bool inputs_have_no_data = true, all_finished = true; for (auto & input : inputs) { + if (!input.isFinished()) + all_finished = false; + if (input.hasData()) { inputs_have_no_data = false; chunk = input.pull(); transform(chunk); - available_inputs--; if (chunk.hasChunkInfo()) { - planning_status = PlanningStatus::WAIT_OUT; + planning_status = PlanningStatus::PUSH; return Status::Ready; } } - - if (available_inputs == 0) - { - planning_status = PlanningStatus::WAIT_IN; - return Status::NeedData; - } } - if (inputs_have_no_data) - { - if (checkInputs()) - return Status::Ready; - - planning_status = PlanningStatus::WAIT_IN; - return Status::NeedData; - } - return Status::Ready; -} - -bool PlanSquashingTransform::checkInputs() -{ - bool all_finished = true; - for (auto & input : inputs) - if (!input.isFinished()) - all_finished = false; - if (all_finished) /// If all inputs are closed, we check if we have data in balancing { if (balance.isDataLeft()) /// If we have data in balancing, we process this data { - planning_status = PlanningStatus::WAIT_OUT_FLUSH; - finished = true; - transform(chunk); + planning_status = PlanningStatus::FLUSH; + flushChunk(); } - // else /// If we don't have data, We send FINISHED - // planning_status = PlanningStatus::FINISH; - return true; + planning_status = PlanningStatus::PUSH; + return Status::Ready; } - return false; -} -bool PlanSquashingTransform::checkOutputs() -{ - bool all_finished = true; - - for (auto & output : outputs) - if (!output.isFinished()) - all_finished = false; + if (inputs_have_no_data) + planning_status = PlanningStatus::WAIT_IN; - if (all_finished) /// If all outputs are closed, we close inputs (just in case) - { - planning_status = PlanningStatus::FINISH; - return true; - } - return false; + return Status::Ready; } IProcessor::Status PlanSquashingTransform::waitForDataIn() { bool all_finished = true; + bool inputs_have_no_data = true; for (auto & input : inputs) { if (input.isFinished()) @@ -163,18 +110,17 @@ IProcessor::Status PlanSquashingTransform::waitForDataIn() all_finished = false; - if (!input.hasData()) - continue; + if (input.hasData()) + inputs_have_no_data = false; - available_inputs++; } if (all_finished) { - checkInputs(); + planning_status = PlanningStatus::READ_IF_CAN; return Status::Ready; } - if (available_inputs > 0) + if (!inputs_have_no_data) { planning_status = PlanningStatus::READ_IF_CAN; return Status::Ready; @@ -185,34 +131,17 @@ IProcessor::Status PlanSquashingTransform::waitForDataIn() void PlanSquashingTransform::transform(Chunk & chunk_) { - if (!finished) - { - Chunk res_chunk = balance.add(std::move(chunk_)); - std::swap(res_chunk, chunk_); - } - else - { - Chunk res_chunk = balance.add({}); - std::swap(res_chunk, chunk_); - } + Chunk res_chunk = balance.add(std::move(chunk_)); + std::swap(res_chunk, chunk_); } -IProcessor::Status PlanSquashingTransform::push() +void PlanSquashingTransform::flushChunk() { - if (!free_output) - throw Exception(ErrorCodes::LOGICAL_ERROR, "There should be a free output in push()"); - - if (finished) - planning_status = PlanningStatus::FINISH; - else - planning_status = PlanningStatus::READ_IF_CAN; - - free_output->push(std::move(chunk)); - free_output = nullptr; - return Status::Ready; + Chunk res_chunk = balance.flush(); + std::swap(res_chunk, chunk); } -IProcessor::Status PlanSquashingTransform::prepareSend() +IProcessor::Status PlanSquashingTransform::sendOrFlush() { if (!chunk) { @@ -224,29 +153,10 @@ IProcessor::Status PlanSquashingTransform::prepareSend() { if (output.canPush()) { - planning_status = PlanningStatus::PUSH; - free_output = &output; - return Status::Ready; - } - } - return Status::PortFull; -} + if (planning_status == PlanningStatus::PUSH) + planning_status = PlanningStatus::READ_IF_CAN; -IProcessor::Status PlanSquashingTransform::prepareSendFlush() -{ - if (!chunk) - { - planning_status = PlanningStatus::FINISH; - return Status::Ready; - } - - for (auto &output : outputs) - { - - if (output.canPush()) - { - planning_status = PlanningStatus::PUSH; - free_output = &output; + output.push(std::move(chunk)); return Status::Ready; } } diff --git a/src/Processors/Transforms/PlanSquashingTransform.h b/src/Processors/Transforms/PlanSquashingTransform.h index 55685b0c532f..dc5b6d669b1d 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.h +++ b/src/Processors/Transforms/PlanSquashingTransform.h @@ -10,9 +10,8 @@ enum PlanningStatus INIT, READ_IF_CAN, WAIT_IN, - WAIT_OUT, PUSH, - WAIT_OUT_FLUSH, + FLUSH, FINISH }; @@ -32,29 +31,19 @@ class PlanSquashingTransform : public IProcessor Status prepare() override; void work() override; - Status init(); + void init(); Status prepareConsume(); - Status prepareSend(); - Status push(); - Status prepareSendFlush(); + Status sendOrFlush(); Status waitForDataIn(); Status finish(); - bool checkInputs(); - bool checkOutputs(); void transform(Chunk & chunk); - -protected: + void flushChunk(); private: Chunk chunk; PlanSquashing balance; PlanningStatus planning_status = PlanningStatus::INIT; - size_t available_inputs = 0; - OutputPort* free_output = nullptr; - - /// When consumption is finished we need to release the final chunk regardless of its size. - bool finished = false; }; } From 04dd58430a75871d742ba8b424023307108eac10 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 15 May 2024 18:09:33 +0200 Subject: [PATCH 049/254] style fix --- src/Processors/Transforms/PlanSquashingTransform.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 9c42b846a7b0..f41062044626 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -6,11 +6,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - PlanSquashingTransform::PlanSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), balance(header, min_block_size_rows, min_block_size_bytes) { From fed6c65858f26e31ad8f3d63a2cb6e9a0b404ff7 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 16 May 2024 17:57:01 +0000 Subject: [PATCH 050/254] add setting to enable planning --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterInsertQuery.cpp | 66 ++++++++++++------ .../Transforms/buildPushingToViewsChain.cpp | 30 ++++++--- src/Server/TCPHandler.cpp | 67 +++++++++++++++++++ src/Storages/ProjectionsDescription.cpp | 11 ++- 5 files changed, 143 insertions(+), 32 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4a0de354a034..393d8202d05c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -668,6 +668,7 @@ class IColumn; M(Bool, mutations_execute_nondeterministic_on_initiator, false, "If true nondeterministic function are executed on initiator and replaced to literals in UPDATE and DELETE queries", 0) \ M(Bool, mutations_execute_subqueries_on_initiator, false, "If true scalar subqueries are executed on initiator and replaced to literals in UPDATE and DELETE queries", 0) \ M(UInt64, mutations_max_literal_size_to_replace, 16384, "The maximum size of serialized literal in bytes to replace in UPDATE and DELETE queries", 0) \ + M(Bool, allow_insert_threads_reduction_optimizaion, false, "If true it allows to apply additional single-insert-transformer for insertion of data", 0) \ \ M(Float, create_replicated_merge_tree_fault_injection_probability, 0.0f, "The probability of a fault injection during table creation after creating metadata in ZooKeeper", 0) \ \ diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 55f84080b13c..f0340bd1f480 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -620,19 +620,32 @@ BlockIO InterpreterInsertQuery::execute() { bool table_prefers_large_blocks = table->prefersLargeBlocks(); - pipeline.addTransform(std::make_shared( - header, - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, - presink_chains.size())); + if (settings.allow_insert_threads_reduction_optimizaion) + { + pipeline.addTransform(std::make_shared( + header, + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, + presink_chains.size())); - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared( + in_header, + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); + }); + } + else { - return std::make_shared( - in_header, - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); - }); + pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared( + in_header, + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); + }); + } } size_t num_select_threads = pipeline.getNumThreads(); @@ -684,20 +697,33 @@ BlockIO InterpreterInsertQuery::execute() { bool table_prefers_large_blocks = table->prefersLargeBlocks(); - auto squashing = std::make_shared( - chain.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); + if (settings.allow_insert_threads_reduction_optimizaion) + { + auto squashing = std::make_shared( + chain.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); - chain.addSource(std::move(squashing)); + chain.addSource(std::move(squashing)); - auto balancing = std::make_shared( + auto balancing = std::make_shared( + chain.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, + presink_chains.size()); + + chain.addSource(std::move(balancing)); + } + else + { + auto squashing = std::make_shared( chain.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, - presink_chains.size()); + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); - chain.addSource(std::move(balancing)); + chain.addSource(std::move(squashing)); + } + } auto context_ptr = getContext(); diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index e7c831c3e0e5..1d0098f0cd96 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -372,16 +372,26 @@ std::optional generateViewChain( bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); const auto & settings = insert_context->getSettingsRef(); - out.addSource(std::make_shared( - out.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); - - out.addSource(std::make_shared( - out.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, - 1)); // Chain requires a single input + if (settings.allow_insert_threads_reduction_optimizaion) + { + out.addSource(std::make_shared( + out.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + + out.addSource(std::make_shared( + out.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, + 1)); // Chain requires a single input + } + else + { + out.addSource(std::make_shared( + out.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + } } auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index be3c1384501e..56c97d0305e8 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -42,6 +43,7 @@ #include #include #include +#include #include #include @@ -181,6 +183,7 @@ void validateClientInfo(const ClientInfo & session_client_info, const ClientInfo namespace DB { +using Which = Field::Types::Which; TCPHandler::TCPHandler( IServer & server_, @@ -1602,6 +1605,70 @@ void TCPHandler::sendHello() nonce.emplace(thread_local_rng()); writeIntBinary(nonce.value(), *out); } + + /// If client is Clickhouse-client we will send server profile settings of this user + if (client_name == (std::string(VERSION_NAME) + " client")) + { + const auto & user = session->sessionContext()->getUser(); + String query = fmt::format( + R"(SELECT setting_name, value FROM system.settings_profile_elements WHERE user_name = '{0}')", + escapeString(user->getName())); + const auto & res_const = executeQuery(query,server.context(), QueryFlags{ .internal = true }).second; + auto & res = const_cast(res_const); + PullingPipelineExecutor pulling_executor(res.pipeline); + Block block; + pulling_executor.pull(block); + /// filter data + std::map server_settings; + for (size_t row = 0; row < block.rows(); ++row) + { + size_t col_index = 0; + String name; + Field value_field; + for (const auto & name_value: block) + { + Field field; + name_value.column->get(row, field); + if (!field.isNull()) + { + if (col_index == 0) + name = field.safeGet(); + else + value_field = field; + } + else + continue; + + ++col_index; + } + if (!name.empty()) + server_settings[name] = value_field; + + } + + writeVarUInt(server_settings.size(), *out); + if (!server_settings.empty()) + { + for (const auto & setting : server_settings) + { + writeStringBinary(setting.first, *out); + writeVarUInt(setting.second.getType(), *out); + switch (setting.second.getType()) + { + case Which::UInt64: + writeVarUInt(setting.second.safeGet(), *out);break; + case Which::String: + writeStringBinary(setting.second.safeGet(), *out);break; + case Which::Bool: + writeVarUInt(setting.second.get(), *out);break; + default: + break; + } + + } + } + } + out->next(); } diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index b31cc1e94f14..8d28d68dc398 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -311,8 +311,15 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) builder.resize(1); // Generate aggregated blocks with rows less or equal than the original block. // There should be only one output block after this transformation. - builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, 1)); - builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); + if (mut_context->getSettings().allow_insert_threads_reduction_optimizaion) + { + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, 1)); + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); + } + else + { + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); + } auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); From 1f7198b3d3576de29485cd7b96bbc9bf97d181bb Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 17 May 2024 12:15:58 +0000 Subject: [PATCH 051/254] style fix + resize optimization for merging columns --- src/Interpreters/InterpreterInsertQuery.cpp | 4 +- src/Interpreters/Squashing.cpp | 63 ++++++++++++--------- src/Interpreters/Squashing.h | 7 +-- 3 files changed, 41 insertions(+), 33 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index f0340bd1f480..3a6329997bda 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -711,7 +711,7 @@ BlockIO InterpreterInsertQuery::execute() table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, presink_chains.size()); - + chain.addSource(std::move(balancing)); } else @@ -723,7 +723,7 @@ BlockIO InterpreterInsertQuery::execute() chain.addSource(std::move(squashing)); } - + } auto context_ptr = getContext(); diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 2d87b47798c7..526af3db2e40 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -1,5 +1,8 @@ +#include #include #include +#include "Core/Block.h" +#include "Core/ColumnsWithTypeAndName.h" namespace DB @@ -128,10 +131,8 @@ bool Squashing::isEnoughSize(size_t rows, size_t bytes) const || (min_block_size_bytes && bytes >= min_block_size_bytes); } -ApplySquashing::ApplySquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) - : min_block_size_rows(min_block_size_rows_) - , min_block_size_bytes(min_block_size_bytes_) - , header(std::move(header_)) +ApplySquashing::ApplySquashing(Block header_) + : header(std::move(header_)) { } @@ -146,37 +147,47 @@ Block ApplySquashing::addImpl(Chunk && input_chunk) return Block(); const auto *info = getInfoFromChunk(input_chunk); - for (auto & chunk : info->chunks) - append(chunk.clone()); + append(info->chunks); - { - Block to_return; - std::swap(to_return, accumulated_block); - return to_return; - } + Block to_return; + std::swap(to_return, accumulated_block); + return to_return; } -void ApplySquashing::append(Chunk && input_chunk) +void ApplySquashing::append(const std::vector & input_chunks) { - if (input_chunk.getNumColumns() == 0) - return; - if (!accumulated_block) + std::vector mutable_columns; + size_t rows = 0; + for (const Chunk & chunk : input_chunks) + rows += chunk.getNumRows(); + + // add here resize of mutable_column + for (const auto & input_chunk : input_chunks) { - for (size_t i = 0; i < input_chunk.getNumColumns(); ++ i) + if (!accumulated_block) { - ColumnWithTypeAndName col = ColumnWithTypeAndName(input_chunk.getColumns()[i], header.getDataTypes()[i], header.getNames()[i]); - accumulated_block.insert(accumulated_block.columns(), col); + for (size_t i = 0; i < input_chunks[0].getNumColumns(); ++ i) + { + ColumnWithTypeAndName col = ColumnWithTypeAndName(input_chunks[0].getColumns()[i], header.getDataTypes()[i], header.getNames()[i]); + mutable_columns.push_back(IColumn::mutate(col.column)); + accumulated_block.insert(col); + } } - return; - } - for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) - { - const auto source_column = input_chunk.getColumns()[i]; + if (input_chunk.getNumColumns() == 0) + continue; - auto mutable_column = IColumn::mutate(std::move(accumulated_block.getByPosition(i).column)); - mutable_column->insertRangeFrom(*source_column, 0, source_column->size()); - accumulated_block.getByPosition(i).column = std::move(mutable_column); + for (auto & column : mutable_columns) + column->reserve(rows); + + for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) + { + const auto source_column = input_chunk.getColumns()[i]; + + mutable_columns[i] = IColumn::mutate(std::move(accumulated_block.getByPosition(i).column)); + mutable_columns[i]->insertRangeFrom(*source_column, 0, source_column->size()); + accumulated_block.getByPosition(i).column = mutable_columns[i]->cloneFinalized(); + } } } diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index 0bb6acf93724..a68b581d40af 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -58,14 +58,11 @@ class Squashing class ApplySquashing { public: - ApplySquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); + explicit ApplySquashing(Block header_); Block add(Chunk && input_chunk); private: - size_t min_block_size_rows; - size_t min_block_size_bytes; - Block accumulated_block; const Block header; @@ -73,7 +70,7 @@ class ApplySquashing const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); - void append(Chunk && input_chunk); + void append(const std::vector & input_chunk); bool isEnoughSize(const Block & block); bool isEnoughSize(size_t rows, size_t bytes) const; From 8d235a4a399b3489ff3a8672134c8905511562a3 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 17 May 2024 15:11:21 +0000 Subject: [PATCH 052/254] remove trash from the code --- src/Server/TCPHandler.cpp | 65 --------------------------------------- 1 file changed, 65 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 56c97d0305e8..3660b4c11875 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -43,7 +43,6 @@ #include #include #include -#include #include #include @@ -183,7 +182,6 @@ void validateClientInfo(const ClientInfo & session_client_info, const ClientInfo namespace DB { -using Which = Field::Types::Which; TCPHandler::TCPHandler( IServer & server_, @@ -1606,69 +1604,6 @@ void TCPHandler::sendHello() writeIntBinary(nonce.value(), *out); } - /// If client is Clickhouse-client we will send server profile settings of this user - if (client_name == (std::string(VERSION_NAME) + " client")) - { - const auto & user = session->sessionContext()->getUser(); - String query = fmt::format( - R"(SELECT setting_name, value FROM system.settings_profile_elements WHERE user_name = '{0}')", - escapeString(user->getName())); - const auto & res_const = executeQuery(query,server.context(), QueryFlags{ .internal = true }).second; - auto & res = const_cast(res_const); - PullingPipelineExecutor pulling_executor(res.pipeline); - Block block; - pulling_executor.pull(block); - /// filter data - std::map server_settings; - for (size_t row = 0; row < block.rows(); ++row) - { - size_t col_index = 0; - String name; - Field value_field; - for (const auto & name_value: block) - { - Field field; - name_value.column->get(row, field); - if (!field.isNull()) - { - if (col_index == 0) - name = field.safeGet(); - else - value_field = field; - } - else - continue; - - ++col_index; - } - if (!name.empty()) - server_settings[name] = value_field; - - } - - writeVarUInt(server_settings.size(), *out); - if (!server_settings.empty()) - { - for (const auto & setting : server_settings) - { - writeStringBinary(setting.first, *out); - writeVarUInt(setting.second.getType(), *out); - switch (setting.second.getType()) - { - case Which::UInt64: - writeVarUInt(setting.second.safeGet(), *out);break; - case Which::String: - writeStringBinary(setting.second.safeGet(), *out);break; - case Which::Bool: - writeVarUInt(setting.second.get(), *out);break; - default: - break; - } - - } - } - } - out->next(); } From 6b835522b3ab12ffc6b210b61c11bb49f4fab918 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 17 May 2024 15:22:32 +0000 Subject: [PATCH 053/254] fix build --- src/Processors/Transforms/ApplySquashingTransform.h | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/ApplySquashingTransform.h b/src/Processors/Transforms/ApplySquashingTransform.h index 584fb72cccb0..abb3a0aad415 100644 --- a/src/Processors/Transforms/ApplySquashingTransform.h +++ b/src/Processors/Transforms/ApplySquashingTransform.h @@ -9,10 +9,9 @@ namespace DB class ApplySquashingTransform : public ExceptionKeepingTransform { public: - ApplySquashingTransform( - const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) + explicit ApplySquashingTransform(const Block & header) : ExceptionKeepingTransform(header, header, false) - , squashing(header, min_block_size_rows, min_block_size_bytes) + , squashing(header) { } From ef9bfbd85a0fa1dbd387f5fa3869be8d2614bb70 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 17 May 2024 15:58:40 +0000 Subject: [PATCH 054/254] fix build --- src/Interpreters/InterpreterInsertQuery.cpp | 10 ++-------- src/Processors/Transforms/buildPushingToViewsChain.cpp | 10 ++-------- src/Storages/ProjectionsDescription.cpp | 2 +- 3 files changed, 5 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 3a6329997bda..47a0567dfec8 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -630,10 +630,7 @@ BlockIO InterpreterInsertQuery::execute() pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { - return std::make_shared( - in_header, - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); + return std::make_shared(in_header); }); } else @@ -699,10 +696,7 @@ BlockIO InterpreterInsertQuery::execute() if (settings.allow_insert_threads_reduction_optimizaion) { - auto squashing = std::make_shared( - chain.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); + auto squashing = std::make_shared(chain.getInputHeader()); chain.addSource(std::move(squashing)); diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 1d0098f0cd96..4e7038285547 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -374,10 +374,7 @@ std::optional generateViewChain( if (settings.allow_insert_threads_reduction_optimizaion) { - out.addSource(std::make_shared( - out.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + out.addSource(std::make_shared(out.getInputHeader())); out.addSource(std::make_shared( out.getInputHeader(), @@ -643,10 +640,7 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat context->getSettingsRef().min_insert_block_size_rows, context->getSettingsRef().min_insert_block_size_bytes, pipeline.getNumStreams())); - pipeline.addTransform(std::make_shared( - pipeline.getHeader(), - context->getSettingsRef().min_insert_block_size_rows, - context->getSettingsRef().min_insert_block_size_bytes)); + pipeline.addTransform(std::make_shared(pipeline.getHeader())); auto converting = ActionsDAG::makeConvertingActions( pipeline.getHeader().getColumnsWithTypeAndName(), diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 8d28d68dc398..45add4332ff2 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -314,7 +314,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) if (mut_context->getSettings().allow_insert_threads_reduction_optimizaion) { builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, 1)); - builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); + builder.addTransform(std::make_shared(builder.getHeader())); } else { From 96e19ac5486511d668b451e6541361d8754a4a67 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 17 May 2024 17:10:40 +0000 Subject: [PATCH 055/254] add merge_workload and mutation_workload settings for server and merge tree --- programs/server/Server.cpp | 4 ++ programs/server/config.xml | 8 ++++ src/Core/ServerSettings.h | 2 + src/Interpreters/Context.cpp | 41 +++++++++++++++++++ src/Interpreters/Context.h | 6 +++ .../MergeTree/MergeFromLogEntryTask.cpp | 2 +- .../MergeTree/MergePlainMergeTreeTask.cpp | 2 +- src/Storages/MergeTree/MergeTask.h | 8 ++-- src/Storages/MergeTree/MergeTreeSettings.h | 2 + .../MergeTree/MutateFromLogEntryTask.cpp | 2 +- .../MergeTree/MutatePlainMergeTreeTask.cpp | 2 +- 11 files changed, 71 insertions(+), 8 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index bcfe3799818f..8ad9329c67c4 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1557,6 +1557,10 @@ try 0, // We don't need any threads one all the parts will be deleted new_server_settings.max_parts_cleaning_thread_pool_size); + + global_context->setMergeWorkload(new_server_settings.merge_workload); + global_context->setMutationWorkload(new_server_settings.mutation_workload); + if (config->has("resources")) { global_context->getResourceManager()->updateConfiguration(*config); diff --git a/programs/server/config.xml b/programs/server/config.xml index 27ed5952fc9d..ad920230036c 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1396,6 +1396,14 @@ + + + + diff --git a/tests/integration/test_scheduler/configs/resources.xml.default b/tests/integration/test_scheduler/configs/resources.xml.default new file mode 100644 index 000000000000..3b003a175575 --- /dev/null +++ b/tests/integration/test_scheduler/configs/resources.xml.default @@ -0,0 +1,76 @@ + + + + inflight_limit1000000 + priority + fifo0 + fair1 + fifo9 + fifo1 + fair90 + fifo + fifo + fifo9 + fifo9 + fifo9 + fifo9 + + + inflight_limit1000000 + priority + fifo0 + fair1 + fifo9 + fifo1 + fair90 + fifo + fifo + fifo9 + fifo9 + fifo9 + fifo9 + + + + + /prio/admin + /prio/admin + + + /prio/fair/prod + /prio/fair/prod + + + /prio/fair/dev + /prio/fair/dev + + + /prio/fair/dev + /prio/fair/dev + + + /prio/fair/sys/merges + /prio/fair/sys/merges + + + /prio/fair/sys/mutations + /prio/fair/sys/mutations + + + /prio/fair/prod_merges + /prio/fair/prod_merges + + + /prio/fair/prod_mutations + /prio/fair/prod_mutations + + + /prio/fair/dev_merges + /prio/fair/dev_merges + + + /prio/fair/dev_mutations + /prio/fair/dev_mutations + + + diff --git a/tests/integration/test_scheduler/configs/scheduler.xml b/tests/integration/test_scheduler/configs/scheduler.xml deleted file mode 100644 index 523ba1a5a987..000000000000 --- a/tests/integration/test_scheduler/configs/scheduler.xml +++ /dev/null @@ -1,62 +0,0 @@ - - - - - s3 - http://minio1:9001/root/data/ - minio - minio123 - 33554432 - 10 - 10 - network_read - network_write - - - - - -
- s3 -
-
-
-
-
- - - inflight_limit1000000 - priority - fifo0 - fair1 - fifo9 - fifo1 - - - inflight_limit1000000 - priority - fifo0 - fair1 - fifo9 - fifo1 - - - - - /prio/admin - /prio/admin - - - /prio/fair/prod - /prio/fair/prod - - - /prio/fair/dev - /prio/fair/dev - - - /prio/fair/dev - /prio/fair/dev - - -
diff --git a/tests/integration/test_scheduler/configs/storage_configuration.xml b/tests/integration/test_scheduler/configs/storage_configuration.xml new file mode 100644 index 000000000000..823a00a05deb --- /dev/null +++ b/tests/integration/test_scheduler/configs/storage_configuration.xml @@ -0,0 +1,26 @@ + + + + + s3 + http://minio1:9001/root/data/ + minio + minio123 + 33554432 + 10 + 10 + network_read + network_write + + + + + +
+ s3 +
+
+
+
+
+
diff --git a/tests/integration/test_scheduler/configs/workloads.xml b/tests/integration/test_scheduler/configs/workloads.xml new file mode 100644 index 000000000000..197bf660500e --- /dev/null +++ b/tests/integration/test_scheduler/configs/workloads.xml @@ -0,0 +1,3 @@ + + + diff --git a/tests/integration/test_scheduler/configs/workloads.xml.default b/tests/integration/test_scheduler/configs/workloads.xml.default new file mode 100644 index 000000000000..f010993335de --- /dev/null +++ b/tests/integration/test_scheduler/configs/workloads.xml.default @@ -0,0 +1,4 @@ + + sys_merges + sys_mutations + diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index e6def99c076b..e0660c036815 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -13,7 +13,13 @@ node = cluster.add_instance( "node", stay_alive=True, - main_configs=["configs/scheduler.xml"], + main_configs=[ + "configs/storage_configuration.xml", + "configs/resources.xml", + "configs/resources.xml.default", + "configs/workloads.xml", + "configs/workloads.xml.default", + ], with_minio=True, ) @@ -27,6 +33,41 @@ def start_cluster(): cluster.shutdown() +@pytest.fixture(scope="function", autouse=True) +def set_default_configs(): + node.exec_in_container( + [ + "bash", + "-c", + "cp /etc/clickhouse-server/config.d/resources.xml.default /etc/clickhouse-server/config.d/resources.xml", + ] + ) + node.exec_in_container( + [ + "bash", + "-c", + "cp /etc/clickhouse-server/config.d/workloads.xml.default /etc/clickhouse-server/config.d/workloads.xml", + ] + ) + node.query("system reload config") + yield + + +def update_workloads_config(**settings): + xml='' + for name in settings: + xml += f"<{name}>{settings[name]}" + print(xml) + node.exec_in_container( + [ + "bash", + "-c", + f"echo '{xml}' > /etc/clickhouse-server/config.d/workloads.xml", + ] + ) + node.query("system reload config") + + def test_s3_disk(): node.query( f""" @@ -110,3 +151,175 @@ def read_query(workload): ) == "1\n" ) + + +def test_merge_workload(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'").strip()) + writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'").strip()) + + node.query(f"insert into data select * from numbers(1e4)") + node.query(f"insert into data select * from numbers(2e4)") + node.query(f"insert into data select * from numbers(3e4)") + node.query(f"optimize table data final") + + reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'").strip()) + writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'").strip()) + + assert (reads_before < reads_after) + assert (writes_before < writes_after) + + +def test_merge_workload_override(): + node.query( + f""" + drop table if exists prod_data; + drop table if exists dev_data; + create table prod_data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3', merge_workload='prod_merges'; + create table dev_data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3', merge_workload='dev_merges'; + """ + ) + + prod_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'").strip()) + prod_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'").strip()) + dev_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'").strip()) + dev_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'").strip()) + + node.query(f"insert into prod_data select * from numbers(1e4)") + node.query(f"insert into prod_data select * from numbers(2e4)") + node.query(f"insert into prod_data select * from numbers(3e4)") + node.query(f"insert into dev_data select * from numbers(1e4)") + node.query(f"insert into dev_data select * from numbers(2e4)") + node.query(f"insert into dev_data select * from numbers(3e4)") + node.query(f"optimize table prod_data final") + node.query(f"optimize table dev_data final") + + prod_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'").strip()) + prod_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'").strip()) + dev_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'").strip()) + dev_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'").strip()) + + assert (prod_reads_before < prod_reads_after) + assert (prod_writes_before < prod_writes_after) + assert (dev_reads_before < dev_reads_after) + assert (dev_writes_before < dev_writes_after) + + +def test_mutate_workload(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + node.query(f"insert into data select * from numbers(1e4)") + node.query(f"optimize table data final") + + reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'").strip()) + writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'").strip()) + + node.query(f"alter table data update key = 1 where key = 42") + node.query(f"optimize table data final") + + reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'").strip()) + writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'").strip()) + + assert (reads_before < reads_after) + assert (writes_before < writes_after) + + +def test_mutation_workload_override(): + node.query( + f""" + drop table if exists prod_data; + drop table if exists dev_data; + create table prod_data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3', mutation_workload='prod_mutations'; + create table dev_data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3', mutation_workload='dev_mutations'; + """ + ) + + node.query(f"insert into prod_data select * from numbers(1e4)") + node.query(f"optimize table prod_data final") + node.query(f"insert into dev_data select * from numbers(1e4)") + node.query(f"optimize table dev_data final") + + prod_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'").strip()) + prod_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'").strip()) + dev_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'").strip()) + dev_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'").strip()) + + node.query(f"alter table prod_data update key = 1 where key = 42") + node.query(f"optimize table prod_data final") + node.query(f"alter table dev_data update key = 1 where key = 42") + node.query(f"optimize table dev_data final") + + prod_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'").strip()) + prod_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'").strip()) + dev_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'").strip()) + dev_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'").strip()) + + assert (prod_reads_before < prod_reads_after) + assert (prod_writes_before < prod_writes_after) + assert (dev_reads_before < dev_reads_after) + assert (dev_writes_before < dev_writes_after) + + +def test_merge_workload_change(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + for env in ['prod', 'dev']: + update_workloads_config(merge_workload=f"{env}_merges") + + reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'").strip()) + writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'").strip()) + + node.query(f"insert into data select * from numbers(1e4)") + node.query(f"insert into data select * from numbers(2e4)") + node.query(f"insert into data select * from numbers(3e4)") + node.query(f"optimize table data final") + + reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'").strip()) + writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'").strip()) + + assert (reads_before < reads_after) + assert (writes_before < writes_after) + +def test_mutation_workload_change(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE)) engine=MergeTree() order by tuple() settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + for env in ['prod', 'dev']: + update_workloads_config(mutation_workload=f"{env}_mutations") + + node.query(f"insert into data select * from numbers(1e4)") + node.query(f"optimize table data final") + + reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'").strip()) + writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'").strip()) + + node.query(f"alter table data update key = 1 where key = 42") + node.query(f"optimize table data final") + + reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'").strip()) + writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'").strip()) + + breakpoint() + + assert (reads_before < reads_after) + assert (writes_before < writes_after) From 22af95b87151d729b4f9353c8835f32066f978b9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 22 May 2024 14:46:57 +0000 Subject: [PATCH 061/254] Automatic style fix --- tests/integration/test_scheduler/test.py | 231 ++++++++++++++++++----- 1 file changed, 180 insertions(+), 51 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index e0660c036815..d1ae51bc0d72 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -54,7 +54,7 @@ def set_default_configs(): def update_workloads_config(**settings): - xml='' + xml = "" for name in settings: xml += f"<{name}>{settings[name]}" print(xml) @@ -161,19 +161,35 @@ def test_merge_workload(): """ ) - reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'").strip()) - writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'").strip()) + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'" + ).strip() + ) + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'" + ).strip() + ) node.query(f"insert into data select * from numbers(1e4)") node.query(f"insert into data select * from numbers(2e4)") node.query(f"insert into data select * from numbers(3e4)") node.query(f"optimize table data final") - reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'").strip()) - writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'").strip()) + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/merges'" + ).strip() + ) + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/merges'" + ).strip() + ) - assert (reads_before < reads_after) - assert (writes_before < writes_after) + assert reads_before < reads_after + assert writes_before < writes_after def test_merge_workload_override(): @@ -186,10 +202,26 @@ def test_merge_workload_override(): """ ) - prod_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'").strip()) - prod_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'").strip()) - dev_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'").strip()) - dev_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'").strip()) + prod_reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'" + ).strip() + ) + prod_writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'" + ).strip() + ) + dev_reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'" + ).strip() + ) + dev_writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'" + ).strip() + ) node.query(f"insert into prod_data select * from numbers(1e4)") node.query(f"insert into prod_data select * from numbers(2e4)") @@ -200,15 +232,31 @@ def test_merge_workload_override(): node.query(f"optimize table prod_data final") node.query(f"optimize table dev_data final") - prod_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'").strip()) - prod_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'").strip()) - dev_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'").strip()) - dev_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'").strip()) + prod_reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_merges'" + ).strip() + ) + prod_writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_merges'" + ).strip() + ) + dev_reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_merges'" + ).strip() + ) + dev_writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_merges'" + ).strip() + ) - assert (prod_reads_before < prod_reads_after) - assert (prod_writes_before < prod_writes_after) - assert (dev_reads_before < dev_reads_after) - assert (dev_writes_before < dev_writes_after) + assert prod_reads_before < prod_reads_after + assert prod_writes_before < prod_writes_after + assert dev_reads_before < dev_reads_after + assert dev_writes_before < dev_writes_after def test_mutate_workload(): @@ -222,17 +270,33 @@ def test_mutate_workload(): node.query(f"insert into data select * from numbers(1e4)") node.query(f"optimize table data final") - reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'").strip()) - writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'").strip()) + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'" + ).strip() + ) + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'" + ).strip() + ) node.query(f"alter table data update key = 1 where key = 42") node.query(f"optimize table data final") - reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'").strip()) - writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'").strip()) + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/sys/mutations'" + ).strip() + ) + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/sys/mutations'" + ).strip() + ) - assert (reads_before < reads_after) - assert (writes_before < writes_after) + assert reads_before < reads_after + assert writes_before < writes_after def test_mutation_workload_override(): @@ -250,25 +314,57 @@ def test_mutation_workload_override(): node.query(f"insert into dev_data select * from numbers(1e4)") node.query(f"optimize table dev_data final") - prod_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'").strip()) - prod_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'").strip()) - dev_reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'").strip()) - dev_writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'").strip()) + prod_reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'" + ).strip() + ) + prod_writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'" + ).strip() + ) + dev_reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'" + ).strip() + ) + dev_writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'" + ).strip() + ) node.query(f"alter table prod_data update key = 1 where key = 42") node.query(f"optimize table prod_data final") node.query(f"alter table dev_data update key = 1 where key = 42") node.query(f"optimize table dev_data final") - prod_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'").strip()) - prod_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'").strip()) - dev_reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'").strip()) - dev_writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'").strip()) + prod_reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/prod_mutations'" + ).strip() + ) + prod_writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/prod_mutations'" + ).strip() + ) + dev_reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/dev_mutations'" + ).strip() + ) + dev_writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/dev_mutations'" + ).strip() + ) - assert (prod_reads_before < prod_reads_after) - assert (prod_writes_before < prod_writes_after) - assert (dev_reads_before < dev_reads_after) - assert (dev_writes_before < dev_writes_after) + assert prod_reads_before < prod_reads_after + assert prod_writes_before < prod_writes_after + assert dev_reads_before < dev_reads_after + assert dev_writes_before < dev_writes_after def test_merge_workload_change(): @@ -279,22 +375,39 @@ def test_merge_workload_change(): """ ) - for env in ['prod', 'dev']: + for env in ["prod", "dev"]: update_workloads_config(merge_workload=f"{env}_merges") - reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'").strip()) - writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'").strip()) + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'" + ).strip() + ) + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'" + ).strip() + ) node.query(f"insert into data select * from numbers(1e4)") node.query(f"insert into data select * from numbers(2e4)") node.query(f"insert into data select * from numbers(3e4)") node.query(f"optimize table data final") - reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'").strip()) - writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'").strip()) + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_merges'" + ).strip() + ) + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_merges'" + ).strip() + ) + + assert reads_before < reads_after + assert writes_before < writes_after - assert (reads_before < reads_after) - assert (writes_before < writes_after) def test_mutation_workload_change(): node.query( @@ -304,22 +417,38 @@ def test_mutation_workload_change(): """ ) - for env in ['prod', 'dev']: + for env in ["prod", "dev"]: update_workloads_config(mutation_workload=f"{env}_mutations") node.query(f"insert into data select * from numbers(1e4)") node.query(f"optimize table data final") - reads_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'").strip()) - writes_before = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'").strip()) + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'" + ).strip() + ) + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'" + ).strip() + ) node.query(f"alter table data update key = 1 where key = 42") node.query(f"optimize table data final") - reads_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'").strip()) - writes_after = int(node.query(f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'").strip()) + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/fair/{env}_mutations'" + ).strip() + ) + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/fair/{env}_mutations'" + ).strip() + ) breakpoint() - assert (reads_before < reads_after) - assert (writes_before < writes_after) + assert reads_before < reads_after + assert writes_before < writes_after From 319542f85bc5c36bbc7c810c0883b721956250be Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 22 May 2024 19:07:14 +0000 Subject: [PATCH 062/254] fix planning algorithm + switch matView to old squashing --- src/Interpreters/Squashing.cpp | 40 ++++++++++++++++--- src/Interpreters/Squashing.h | 2 +- .../Transforms/buildPushingToViewsChain.cpp | 13 ++---- 3 files changed, 40 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 9e398febdcab..7ebe4a930c97 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB @@ -197,7 +198,7 @@ PlanSquashing::PlanSquashing(Block header_, size_t min_block_size_rows_, size_t Chunk PlanSquashing::flush() { - return convertToChunk(chunks_to_merge_vec); + return convertToChunk(std::move(chunks_to_merge_vec)); } Chunk PlanSquashing::add(Chunk && input_chunk) @@ -210,21 +211,49 @@ Chunk PlanSquashing::addImpl(Chunk && input_chunk) if (!input_chunk) return {}; - if (isEnoughSize(chunks_to_merge_vec)) + /// Just read block is already enough. + if (isEnoughSize(input_chunk.getNumRows(), input_chunk.bytes())) + { + /// If no accumulated data, return just read block. + if (chunks_to_merge_vec.empty()) + { + chunks_to_merge_vec.push_back(std::move(input_chunk)); + Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); + chunks_to_merge_vec.clear(); + return res_chunk; + } + + /// Return accumulated data (maybe it has small size) and place new block to accumulated data. + Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); chunks_to_merge_vec.clear(); + chunks_to_merge_vec.push_back(std::move(input_chunk)); + return res_chunk; + } - if (input_chunk) + /// Accumulated block is already enough. + if (isEnoughSize(chunks_to_merge_vec)) + { + /// Return accumulated data and place new block to accumulated data. + Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); + chunks_to_merge_vec.clear(); chunks_to_merge_vec.push_back(std::move(input_chunk)); + return res_chunk; + } + /// Pushing data into accumulating vector + chunks_to_merge_vec.push_back(std::move(input_chunk)); + + /// If accumulated data is big enough, we send it if (isEnoughSize(chunks_to_merge_vec)) { - Chunk res_chunk = convertToChunk(chunks_to_merge_vec); + Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); + chunks_to_merge_vec.clear(); return res_chunk; } return {}; } -Chunk PlanSquashing::convertToChunk(std::vector &chunks) +Chunk PlanSquashing::convertToChunk(std::vector && chunks) { if (chunks.empty()) return {}; @@ -254,6 +283,7 @@ bool PlanSquashing::isEnoughSize(const std::vector & chunks) bool PlanSquashing::isEnoughSize(size_t rows, size_t bytes) const { + LOG_TRACE(getLogger("Planning"), "rows: {}, bytes: {}", rows, bytes); return (!min_block_size_rows && !min_block_size_bytes) || (min_block_size_rows && rows >= min_block_size_rows) || (min_block_size_bytes && bytes >= min_block_size_bytes); diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index 8273ae8cc8e6..0e9f001762f2 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -100,7 +100,7 @@ class PlanSquashing bool isEnoughSize(const std::vector & chunks); bool isEnoughSize(size_t rows, size_t bytes) const; - Chunk convertToChunk(std::vector &chunks); + Chunk convertToChunk(std::vector && chunks); }; } diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 951f40dadb99..cf407a75879a 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -370,13 +370,10 @@ std::optional generateViewChain( bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); const auto & settings = insert_context->getSettingsRef(); - out.addSource(std::make_shared(out.getInputHeader())); - - out.addSource(std::make_shared( + out.addSource(std::make_shared( out.getInputHeader(), table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, - 1)); // Chain requires a single input + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); counting->setProcessListElement(insert_context->getProcessListElement()); @@ -622,12 +619,10 @@ static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsDat /// Squashing is needed here because the materialized view query can generate a lot of blocks /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY /// and two-level aggregation is triggered). - pipeline.addTransform(std::make_shared( + pipeline.addTransform(std::make_shared( pipeline.getHeader(), context->getSettingsRef().min_insert_block_size_rows, - context->getSettingsRef().min_insert_block_size_bytes, - pipeline.getNumStreams())); - pipeline.addTransform(std::make_shared(pipeline.getHeader())); + context->getSettingsRef().min_insert_block_size_bytes)); auto converting = ActionsDAG::makeConvertingActions( pipeline.getHeader().getColumnsWithTypeAndName(), From 2db07e64e3a9a4897220f453e78c1d82d1a75d42 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 23 May 2024 11:59:31 +0000 Subject: [PATCH 063/254] fix tidy --- src/Processors/Transforms/PlanSquashingTransform.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 51781b03853a..51c637f745b6 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -35,7 +35,10 @@ IProcessor::Status PlanSquashingTransform::prepare() break; /// never reached } } - status = finish(); + if (status == Status::Ready) + status = finish(); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "There should be a Ready status to finish the PlanSquashing"); return status; } From 58000be1a7b0e6fd659073f383017cf8b913baaa Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 23 May 2024 13:55:06 +0000 Subject: [PATCH 064/254] fix style --- src/Processors/Transforms/PlanSquashingTransform.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 51c637f745b6..2cb0a19ecdb5 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -5,6 +5,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + PlanSquashingTransform::PlanSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), balance(header, min_block_size_rows, min_block_size_bytes) { From f632636f210c34841a7634790e32ba2153633ebf Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 23 May 2024 19:12:02 +0000 Subject: [PATCH 065/254] apply double-phased squashing in all transformers, resize optimization --- src/Interpreters/Squashing.cpp | 50 ++++++++++--------- src/Interpreters/Squashing.h | 8 ++- .../Transforms/ApplySquashingTransform.h | 8 +-- .../Transforms/PlanSquashingTransform.h | 1 - .../Transforms/SquashingTransform.cpp | 30 +++++++---- .../Transforms/SquashingTransform.h | 6 ++- 6 files changed, 57 insertions(+), 46 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 7ebe4a930c97..46e21635a307 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -1,7 +1,6 @@ #include #include #include -#include namespace DB @@ -135,46 +134,52 @@ ApplySquashing::ApplySquashing(Block header_) { } -Block ApplySquashing::add(Chunk && input_chunk) +Chunk ApplySquashing::add(Chunk && input_chunk) { return addImpl(std::move(input_chunk)); } -Block ApplySquashing::addImpl(Chunk && input_chunk) +Chunk ApplySquashing::addImpl(Chunk && input_chunk) { if (!input_chunk.hasChunkInfo()) - return Block(); + return Chunk(); const auto *info = getInfoFromChunk(input_chunk); - for (auto & chunk : info->chunks) - append(chunk); + append(info->chunks); Block to_return; std::swap(to_return, accumulated_block); - return to_return; + return Chunk(to_return.getColumns(), to_return.rows()); } -void ApplySquashing::append(Chunk & input_chunk) +void ApplySquashing::append(const std::vector & input_chunks) { - if (input_chunk.getNumColumns() == 0) - return; - if (!accumulated_block) + std::vector mutable_columns; + size_t rows = 0; + for (const Chunk & chunk : input_chunks) + rows += chunk.getNumRows(); + + for (const auto & input_chunk : input_chunks) { - for (size_t i = 0; i < input_chunk.getNumColumns(); ++ i) + if (!accumulated_block) { - ColumnWithTypeAndName col = ColumnWithTypeAndName(input_chunk.getColumns()[i], header.getDataTypes()[i], header.getNames()[i]); - accumulated_block.insert(accumulated_block.columns(), col); + for (size_t i = 0; i < input_chunks[0].getNumColumns(); ++i) + { // We can put this part of code out of the cycle, but it will consume more memory + ColumnWithTypeAndName col = ColumnWithTypeAndName(input_chunks[0].getColumns()[i],header.getDataTypes()[i], header.getNames()[i]); + mutable_columns.push_back(IColumn::mutate(col.column)); + mutable_columns[i]->reserve(rows); + accumulated_block.insert(col); + } + continue; } - return; - } - for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) - { - const auto source_column = input_chunk.getColumns()[i]; + for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) + { + const auto source_column = input_chunk.getColumns()[i]; - auto mutable_column = IColumn::mutate(std::move(accumulated_block.getByPosition(i).column)); - mutable_column->insertRangeFrom(*source_column, 0, source_column->size()); - accumulated_block.getByPosition(i).column = std::move(mutable_column); + mutable_columns[i]->insertRangeFrom(*source_column, 0, source_column->size()); + accumulated_block.getByPosition(i).column = mutable_columns[i]->cloneFinalized(); + } } } @@ -283,7 +288,6 @@ bool PlanSquashing::isEnoughSize(const std::vector & chunks) bool PlanSquashing::isEnoughSize(size_t rows, size_t bytes) const { - LOG_TRACE(getLogger("Planning"), "rows: {}, bytes: {}", rows, bytes); return (!min_block_size_rows && !min_block_size_bytes) || (min_block_size_rows && rows >= min_block_size_rows) || (min_block_size_bytes && bytes >= min_block_size_bytes); diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index 0e9f001762f2..d116ff1edddb 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -1,7 +1,5 @@ #pragma once -#include -#include #include #include #include @@ -60,17 +58,17 @@ class ApplySquashing public: explicit ApplySquashing(Block header_); - Block add(Chunk && input_chunk); + Chunk add(Chunk && input_chunk); private: Block accumulated_block; const Block header; - Block addImpl(Chunk && chunk); + Chunk addImpl(Chunk && chunk); const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); - void append(Chunk & input_chunk); + void append(const std::vector & input_chunks); bool isEnoughSize(const Block & block); bool isEnoughSize(size_t rows, size_t bytes) const; diff --git a/src/Processors/Transforms/ApplySquashingTransform.h b/src/Processors/Transforms/ApplySquashingTransform.h index abb3a0aad415..e63691fcc6a0 100644 --- a/src/Processors/Transforms/ApplySquashingTransform.h +++ b/src/Processors/Transforms/ApplySquashingTransform.h @@ -37,8 +37,8 @@ class ApplySquashingTransform : public ExceptionKeepingTransform protected: void onConsume(Chunk chunk) override { - if (auto block = squashing.add(std::move(chunk))) - cur_chunk.setColumns(block.getColumns(), block.rows()); + if (auto res_chunk = squashing.add(std::move(chunk))) + cur_chunk.setColumns(res_chunk.getColumns(), res_chunk.getNumRows()); } GenerateResult onGenerate() override @@ -50,8 +50,8 @@ class ApplySquashingTransform : public ExceptionKeepingTransform } void onFinish() override { - auto block = squashing.add({}); - finish_chunk.setColumns(block.getColumns(), block.rows()); + auto chunk = squashing.add({}); + finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); } private: diff --git a/src/Processors/Transforms/PlanSquashingTransform.h b/src/Processors/Transforms/PlanSquashingTransform.h index dc5b6d669b1d..7afc942a7f23 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.h +++ b/src/Processors/Transforms/PlanSquashingTransform.h @@ -3,7 +3,6 @@ #include #include #include -#include "Processors/Port.h" enum PlanningStatus { diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 8f7f6488d3eb..a516811bf456 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { @@ -12,14 +12,16 @@ extern const int LOGICAL_ERROR; SquashingTransform::SquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(min_block_size_rows, min_block_size_bytes) + , planSquashing(header, min_block_size_rows, min_block_size_bytes) + , applySquashing(header) { } void SquashingTransform::onConsume(Chunk chunk) { - if (auto block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()))) - cur_chunk.setColumns(block.getColumns(), block.rows()); + Chunk planned_chunk = planSquashing.add(std::move(chunk)); + if (planned_chunk.hasChunkInfo()) + cur_chunk = applySquashing.add(std::move(planned_chunk)); } SquashingTransform::GenerateResult SquashingTransform::onGenerate() @@ -32,8 +34,10 @@ SquashingTransform::GenerateResult SquashingTransform::onGenerate() void SquashingTransform::onFinish() { - auto block = squashing.add({}); - finish_chunk.setColumns(block.getColumns(), block.rows()); + Chunk chunk = planSquashing.flush(); + if (chunk.hasChunkInfo()) + chunk = applySquashing.add(std::move(chunk)); + finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); } void SquashingTransform::work() @@ -55,7 +59,9 @@ void SquashingTransform::work() SimpleSquashingTransform::SimpleSquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) - : ISimpleTransform(header, header, false), squashing(min_block_size_rows, min_block_size_bytes) + : ISimpleTransform(header, header, false) + , planSquashing(header, min_block_size_rows, min_block_size_bytes) + , applySquashing(header) { } @@ -63,16 +69,18 @@ void SimpleSquashingTransform::transform(Chunk & chunk) { if (!finished) { - if (auto block = squashing.add(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()))) - chunk.setColumns(block.getColumns(), block.rows()); + Chunk planned_chunk = planSquashing.add(std::move(chunk)); + if (planned_chunk.hasChunkInfo()) + chunk = applySquashing.add(std::move(planned_chunk)); } else { if (chunk.hasRows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk expected to be empty, otherwise it will be lost"); - auto block = squashing.add({}); - chunk.setColumns(block.getColumns(), block.rows()); + chunk = planSquashing.flush(); + if (chunk.hasChunkInfo()) + chunk = applySquashing.add(std::move(chunk)); } } diff --git a/src/Processors/Transforms/SquashingTransform.h b/src/Processors/Transforms/SquashingTransform.h index c5b727ac6ec7..b5b3c6616d2c 100644 --- a/src/Processors/Transforms/SquashingTransform.h +++ b/src/Processors/Transforms/SquashingTransform.h @@ -24,7 +24,8 @@ class SquashingTransform : public ExceptionKeepingTransform void onFinish() override; private: - Squashing squashing; + PlanSquashing planSquashing; + ApplySquashing applySquashing; Chunk cur_chunk; Chunk finish_chunk; }; @@ -43,7 +44,8 @@ class SimpleSquashingTransform : public ISimpleTransform IProcessor::Status prepare() override; private: - Squashing squashing; + PlanSquashing planSquashing; + ApplySquashing applySquashing; bool finished = false; }; From e6f83386665d1c129e5b94c82cdd643215f198c6 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 24 May 2024 20:38:45 +0000 Subject: [PATCH 066/254] memory optimizations and removed unused parts --- src/Interpreters/Squashing.cpp | 69 ++++++++----------- src/Interpreters/Squashing.h | 20 +++--- .../Transforms/PlanSquashingTransform.cpp | 5 +- .../Transforms/SquashingTransform.cpp | 4 +- src/Server/TCPHandler.cpp | 2 - src/Storages/buildQueryTreeForShard.cpp | 1 - 6 files changed, 47 insertions(+), 54 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 46e21635a307..12dcac7eb96a 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -135,11 +135,6 @@ ApplySquashing::ApplySquashing(Block header_) } Chunk ApplySquashing::add(Chunk && input_chunk) -{ - return addImpl(std::move(input_chunk)); -} - -Chunk ApplySquashing::addImpl(Chunk && input_chunk) { if (!input_chunk.hasChunkInfo()) return Chunk(); @@ -147,40 +142,38 @@ Chunk ApplySquashing::addImpl(Chunk && input_chunk) const auto *info = getInfoFromChunk(input_chunk); append(info->chunks); - Block to_return; - std::swap(to_return, accumulated_block); - return Chunk(to_return.getColumns(), to_return.rows()); + return std::move(accumulated_chunk); } -void ApplySquashing::append(const std::vector & input_chunks) +void ApplySquashing::append(std::vector & input_chunks) { - std::vector mutable_columns; + accumulated_chunk = {}; + std::vector mutable_columns = {}; size_t rows = 0; for (const Chunk & chunk : input_chunks) rows += chunk.getNumRows(); - for (const auto & input_chunk : input_chunks) + for (auto & input_chunk : input_chunks) { - if (!accumulated_block) + Columns columns = input_chunk.detachColumns(); + if (mutable_columns.empty()) { - for (size_t i = 0; i < input_chunks[0].getNumColumns(); ++i) - { // We can put this part of code out of the cycle, but it will consume more memory - ColumnWithTypeAndName col = ColumnWithTypeAndName(input_chunks[0].getColumns()[i],header.getDataTypes()[i], header.getNames()[i]); - mutable_columns.push_back(IColumn::mutate(col.column)); + for (size_t i = 0; i < columns.size(); ++i) + { + mutable_columns.push_back(IColumn::mutate(columns[i])); mutable_columns[i]->reserve(rows); - accumulated_block.insert(col); } continue; } - for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) + for (size_t i = 0, size = mutable_columns.size(); i < size; ++i) { - const auto source_column = input_chunk.getColumns()[i]; + const auto source_column = columns[i]; mutable_columns[i]->insertRangeFrom(*source_column, 0, source_column->size()); - accumulated_block.getByPosition(i).column = mutable_columns[i]->cloneFinalized(); } } + accumulated_chunk.setColumns(std::move(mutable_columns), rows); } const ChunksToSquash* ApplySquashing::getInfoFromChunk(const Chunk & chunk) @@ -206,12 +199,7 @@ Chunk PlanSquashing::flush() return convertToChunk(std::move(chunks_to_merge_vec)); } -Chunk PlanSquashing::add(Chunk && input_chunk) -{ - return addImpl(std::move(input_chunk)); -} - -Chunk PlanSquashing::addImpl(Chunk && input_chunk) +Chunk PlanSquashing::add(Chunk & input_chunk) { if (!input_chunk) return {}; @@ -231,27 +219,31 @@ Chunk PlanSquashing::addImpl(Chunk && input_chunk) /// Return accumulated data (maybe it has small size) and place new block to accumulated data. Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); chunks_to_merge_vec.clear(); + changeCurrentSize(input_chunk.getNumRows(), input_chunk.bytes()); chunks_to_merge_vec.push_back(std::move(input_chunk)); return res_chunk; } /// Accumulated block is already enough. - if (isEnoughSize(chunks_to_merge_vec)) + if (isEnoughSize(accumulated_size.rows, accumulated_size.bytes)) { /// Return accumulated data and place new block to accumulated data. Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); chunks_to_merge_vec.clear(); + changeCurrentSize(input_chunk.getNumRows(), input_chunk.bytes()); chunks_to_merge_vec.push_back(std::move(input_chunk)); return res_chunk; } /// Pushing data into accumulating vector + expandCurrentSize(input_chunk.getNumRows(), input_chunk.bytes()); chunks_to_merge_vec.push_back(std::move(input_chunk)); /// If accumulated data is big enough, we send it - if (isEnoughSize(chunks_to_merge_vec)) + if (isEnoughSize(accumulated_size.rows, accumulated_size.bytes)) { Chunk res_chunk = convertToChunk(std::move(chunks_to_merge_vec)); + changeCurrentSize(0, 0); chunks_to_merge_vec.clear(); return res_chunk; } @@ -264,26 +256,23 @@ Chunk PlanSquashing::convertToChunk(std::vector && chunks) return {}; auto info = std::make_shared(); - for (auto &chunk : chunks) - info->chunks.push_back(std::move(chunk)); + info->chunks = std::move(chunks); chunks.clear(); return Chunk(header.cloneEmptyColumns(), 0, info); } -bool PlanSquashing::isEnoughSize(const std::vector & chunks) +void PlanSquashing::expandCurrentSize(size_t rows, size_t bytes) { - size_t rows = 0; - size_t bytes = 0; - - for (const Chunk & chunk : chunks) - { - rows += chunk.getNumRows(); - bytes += chunk.bytes(); - } + accumulated_size.rows += rows; + accumulated_size.bytes += bytes; +} - return isEnoughSize(rows, bytes); +void PlanSquashing::changeCurrentSize(size_t rows, size_t bytes) +{ + accumulated_size.rows = rows; + accumulated_size.bytes = bytes; } bool PlanSquashing::isEnoughSize(size_t rows, size_t bytes) const diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index d116ff1edddb..0e844c4912be 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -61,14 +61,12 @@ class ApplySquashing Chunk add(Chunk && input_chunk); private: - Block accumulated_block; + Chunk accumulated_chunk; const Block header; - Chunk addImpl(Chunk && chunk); - const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); - void append(const std::vector & input_chunks); + void append(std::vector & input_chunks); bool isEnoughSize(const Block & block); bool isEnoughSize(size_t rows, size_t bytes) const; @@ -79,7 +77,7 @@ class PlanSquashing public: PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); - Chunk add(Chunk && input_chunk); + Chunk add(Chunk & input_chunk); Chunk flush(); bool isDataLeft() { @@ -87,15 +85,21 @@ class PlanSquashing } private: + struct CurrentSize + { + size_t rows = 0; + size_t bytes = 0; + }; + std::vector chunks_to_merge_vec = {}; size_t min_block_size_rows; size_t min_block_size_bytes; const Block header; + CurrentSize accumulated_size; - Chunk addImpl(Chunk && input_chunk); - - bool isEnoughSize(const std::vector & chunks); + void expandCurrentSize(size_t rows, size_t bytes); + void changeCurrentSize(size_t rows, size_t bytes); bool isEnoughSize(size_t rows, size_t bytes) const; Chunk convertToChunk(std::vector && chunks); diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 2cb0a19ecdb5..1384f760d486 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -90,6 +90,7 @@ IProcessor::Status PlanSquashingTransform::prepareConsume() { planning_status = PlanningStatus::FLUSH; flushChunk(); + return Status::Ready; } planning_status = PlanningStatus::PUSH; return Status::Ready; @@ -133,7 +134,7 @@ IProcessor::Status PlanSquashingTransform::waitForDataIn() void PlanSquashingTransform::transform(Chunk & chunk_) { - Chunk res_chunk = balance.add(std::move(chunk_)); + Chunk res_chunk = balance.add(chunk_); std::swap(res_chunk, chunk_); } @@ -157,6 +158,8 @@ IProcessor::Status PlanSquashingTransform::sendOrFlush() { if (planning_status == PlanningStatus::PUSH) planning_status = PlanningStatus::READ_IF_CAN; + else + planning_status = PlanningStatus::FINISH; output.push(std::move(chunk)); return Status::Ready; diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index a516811bf456..67358316d48d 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -19,7 +19,7 @@ SquashingTransform::SquashingTransform( void SquashingTransform::onConsume(Chunk chunk) { - Chunk planned_chunk = planSquashing.add(std::move(chunk)); + Chunk planned_chunk = planSquashing.add(chunk); if (planned_chunk.hasChunkInfo()) cur_chunk = applySquashing.add(std::move(planned_chunk)); } @@ -69,7 +69,7 @@ void SimpleSquashingTransform::transform(Chunk & chunk) { if (!finished) { - Chunk planned_chunk = planSquashing.add(std::move(chunk)); + Chunk planned_chunk = planSquashing.add(chunk); if (planned_chunk.hasChunkInfo()) chunk = applySquashing.add(std::move(planned_chunk)); } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 3f8e43d81dd6..9f14facdf8f0 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include @@ -1603,7 +1602,6 @@ void TCPHandler::sendHello() nonce.emplace(thread_local_rng()); writeIntBinary(nonce.value(), *out); } - out->next(); } diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index 8ebb2173f6ca..0f7d65fc7c95 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -23,7 +23,6 @@ #include #include - namespace DB { From b08ecfe6c0ee5b842f30999257daa3ae89ba0916 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Sat, 25 May 2024 10:40:51 +0000 Subject: [PATCH 067/254] fix for nullable columns --- src/Interpreters/Squashing.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 12dcac7eb96a..e2abcd00be35 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -160,7 +160,10 @@ void ApplySquashing::append(std::vector & input_chunks) { for (size_t i = 0; i < columns.size(); ++i) { - mutable_columns.push_back(IColumn::mutate(columns[i])); + if (columns[i]->isNullable()) + mutable_columns.push_back(IColumn::mutate(columns[i])); + else + mutable_columns.push_back(columns[i]->assumeMutable()); mutable_columns[i]->reserve(rows); } continue; From 01a16fd8e2d7ba65ee9feddcfb1cf7609be02947 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 27 May 2024 11:33:01 +0000 Subject: [PATCH 068/254] squashing refactoring --- src/Interpreters/Squashing.cpp | 27 ++++++++++++--------------- 1 file changed, 12 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index e2abcd00be35..855bf32abe99 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -153,27 +153,24 @@ void ApplySquashing::append(std::vector & input_chunks) for (const Chunk & chunk : input_chunks) rows += chunk.getNumRows(); - for (auto & input_chunk : input_chunks) { - Columns columns = input_chunk.detachColumns(); - if (mutable_columns.empty()) + auto & first_chunk = input_chunks[0]; + Columns columns = first_chunk.detachColumns(); + for (size_t i = 0; i < columns.size(); ++i) { - for (size_t i = 0; i < columns.size(); ++i) - { - if (columns[i]->isNullable()) - mutable_columns.push_back(IColumn::mutate(columns[i])); - else - mutable_columns.push_back(columns[i]->assumeMutable()); - mutable_columns[i]->reserve(rows); - } - continue; + mutable_columns.push_back(IColumn::mutate(std::move(columns[i]))); + mutable_columns[i]->reserve(rows); } + } - for (size_t i = 0, size = mutable_columns.size(); i < size; ++i) + for (size_t i = 1; i < input_chunks.size(); ++i) // We've already processed the first chunk above + { + Columns columns = input_chunks[i].detachColumns(); + for (size_t j = 0, size = mutable_columns.size(); j < size; ++j) { - const auto source_column = columns[i]; + const auto source_column = columns[j]; - mutable_columns[i]->insertRangeFrom(*source_column, 0, source_column->size()); + mutable_columns[j]->insertRangeFrom(*source_column, 0, source_column->size()); } } accumulated_chunk.setColumns(std::move(mutable_columns), rows); From 00b07bba146848cfecf8a4f59c78161d24aa3566 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 27 May 2024 16:23:01 +0000 Subject: [PATCH 069/254] remove squashing --- src/Interpreters/Squashing.cpp | 127 +----------------- src/Interpreters/Squashing.h | 33 +---- .../Transforms/PlanSquashingTransform.cpp | 4 +- .../Transforms/SquashingTransform.cpp | 8 +- src/Server/TCPHandler.cpp | 23 +++- src/Storages/MergeTree/MutateTask.cpp | 28 +++- 6 files changed, 56 insertions(+), 167 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 855bf32abe99..3872c2ba6b99 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -1,6 +1,7 @@ #include #include #include +#include "Columns/IColumn.h" namespace DB @@ -11,124 +12,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -Squashing::Squashing(size_t min_block_size_rows_, size_t min_block_size_bytes_) - : min_block_size_rows(min_block_size_rows_) - , min_block_size_bytes(min_block_size_bytes_) -{ -} - -Block Squashing::add(Block && input_block) -{ - return addImpl(std::move(input_block)); -} - -Block Squashing::add(const Block & input_block) -{ - return addImpl(input_block); -} - -/* - * To minimize copying, accept two types of argument: const reference for output - * stream, and rvalue reference for input stream, and decide whether to copy - * inside this function. This allows us not to copy Block unless we absolutely - * have to. - */ -template -Block Squashing::addImpl(ReferenceType input_block) -{ - /// End of input stream. - if (!input_block) - { - Block to_return; - std::swap(to_return, accumulated_block); - return to_return; - } - - /// Just read block is already enough. - if (isEnoughSize(input_block)) - { - /// If no accumulated data, return just read block. - if (!accumulated_block) - { - return std::move(input_block); - } - - /// Return accumulated data (maybe it has small size) and place new block to accumulated data. - Block to_return = std::move(input_block); - std::swap(to_return, accumulated_block); - return to_return; - } - - /// Accumulated block is already enough. - if (isEnoughSize(accumulated_block)) - { - /// Return accumulated data and place new block to accumulated data. - Block to_return = std::move(input_block); - std::swap(to_return, accumulated_block); - return to_return; - } - - append(std::move(input_block)); - if (isEnoughSize(accumulated_block)) - { - Block to_return; - std::swap(to_return, accumulated_block); - return to_return; - } - - /// Squashed block is not ready. - return {}; -} - - -template -void Squashing::append(ReferenceType input_block) -{ - if (!accumulated_block) - { - accumulated_block = std::move(input_block); - return; - } - - assert(blocksHaveEqualStructure(input_block, accumulated_block)); - - for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) - { - const auto source_column = input_block.getByPosition(i).column; - - auto mutable_column = IColumn::mutate(std::move(accumulated_block.getByPosition(i).column)); - mutable_column->insertRangeFrom(*source_column, 0, source_column->size()); - accumulated_block.getByPosition(i).column = std::move(mutable_column); - } -} - - -bool Squashing::isEnoughSize(const Block & block) -{ - size_t rows = 0; - size_t bytes = 0; - - for (const auto & [column, type, name] : block) - { - if (!rows) - rows = column->size(); - else if (rows != column->size()) - throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Sizes of columns doesn't match"); - - bytes += column->byteSize(); - } - - return isEnoughSize(rows, bytes); -} - - -bool Squashing::isEnoughSize(size_t rows, size_t bytes) const -{ - return (!min_block_size_rows && !min_block_size_bytes) - || (min_block_size_rows && rows >= min_block_size_rows) - || (min_block_size_bytes && bytes >= min_block_size_bytes); -} - ApplySquashing::ApplySquashing(Block header_) : header(std::move(header_)) { @@ -187,10 +70,9 @@ const ChunksToSquash* ApplySquashing::getInfoFromChunk(const Chunk & chunk) return agg_info; } -PlanSquashing::PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) +PlanSquashing::PlanSquashing(size_t min_block_size_rows_, size_t min_block_size_bytes_) : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) - , header(std::move(header_)) { } @@ -199,7 +81,7 @@ Chunk PlanSquashing::flush() return convertToChunk(std::move(chunks_to_merge_vec)); } -Chunk PlanSquashing::add(Chunk & input_chunk) +Chunk PlanSquashing::add(Chunk && input_chunk) { if (!input_chunk) return {}; @@ -260,7 +142,8 @@ Chunk PlanSquashing::convertToChunk(std::vector && chunks) chunks.clear(); - return Chunk(header.cloneEmptyColumns(), 0, info); + Columns cols = {}; + return Chunk(cols, 0, info); } void PlanSquashing::expandCurrentSize(size_t rows, size_t bytes) diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index 0e844c4912be..d9d430c18359 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -25,33 +25,6 @@ struct ChunksToSquash : public ChunkInfo * * Order of data is kept. */ -class Squashing -{ -public: - /// Conditions on rows and bytes are OR-ed. If one of them is zero, then corresponding condition is ignored. - Squashing(size_t min_block_size_rows_, size_t min_block_size_bytes_); - - /** Add next block and possibly returns squashed block. - * At end, you need to pass empty block. As the result for last (empty) block, you will get last Result with ready = true. - */ - Block add(Block && block); - Block add(const Block & block); - -private: - size_t min_block_size_rows; - size_t min_block_size_bytes; - - Block accumulated_block; - - template - Block addImpl(ReferenceType block); - - template - void append(ReferenceType block); - - bool isEnoughSize(const Block & block); - bool isEnoughSize(size_t rows, size_t bytes) const; -}; class ApplySquashing { @@ -75,9 +48,9 @@ class ApplySquashing class PlanSquashing { public: - PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); + PlanSquashing(size_t min_block_size_rows_, size_t min_block_size_bytes_); - Chunk add(Chunk & input_chunk); + Chunk add(Chunk && input_chunk); Chunk flush(); bool isDataLeft() { @@ -95,7 +68,7 @@ class PlanSquashing size_t min_block_size_rows; size_t min_block_size_bytes; - const Block header; + // const Block header; CurrentSize accumulated_size; void expandCurrentSize(size_t rows, size_t bytes); diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 1384f760d486..96f41e37d2fc 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes } PlanSquashingTransform::PlanSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) - : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), balance(header, min_block_size_rows, min_block_size_bytes) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), balance(min_block_size_rows, min_block_size_bytes) { } @@ -134,7 +134,7 @@ IProcessor::Status PlanSquashingTransform::waitForDataIn() void PlanSquashingTransform::transform(Chunk & chunk_) { - Chunk res_chunk = balance.add(chunk_); + Chunk res_chunk = balance.add(std::move(chunk_)); std::swap(res_chunk, chunk_); } diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 67358316d48d..6f7c877b2f3f 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -12,14 +12,14 @@ extern const int LOGICAL_ERROR; SquashingTransform::SquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , planSquashing(header, min_block_size_rows, min_block_size_bytes) + , planSquashing(min_block_size_rows, min_block_size_bytes) , applySquashing(header) { } void SquashingTransform::onConsume(Chunk chunk) { - Chunk planned_chunk = planSquashing.add(chunk); + Chunk planned_chunk = planSquashing.add(std::move(chunk)); if (planned_chunk.hasChunkInfo()) cur_chunk = applySquashing.add(std::move(planned_chunk)); } @@ -60,7 +60,7 @@ void SquashingTransform::work() SimpleSquashingTransform::SimpleSquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ISimpleTransform(header, header, false) - , planSquashing(header, min_block_size_rows, min_block_size_bytes) + , planSquashing(min_block_size_rows, min_block_size_bytes) , applySquashing(header) { } @@ -69,7 +69,7 @@ void SimpleSquashingTransform::transform(Chunk & chunk) { if (!finished) { - Chunk planned_chunk = planSquashing.add(chunk); + Chunk planned_chunk = planSquashing.add(std::move(chunk)); if (planned_chunk.hasChunkInfo()) chunk = applySquashing.add(std::move(planned_chunk)); } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 9f14facdf8f0..476c4dd372b9 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -885,13 +885,21 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro using PushResult = AsynchronousInsertQueue::PushResult; startInsertQuery(); - Squashing squashing(0, query_context->getSettingsRef().async_insert_max_data_size); + PlanSquashing plan_squashing(0, query_context->getSettingsRef().async_insert_max_data_size); + ApplySquashing apply_squashing(state.input_header); while (readDataNext()) { - auto result = squashing.add(std::move(state.block_for_insert)); - if (result) + auto planned_chunk = plan_squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); + Chunk result_chunk; + if (planned_chunk.hasChunkInfo()) + result_chunk = apply_squashing.add(std::move(planned_chunk)); + if (result_chunk) { + ColumnsWithTypeAndName cols; + for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.input_header.getDataTypes()[j], state.input_header.getNames()[j])); + auto result = Block(cols); return PushResult { .status = PushResult::TOO_MUCH_DATA, @@ -900,7 +908,14 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro } } - auto result = squashing.add({}); + auto planned_chunk = plan_squashing.flush(); + Chunk result_chunk; + if (planned_chunk.hasChunkInfo()) + result_chunk = apply_squashing.add(std::move(planned_chunk)); + ColumnsWithTypeAndName cols; + for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.input_header.getDataTypes()[j], state.input_header.getNames()[j])); + auto result = Block(cols); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 8052ee8f6308..f7a4651f6fda 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -28,6 +28,7 @@ #include #include #include +#include namespace ProfileEvents @@ -1266,7 +1267,8 @@ class PartMergerWriter ProjectionNameToItsBlocks projection_parts; std::move_iterator projection_parts_iterator; - std::vector projection_squashes; + std::vector projection_squash_plannings; + std::vector projection_squashes; const ProjectionsDescription & projections; ExecutableTaskPtr merge_projection_parts_task_ptr; @@ -1285,7 +1287,8 @@ void PartMergerWriter::prepare() for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { // We split the materialization into multiple stages similar to the process of INSERT SELECT query. - projection_squashes.emplace_back(settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + projection_squash_plannings.emplace_back(settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + projection_squashes.emplace_back(ctx->updated_header); } existing_rows_count = 0; @@ -1313,7 +1316,15 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() Block projection_block; { ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); - projection_block = projection_squashes[i].add(projection.calculate(cur_block, ctx->context)); + Block to_plan = projection.calculate(cur_block, ctx->context); + Chunk planned_chunk = projection_squash_plannings[i].add({to_plan.getColumns(), to_plan.rows()}); + Chunk projection_chunk; + if (planned_chunk.hasChunkInfo()) + projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); + ColumnsWithTypeAndName cols; + for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) + cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); + projection_block = Block(cols); } if (projection_block) @@ -1337,8 +1348,15 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { const auto & projection = *ctx->projections_to_build[i]; - auto & projection_squash = projection_squashes[i]; - auto projection_block = projection_squash.add({}); + auto & projection_squash_plan = projection_squash_plannings[i]; + auto planned_chunk = projection_squash_plan.flush(); + Chunk projection_chunk; + if (planned_chunk.hasChunkInfo()) + projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); + ColumnsWithTypeAndName cols; + for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) + cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); + auto projection_block = Block(cols); if (projection_block) { auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( From d0d5b6d0cbbc9841f6bce59ef2feee4cc00b1b1f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 27 May 2024 16:30:46 +0000 Subject: [PATCH 070/254] fix style --- src/Interpreters/Squashing.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 3872c2ba6b99..82d80114a85a 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -8,7 +8,6 @@ namespace DB { namespace ErrorCodes { - extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int LOGICAL_ERROR; } From 78e161ff15b5399aa18141b5cf896353a2fc9e00 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 27 May 2024 19:02:17 +0000 Subject: [PATCH 071/254] fixes (added header to planner) --- src/Interpreters/Squashing.cpp | 7 ++- src/Interpreters/Squashing.h | 4 +- .../Transforms/PlanSquashingTransform.cpp | 2 +- .../Transforms/SquashingTransform.cpp | 4 +- src/Server/TCPHandler.cpp | 6 +-- src/Storages/MergeTree/MutateTask.cpp | 44 +++++++++---------- 6 files changed, 31 insertions(+), 36 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 82d80114a85a..9ecd92f732cf 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -1,7 +1,6 @@ #include #include #include -#include "Columns/IColumn.h" namespace DB @@ -69,9 +68,10 @@ const ChunksToSquash* ApplySquashing::getInfoFromChunk(const Chunk & chunk) return agg_info; } -PlanSquashing::PlanSquashing(size_t min_block_size_rows_, size_t min_block_size_bytes_) +PlanSquashing::PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) + , header(std::move(header_)) { } @@ -141,8 +141,7 @@ Chunk PlanSquashing::convertToChunk(std::vector && chunks) chunks.clear(); - Columns cols = {}; - return Chunk(cols, 0, info); + return Chunk(header.cloneEmptyColumns(), 0, info); } void PlanSquashing::expandCurrentSize(size_t rows, size_t bytes) diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index d9d430c18359..a2928e0eeb62 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -48,7 +48,7 @@ class ApplySquashing class PlanSquashing { public: - PlanSquashing(size_t min_block_size_rows_, size_t min_block_size_bytes_); + explicit PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); Chunk add(Chunk && input_chunk); Chunk flush(); @@ -68,7 +68,7 @@ class PlanSquashing size_t min_block_size_rows; size_t min_block_size_bytes; - // const Block header; + const Block header; CurrentSize accumulated_size; void expandCurrentSize(size_t rows, size_t bytes); diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 96f41e37d2fc..7945bd97e04e 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes } PlanSquashingTransform::PlanSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) - : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), balance(min_block_size_rows, min_block_size_bytes) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), balance(header, min_block_size_rows, min_block_size_bytes) { } diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 6f7c877b2f3f..a516811bf456 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -12,7 +12,7 @@ extern const int LOGICAL_ERROR; SquashingTransform::SquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , planSquashing(min_block_size_rows, min_block_size_bytes) + , planSquashing(header, min_block_size_rows, min_block_size_bytes) , applySquashing(header) { } @@ -60,7 +60,7 @@ void SquashingTransform::work() SimpleSquashingTransform::SimpleSquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ISimpleTransform(header, header, false) - , planSquashing(min_block_size_rows, min_block_size_bytes) + , planSquashing(header, min_block_size_rows, min_block_size_bytes) , applySquashing(header) { } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 476c4dd372b9..dfe2d909b43f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -885,17 +885,15 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro using PushResult = AsynchronousInsertQueue::PushResult; startInsertQuery(); - PlanSquashing plan_squashing(0, query_context->getSettingsRef().async_insert_max_data_size); + PlanSquashing plan_squashing(state.input_header, 0, query_context->getSettingsRef().async_insert_max_data_size); ApplySquashing apply_squashing(state.input_header); while (readDataNext()) { auto planned_chunk = plan_squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); - Chunk result_chunk; if (planned_chunk.hasChunkInfo()) - result_chunk = apply_squashing.add(std::move(planned_chunk)); - if (result_chunk) { + Chunk result_chunk = apply_squashing.add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.input_header.getDataTypes()[j], state.input_header.getNames()[j])); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index f7a4651f6fda..5e14d4c5b38e 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1267,7 +1267,7 @@ class PartMergerWriter ProjectionNameToItsBlocks projection_parts; std::move_iterator projection_parts_iterator; - std::vector projection_squash_plannings; + std::vector projection_squash_plannings; std::vector projection_squashes; const ProjectionsDescription & projections; @@ -1282,12 +1282,15 @@ class PartMergerWriter void PartMergerWriter::prepare() { + projection_squash_plannings.reserve(ctx->projections_to_build.size()); + projection_squashes.reserve(ctx->projections_to_build.size()); const auto & settings = ctx->context->getSettingsRef(); for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { + PlanSquashing plan_squashing(ctx->updated_header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); // We split the materialization into multiple stages similar to the process of INSERT SELECT query. - projection_squash_plannings.emplace_back(settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + projection_squash_plannings.push_back(&plan_squashing); projection_squashes.emplace_back(ctx->updated_header); } @@ -1313,24 +1316,21 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { const auto & projection = *ctx->projections_to_build[i]; - Block projection_block; + Chunk planned_chunk; { ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); - Block to_plan = projection.calculate(cur_block, ctx->context); - Chunk planned_chunk = projection_squash_plannings[i].add({to_plan.getColumns(), to_plan.rows()}); - Chunk projection_chunk; - if (planned_chunk.hasChunkInfo()) - projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); - ColumnsWithTypeAndName cols; - for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); - projection_block = Block(cols); + Block block_to_squash = projection.calculate(cur_block, ctx->context); + planned_chunk = projection_squash_plannings[i]->add({block_to_squash.getColumns(), block_to_squash.rows()}); } - if (projection_block) + if (planned_chunk.hasChunkInfo()) { + Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); + ColumnsWithTypeAndName cols; + for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) + cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( - *ctx->data, ctx->log, projection_block, projection, ctx->new_data_part.get(), ++block_num); + *ctx->data, ctx->log, Block(cols), projection, ctx->new_data_part.get(), ++block_num); tmp_part.finalize(); tmp_part.part->getDataPartStorage().commitTransaction(); projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); @@ -1349,18 +1349,16 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { const auto & projection = *ctx->projections_to_build[i]; auto & projection_squash_plan = projection_squash_plannings[i]; - auto planned_chunk = projection_squash_plan.flush(); - Chunk projection_chunk; + auto planned_chunk = projection_squash_plan->flush(); if (planned_chunk.hasChunkInfo()) - projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); - ColumnsWithTypeAndName cols; - for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); - auto projection_block = Block(cols); - if (projection_block) { + Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); + ColumnsWithTypeAndName cols; + for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) + cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); + auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( - *ctx->data, ctx->log, projection_block, projection, ctx->new_data_part.get(), ++block_num); + *ctx->data, ctx->log, Block(cols), projection, ctx->new_data_part.get(), ++block_num); temp_part.finalize(); temp_part.part->getDataPartStorage().commitTransaction(); projection_parts[projection.name].emplace_back(std::move(temp_part.part)); From b73902b905aef2f5b05441e32b3e7e198960e249 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 28 May 2024 10:53:00 +0000 Subject: [PATCH 072/254] remove breakpoint --- tests/integration/test_scheduler/test.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index d1ae51bc0d72..5779008cc135 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -448,7 +448,5 @@ def test_mutation_workload_change(): ).strip() ) - breakpoint() - assert reads_before < reads_after assert writes_before < writes_after From 01183902a667d94d71ed9faabeffdc60cdcf95cd Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 28 May 2024 12:07:20 +0000 Subject: [PATCH 073/254] try to fix a segfault --- src/Server/TCPHandler.cpp | 10 ++++++---- src/Storages/MergeTree/MutateTask.cpp | 12 ++++++------ 2 files changed, 12 insertions(+), 10 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index dfe2d909b43f..d0e9dc5f3ee8 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -895,8 +895,9 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro { Chunk result_chunk = apply_squashing.add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.input_header.getDataTypes()[j], state.input_header.getNames()[j])); + if (result_chunk.hasColumns()) + for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.input_header.getDataTypes()[j], state.input_header.getNames()[j])); auto result = Block(cols); return PushResult { @@ -911,8 +912,9 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro if (planned_chunk.hasChunkInfo()) result_chunk = apply_squashing.add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.input_header.getDataTypes()[j], state.input_header.getNames()[j])); + if (result_chunk.hasColumns()) + for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.input_header.getDataTypes()[j], state.input_header.getNames()[j])); auto result = Block(cols); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 5e14d4c5b38e..0e272fc8eb92 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1282,8 +1282,6 @@ class PartMergerWriter void PartMergerWriter::prepare() { - projection_squash_plannings.reserve(ctx->projections_to_build.size()); - projection_squashes.reserve(ctx->projections_to_build.size()); const auto & settings = ctx->context->getSettingsRef(); for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) @@ -1327,8 +1325,9 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); + if (projection_chunk.hasColumns()) + for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) + cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, Block(cols), projection, ctx->new_data_part.get(), ++block_num); tmp_part.finalize(); @@ -1354,8 +1353,9 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); + if (projection_chunk.hasColumns()) + for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) + cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, Block(cols), projection, ctx->new_data_part.get(), ++block_num); From bb8f887dfb6ff97a06625c3798f8a27ca5269720 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 28 May 2024 14:19:48 +0000 Subject: [PATCH 074/254] fix typo --- docs/en/operations/workload-scheduling.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/workload-scheduling.md b/docs/en/operations/workload-scheduling.md index 3b950bd36b14..08629492ec69 100644 --- a/docs/en/operations/workload-scheduling.md +++ b/docs/en/operations/workload-scheduling.md @@ -156,6 +156,6 @@ Example: ## See also - [system.scheduler](/docs/en/operations/system-tables/scheduler.md) - [merge_workload](/docs/en/operations/settings/merge-tree-settings.md#merge_workload) merge tree setting - - [merge_workload](/docs/en/operations/server-configuration-parametes/settings.md#merge_workload) global server setting + - [merge_workload](/docs/en/operations/server-configuration-parameters/settings.md#merge_workload) global server setting - [mutation_workload](/docs/en/operations/settings/merge-tree-settings.md#mutation_workload) merge tree setting - - [mutation_workload](/docs/en/operations/server-configuration-parametes/settings.md#mutation_workload) global server setting + - [mutation_workload](/docs/en/operations/server-configuration-parameters/settings.md#mutation_workload) global server setting From 45f6c19c9df5c3f62b1ed4933321053ef6f77c91 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 28 May 2024 15:36:19 +0000 Subject: [PATCH 075/254] attempt #2 --- src/Storages/MergeTree/MutateTask.cpp | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 0e272fc8eb92..8c4e0c6e6547 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1315,11 +1315,9 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() const auto & projection = *ctx->projections_to_build[i]; Chunk planned_chunk; - { - ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); - Block block_to_squash = projection.calculate(cur_block, ctx->context); - planned_chunk = projection_squash_plannings[i]->add({block_to_squash.getColumns(), block_to_squash.rows()}); - } + ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); + Block block_to_squash = projection.calculate(cur_block, ctx->context); + planned_chunk = projection_squash_plannings[i]->add({block_to_squash.getColumns(), block_to_squash.rows()}); if (planned_chunk.hasChunkInfo()) { @@ -1327,7 +1325,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() ColumnsWithTypeAndName cols; if (projection_chunk.hasColumns()) for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], block_to_squash.getDataTypes()[j], block_to_squash.getNames()[j])); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, Block(cols), projection, ctx->new_data_part.get(), ++block_num); tmp_part.finalize(); From f46a7d64a0163e0cf9140eb0e56c88f2cc6471bb Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 28 May 2024 17:00:35 +0000 Subject: [PATCH 076/254] fix segfault in TCPHandler --- src/Server/TCPHandler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d0e9dc5f3ee8..b95face57e1f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -897,7 +897,7 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro ColumnsWithTypeAndName cols; if (result_chunk.hasColumns()) for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.input_header.getDataTypes()[j], state.input_header.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.block_for_insert.getDataTypes()[j], state.block_for_insert.getNames()[j])); auto result = Block(cols); return PushResult { @@ -914,7 +914,7 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro ColumnsWithTypeAndName cols; if (result_chunk.hasColumns()) for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.input_header.getDataTypes()[j], state.input_header.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.block_for_insert.getDataTypes()[j], state.block_for_insert.getNames()[j])); auto result = Block(cols); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } From 513de6ce19867dc10fedf5c9820363b84655a9f1 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 28 May 2024 17:59:44 +0000 Subject: [PATCH 077/254] using of header from applySquashing --- src/Interpreters/Squashing.h | 2 +- src/Storages/MergeTree/MutateTask.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index a2928e0eeb62..05259bbc0c32 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -32,10 +32,10 @@ class ApplySquashing explicit ApplySquashing(Block header_); Chunk add(Chunk && input_chunk); + const Block header; private: Chunk accumulated_chunk; - const Block header; const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 8c4e0c6e6547..0a3a217d9434 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1353,7 +1353,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() ColumnsWithTypeAndName cols; if (projection_chunk.hasColumns()) for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], projection_squashes[i].header.getDataTypes()[j], projection_squashes[i].header.getNames()[j])); auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, Block(cols), projection, ctx->new_data_part.get(), ++block_num); From f0e9d6b459cfee4331861d4f0e3c92c1e9d67c72 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 29 May 2024 14:30:48 +0000 Subject: [PATCH 078/254] revert changes in mv --- .../Transforms/buildPushingToViewsChain.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index ff1be9323f59..177d45650dde 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -367,13 +367,16 @@ std::optional generateViewChain( bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type; out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, thread_status_holder, view_counter_ms, check_access); - bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); - const auto & settings = insert_context->getSettingsRef(); + if (interpreter.shouldAddSquashingFroStorage(inner_table)) + { + bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); + const auto & settings = insert_context->getSettingsRef(); - out.addSource(std::make_shared( - out.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + out.addSource(std::make_shared( + out.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + } auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); counting->setProcessListElement(insert_context->getProcessListElement()); From f51a145437df6f173d67e5fc7f1259c1e0154a98 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 29 May 2024 15:00:22 +0000 Subject: [PATCH 079/254] fixes for segfault --- src/Server/TCPHandler.cpp | 8 ++++---- src/Storages/MergeTree/MutateTask.cpp | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b95face57e1f..af184940c7eb 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -895,9 +895,9 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro { Chunk result_chunk = apply_squashing.add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - if (result_chunk.hasColumns()) + if (result_chunk.hasColumns() && apply_squashing.header) for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.block_for_insert.getDataTypes()[j], state.block_for_insert.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], apply_squashing.header.getDataTypes()[j], apply_squashing.header.getNames()[j])); auto result = Block(cols); return PushResult { @@ -912,9 +912,9 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro if (planned_chunk.hasChunkInfo()) result_chunk = apply_squashing.add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - if (result_chunk.hasColumns()) + if (result_chunk.hasColumns() && apply_squashing.header) for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.block_for_insert.getDataTypes()[j], state.block_for_insert.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], apply_squashing.header.getDataTypes()[j], apply_squashing.header.getNames()[j])); auto result = Block(cols); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 0a3a217d9434..3469b609f6b6 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1323,9 +1323,9 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - if (projection_chunk.hasColumns()) + if (projection_chunk.hasColumns() && projection_squashes[i].header) for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], block_to_squash.getDataTypes()[j], block_to_squash.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], projection_squashes[i].header.getDataTypes()[j], projection_squashes[i].header.getNames()[j])); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, Block(cols), projection, ctx->new_data_part.get(), ++block_num); tmp_part.finalize(); @@ -1351,7 +1351,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - if (projection_chunk.hasColumns()) + if (projection_chunk.hasColumns() && projection_squashes[i].header) for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], projection_squashes[i].header.getDataTypes()[j], projection_squashes[i].header.getNames()[j])); From d351c05243cc42dd05b3a4edf90dfe2044786e9a Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 29 May 2024 16:13:21 +0000 Subject: [PATCH 080/254] reset + try to fix mv and mutations --- .../Transforms/buildPushingToViewsChain.cpp | 15 +++++++++------ src/Server/TCPHandler.cpp | 6 +++--- src/Storages/MergeTree/MutateTask.cpp | 4 ++-- 3 files changed, 14 insertions(+), 11 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index ff1be9323f59..177d45650dde 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -367,13 +367,16 @@ std::optional generateViewChain( bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type; out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, thread_status_holder, view_counter_ms, check_access); - bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); - const auto & settings = insert_context->getSettingsRef(); + if (interpreter.shouldAddSquashingFroStorage(inner_table)) + { + bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); + const auto & settings = insert_context->getSettingsRef(); - out.addSource(std::make_shared( - out.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + out.addSource(std::make_shared( + out.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + } auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); counting->setProcessListElement(insert_context->getProcessListElement()); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b95face57e1f..3cbaffe857a9 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -895,7 +895,7 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro { Chunk result_chunk = apply_squashing.add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - if (result_chunk.hasColumns()) + if (result_chunk.hasColumns() && state.block_for_insert) for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.block_for_insert.getDataTypes()[j], state.block_for_insert.getNames()[j])); auto result = Block(cols); @@ -912,9 +912,9 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro if (planned_chunk.hasChunkInfo()) result_chunk = apply_squashing.add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - if (result_chunk.hasColumns()) + if (result_chunk.hasColumns() && apply_squashing.header) for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.block_for_insert.getDataTypes()[j], state.block_for_insert.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], apply_squashing.header.getDataTypes()[j], apply_squashing.header.getNames()[j])); auto result = Block(cols); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 0a3a217d9434..af36b7bb3e80 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1323,7 +1323,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - if (projection_chunk.hasColumns()) + if (projection_chunk.hasColumns() && block_to_squash) for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], block_to_squash.getDataTypes()[j], block_to_squash.getNames()[j])); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( @@ -1351,7 +1351,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - if (projection_chunk.hasColumns()) + if (projection_chunk.hasColumns() && projection_squashes[i].header) for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], projection_squashes[i].header.getDataTypes()[j], projection_squashes[i].header.getNames()[j])); From d86580ef049fc402d48808e3c125a61f824ed40f Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 29 May 2024 20:37:49 +0000 Subject: [PATCH 081/254] try to fix segfaults --- src/Interpreters/Squashing.h | 11 ++++++++++- src/Server/TCPHandler.cpp | 4 ++-- src/Storages/MergeTree/MutateTask.cpp | 5 +++-- 3 files changed, 15 insertions(+), 5 deletions(-) mode change 100644 => 100755 src/Interpreters/Squashing.h mode change 100644 => 100755 src/Server/TCPHandler.cpp mode change 100644 => 100755 src/Storages/MergeTree/MutateTask.cpp diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h old mode 100644 new mode 100755 index 05259bbc0c32..84e67e5d4c16 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -32,10 +32,19 @@ class ApplySquashing explicit ApplySquashing(Block header_); Chunk add(Chunk && input_chunk); - const Block header; + + void setHeader(Block header_) + { + header = header_; + } + Block getHeader() + { + return header; + } private: Chunk accumulated_chunk; + Block header; const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp old mode 100644 new mode 100755 index 3cbaffe857a9..77f84dba6e44 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -912,9 +912,9 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro if (planned_chunk.hasChunkInfo()) result_chunk = apply_squashing.add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - if (result_chunk.hasColumns() && apply_squashing.header) + if (result_chunk.hasColumns() && apply_squashing.getHeader()) for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], apply_squashing.header.getDataTypes()[j], apply_squashing.header.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], apply_squashing.getHeader().getDataTypes()[j], apply_squashing.getHeader().getNames()[j])); auto result = Block(cols); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp old mode 100644 new mode 100755 index af36b7bb3e80..ff1d7c0b7c20 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1331,6 +1331,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() tmp_part.finalize(); tmp_part.part->getDataPartStorage().commitTransaction(); projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); + projection_squashes[i].setHeader(block_to_squash); } } @@ -1351,9 +1352,9 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - if (projection_chunk.hasColumns() && projection_squashes[i].header) + if (projection_chunk.hasColumns() && projection_squashes[i].getHeader()) for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], projection_squashes[i].header.getDataTypes()[j], projection_squashes[i].header.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], projection_squashes[i].getHeader().getDataTypes()[j], projection_squashes[i].getHeader().getNames()[j])); auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, Block(cols), projection, ctx->new_data_part.get(), ++block_num); From aa52e9036ef1aef21c037bb2d8f3722f4cd24de3 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 29 May 2024 23:29:19 +0000 Subject: [PATCH 082/254] reset last commit --- src/Interpreters/Squashing.h | 11 ++++++++++- src/Server/TCPHandler.cpp | 4 ++-- src/Storages/MergeTree/MutateTask.cpp | 5 +++-- 3 files changed, 15 insertions(+), 5 deletions(-) mode change 100644 => 100755 src/Interpreters/Squashing.h mode change 100644 => 100755 src/Server/TCPHandler.cpp mode change 100644 => 100755 src/Storages/MergeTree/MutateTask.cpp diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h old mode 100644 new mode 100755 index 05259bbc0c32..84e67e5d4c16 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -32,10 +32,19 @@ class ApplySquashing explicit ApplySquashing(Block header_); Chunk add(Chunk && input_chunk); - const Block header; + + void setHeader(Block header_) + { + header = header_; + } + Block getHeader() + { + return header; + } private: Chunk accumulated_chunk; + Block header; const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp old mode 100644 new mode 100755 index 3cbaffe857a9..77f84dba6e44 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -912,9 +912,9 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro if (planned_chunk.hasChunkInfo()) result_chunk = apply_squashing.add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - if (result_chunk.hasColumns() && apply_squashing.header) + if (result_chunk.hasColumns() && apply_squashing.getHeader()) for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], apply_squashing.header.getDataTypes()[j], apply_squashing.header.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], apply_squashing.getHeader().getDataTypes()[j], apply_squashing.getHeader().getNames()[j])); auto result = Block(cols); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp old mode 100644 new mode 100755 index af36b7bb3e80..ff1d7c0b7c20 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1331,6 +1331,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() tmp_part.finalize(); tmp_part.part->getDataPartStorage().commitTransaction(); projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); + projection_squashes[i].setHeader(block_to_squash); } } @@ -1351,9 +1352,9 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - if (projection_chunk.hasColumns() && projection_squashes[i].header) + if (projection_chunk.hasColumns() && projection_squashes[i].getHeader()) for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], projection_squashes[i].header.getDataTypes()[j], projection_squashes[i].header.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], projection_squashes[i].getHeader().getDataTypes()[j], projection_squashes[i].getHeader().getNames()[j])); auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, Block(cols), projection, ctx->new_data_part.get(), ++block_num); From b160548aafc07e0db47ece097943cf3e61422c4c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 29 May 2024 23:39:16 +0000 Subject: [PATCH 083/254] change the chmod back --- src/Interpreters/Squashing.h | 0 src/Server/TCPHandler.cpp | 0 src/Storages/MergeTree/MutateTask.cpp | 0 3 files changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 src/Interpreters/Squashing.h mode change 100755 => 100644 src/Server/TCPHandler.cpp mode change 100755 => 100644 src/Storages/MergeTree/MutateTask.cpp diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h old mode 100755 new mode 100644 diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp old mode 100755 new mode 100644 diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp old mode 100755 new mode 100644 From 3e0947c759f5b9a70add338681cfcb660388e2a8 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 30 May 2024 00:16:35 +0000 Subject: [PATCH 084/254] try to remove if in mv --- .../Transforms/buildPushingToViewsChain.cpp | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 177d45650dde..ff1be9323f59 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -367,16 +367,13 @@ std::optional generateViewChain( bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type; out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, thread_status_holder, view_counter_ms, check_access); - if (interpreter.shouldAddSquashingFroStorage(inner_table)) - { - bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); - const auto & settings = insert_context->getSettingsRef(); + bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); + const auto & settings = insert_context->getSettingsRef(); - out.addSource(std::make_shared( - out.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); - } + out.addSource(std::make_shared( + out.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); counting->setProcessListElement(insert_context->getProcessListElement()); From cadf9d466664ceb693dbebb31a3f3df57af84c8b Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 30 May 2024 10:41:36 +0000 Subject: [PATCH 085/254] Revert "try to remove if in mv" This reverts commit 3e0947c759f5b9a70add338681cfcb660388e2a8. --- .../Transforms/buildPushingToViewsChain.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index ff1be9323f59..177d45650dde 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -367,13 +367,16 @@ std::optional generateViewChain( bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type; out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, thread_status_holder, view_counter_ms, check_access); - bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); - const auto & settings = insert_context->getSettingsRef(); + if (interpreter.shouldAddSquashingFroStorage(inner_table)) + { + bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); + const auto & settings = insert_context->getSettingsRef(); - out.addSource(std::make_shared( - out.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + out.addSource(std::make_shared( + out.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + } auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); counting->setProcessListElement(insert_context->getProcessListElement()); From 6c6bf069e211c17182d6b54d0afdaff48f932bfe Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 30 May 2024 12:07:18 +0000 Subject: [PATCH 086/254] remove moving of header --- src/Interpreters/Squashing.cpp | 4 ++-- src/Storages/MergeTree/MutateTask.cpp | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 9ecd92f732cf..47add4954219 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes } ApplySquashing::ApplySquashing(Block header_) - : header(std::move(header_)) + : header(header_) { } @@ -71,7 +71,7 @@ const ChunksToSquash* ApplySquashing::getInfoFromChunk(const Chunk & chunk) PlanSquashing::PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) - , header(std::move(header_)) + , header(header_) { } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index ff1d7c0b7c20..2269b16b443c 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1314,10 +1314,9 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { const auto & projection = *ctx->projections_to_build[i]; - Chunk planned_chunk; ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); Block block_to_squash = projection.calculate(cur_block, ctx->context); - planned_chunk = projection_squash_plannings[i]->add({block_to_squash.getColumns(), block_to_squash.rows()}); + Chunk planned_chunk = projection_squash_plannings[i]->add({block_to_squash.getColumns(), block_to_squash.rows()}); if (planned_chunk.hasChunkInfo()) { From 0579fc9436528221d88ffc02f23e42de7ad4dc81 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 30 May 2024 14:48:14 +0000 Subject: [PATCH 087/254] remove moving from planSquashing --- src/Interpreters/Squashing.cpp | 2 +- src/Interpreters/Squashing.h | 2 +- src/Processors/Transforms/PlanSquashingTransform.cpp | 2 +- src/Processors/Transforms/SquashingTransform.cpp | 4 ++-- src/Server/TCPHandler.cpp | 3 ++- src/Storages/MergeTree/MutateTask.cpp | 5 +++-- 6 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 47add4954219..6706399a3d25 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -80,7 +80,7 @@ Chunk PlanSquashing::flush() return convertToChunk(std::move(chunks_to_merge_vec)); } -Chunk PlanSquashing::add(Chunk && input_chunk) +Chunk PlanSquashing::add(Chunk & input_chunk) { if (!input_chunk) return {}; diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index 84e67e5d4c16..802e77847e98 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -59,7 +59,7 @@ class PlanSquashing public: explicit PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); - Chunk add(Chunk && input_chunk); + Chunk add(Chunk & input_chunk); Chunk flush(); bool isDataLeft() { diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 7945bd97e04e..1384f760d486 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -134,7 +134,7 @@ IProcessor::Status PlanSquashingTransform::waitForDataIn() void PlanSquashingTransform::transform(Chunk & chunk_) { - Chunk res_chunk = balance.add(std::move(chunk_)); + Chunk res_chunk = balance.add(chunk_); std::swap(res_chunk, chunk_); } diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index a516811bf456..67358316d48d 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -19,7 +19,7 @@ SquashingTransform::SquashingTransform( void SquashingTransform::onConsume(Chunk chunk) { - Chunk planned_chunk = planSquashing.add(std::move(chunk)); + Chunk planned_chunk = planSquashing.add(chunk); if (planned_chunk.hasChunkInfo()) cur_chunk = applySquashing.add(std::move(planned_chunk)); } @@ -69,7 +69,7 @@ void SimpleSquashingTransform::transform(Chunk & chunk) { if (!finished) { - Chunk planned_chunk = planSquashing.add(std::move(chunk)); + Chunk planned_chunk = planSquashing.add(chunk); if (planned_chunk.hasChunkInfo()) chunk = applySquashing.add(std::move(planned_chunk)); } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 77f84dba6e44..6973808078ca 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -890,7 +890,8 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro while (readDataNext()) { - auto planned_chunk = plan_squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); + Chunk input_chunk = {state.block_for_insert.getColumns(), state.block_for_insert.rows()}; + auto planned_chunk = plan_squashing.add(input_chunk); if (planned_chunk.hasChunkInfo()) { Chunk result_chunk = apply_squashing.add(std::move(planned_chunk)); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 2269b16b443c..5267143bf65e 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1314,9 +1314,10 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { const auto & projection = *ctx->projections_to_build[i]; - ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); + ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); // Not clear why is it needed heee Block block_to_squash = projection.calculate(cur_block, ctx->context); - Chunk planned_chunk = projection_squash_plannings[i]->add({block_to_squash.getColumns(), block_to_squash.rows()}); + Chunk input_chunk = {block_to_squash.getColumns(), block_to_squash.rows()}; + Chunk planned_chunk = projection_squash_plannings[i]->add(input_chunk); if (planned_chunk.hasChunkInfo()) { From 826bec2575b8ccd3800b0e04be5422c000367dc2 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 30 May 2024 16:33:48 +0000 Subject: [PATCH 088/254] added move constructor and removed unused parts --- src/Interpreters/Squashing.cpp | 7 ++- src/Interpreters/Squashing.h | 8 ++-- .../Transforms/PlanSquashingTransform.cpp | 2 +- .../Transforms/SquashingTransform.cpp | 4 +- .../Transforms/buildPushingToViewsChain.cpp | 15 ++++--- src/Server/TCPHandler.cpp | 16 +++---- src/Storages/MergeTree/MutateTask.cpp | 43 ++++++++----------- 7 files changed, 47 insertions(+), 48 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 82d80114a85a..a15abf968d2a 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -1,7 +1,6 @@ #include #include #include -#include "Columns/IColumn.h" namespace DB @@ -69,9 +68,10 @@ const ChunksToSquash* ApplySquashing::getInfoFromChunk(const Chunk & chunk) return agg_info; } -PlanSquashing::PlanSquashing(size_t min_block_size_rows_, size_t min_block_size_bytes_) +PlanSquashing::PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) + , header(header_) { } @@ -141,8 +141,7 @@ Chunk PlanSquashing::convertToChunk(std::vector && chunks) chunks.clear(); - Columns cols = {}; - return Chunk(cols, 0, info); + return Chunk(header.cloneEmptyColumns(), 0, info); } void PlanSquashing::expandCurrentSize(size_t rows, size_t bytes) diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index d9d430c18359..77191e630500 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -33,9 +33,10 @@ class ApplySquashing Chunk add(Chunk && input_chunk); + Block header; + private: Chunk accumulated_chunk; - const Block header; const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); @@ -48,7 +49,8 @@ class ApplySquashing class PlanSquashing { public: - PlanSquashing(size_t min_block_size_rows_, size_t min_block_size_bytes_); + explicit PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); + PlanSquashing(PlanSquashing && other) = default; Chunk add(Chunk && input_chunk); Chunk flush(); @@ -68,7 +70,7 @@ class PlanSquashing size_t min_block_size_rows; size_t min_block_size_bytes; - // const Block header; + const Block header; CurrentSize accumulated_size; void expandCurrentSize(size_t rows, size_t bytes); diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 96f41e37d2fc..7945bd97e04e 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes } PlanSquashingTransform::PlanSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) - : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), balance(min_block_size_rows, min_block_size_bytes) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), balance(header, min_block_size_rows, min_block_size_bytes) { } diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 6f7c877b2f3f..a516811bf456 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -12,7 +12,7 @@ extern const int LOGICAL_ERROR; SquashingTransform::SquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , planSquashing(min_block_size_rows, min_block_size_bytes) + , planSquashing(header, min_block_size_rows, min_block_size_bytes) , applySquashing(header) { } @@ -60,7 +60,7 @@ void SquashingTransform::work() SimpleSquashingTransform::SimpleSquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ISimpleTransform(header, header, false) - , planSquashing(min_block_size_rows, min_block_size_bytes) + , planSquashing(header, min_block_size_rows, min_block_size_bytes) , applySquashing(header) { } diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index ff1be9323f59..177d45650dde 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -367,13 +367,16 @@ std::optional generateViewChain( bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type; out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, thread_status_holder, view_counter_ms, check_access); - bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); - const auto & settings = insert_context->getSettingsRef(); + if (interpreter.shouldAddSquashingFroStorage(inner_table)) + { + bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); + const auto & settings = insert_context->getSettingsRef(); - out.addSource(std::make_shared( - out.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + out.addSource(std::make_shared( + out.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + } auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); counting->setProcessListElement(insert_context->getProcessListElement()); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 476c4dd372b9..06f20fef613d 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -885,20 +885,19 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro using PushResult = AsynchronousInsertQueue::PushResult; startInsertQuery(); - PlanSquashing plan_squashing(0, query_context->getSettingsRef().async_insert_max_data_size); + PlanSquashing plan_squashing(state.input_header, 0, query_context->getSettingsRef().async_insert_max_data_size); ApplySquashing apply_squashing(state.input_header); while (readDataNext()) { auto planned_chunk = plan_squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); - Chunk result_chunk; if (planned_chunk.hasChunkInfo()) - result_chunk = apply_squashing.add(std::move(planned_chunk)); - if (result_chunk) { + Chunk result_chunk = apply_squashing.add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.input_header.getDataTypes()[j], state.input_header.getNames()[j])); + if (result_chunk.hasColumns() && state.block_for_insert) + for (size_t j = 0; j < result_chunk.getNumColumns(); ++j) + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.block_for_insert.getDataTypes()[j], state.block_for_insert.getNames()[j])); auto result = Block(cols); return PushResult { @@ -913,8 +912,9 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro if (planned_chunk.hasChunkInfo()) result_chunk = apply_squashing.add(std::move(planned_chunk)); ColumnsWithTypeAndName cols; - for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.input_header.getDataTypes()[j], state.input_header.getNames()[j])); + if (result_chunk.hasColumns()) + for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.input_header.getDataTypes()[j], state.input_header.getNames()[j])); auto result = Block(cols); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index f7a4651f6fda..c24a643c3745 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1287,7 +1287,7 @@ void PartMergerWriter::prepare() for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { // We split the materialization into multiple stages similar to the process of INSERT SELECT query. - projection_squash_plannings.emplace_back(settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + projection_squash_plannings.emplace_back(ctx->updated_header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); projection_squashes.emplace_back(ctx->updated_header); } @@ -1313,24 +1313,20 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { const auto & projection = *ctx->projections_to_build[i]; - Block projection_block; - { - ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); - Block to_plan = projection.calculate(cur_block, ctx->context); - Chunk planned_chunk = projection_squash_plannings[i].add({to_plan.getColumns(), to_plan.rows()}); - Chunk projection_chunk; - if (planned_chunk.hasChunkInfo()) - projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); - ColumnsWithTypeAndName cols; - for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); - projection_block = Block(cols); - } + ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); + Block block_to_squash = projection.calculate(cur_block, ctx->context); + projection_squashes[i].header = block_to_squash; + Chunk planned_chunk = projection_squash_plannings[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); - if (projection_block) + if (planned_chunk.hasChunkInfo()) { + Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); + ColumnsWithTypeAndName cols; + if (projection_chunk.hasColumns()) + for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) + cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], block_to_squash.getDataTypes()[j], block_to_squash.getNames()[j])); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( - *ctx->data, ctx->log, projection_block, projection, ctx->new_data_part.get(), ++block_num); + *ctx->data, ctx->log, Block(cols), projection, ctx->new_data_part.get(), ++block_num); tmp_part.finalize(); tmp_part.part->getDataPartStorage().commitTransaction(); projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); @@ -1350,17 +1346,16 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() const auto & projection = *ctx->projections_to_build[i]; auto & projection_squash_plan = projection_squash_plannings[i]; auto planned_chunk = projection_squash_plan.flush(); - Chunk projection_chunk; if (planned_chunk.hasChunkInfo()) - projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); - ColumnsWithTypeAndName cols; - for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], ctx->updated_header.getDataTypes()[j], ctx->updated_header.getNames()[j])); - auto projection_block = Block(cols); - if (projection_block) { + Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); + ColumnsWithTypeAndName cols; + if (projection_chunk.hasColumns()) + for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) + cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], projection_squashes[i].header.getDataTypes()[j], projection_squashes[i].header.getNames()[j])); + auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( - *ctx->data, ctx->log, projection_block, projection, ctx->new_data_part.get(), ++block_num); + *ctx->data, ctx->log, Block(cols), projection, ctx->new_data_part.get(), ++block_num); temp_part.finalize(); temp_part.part->getDataPartStorage().commitTransaction(); projection_parts[projection.name].emplace_back(std::move(temp_part.part)); From 5a09dce95d0c1be55e9283dcf49b8fb5592f923d Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 30 May 2024 18:06:11 +0000 Subject: [PATCH 089/254] adding header in TCPHandler --- src/Server/TCPHandler.cpp | 6 +++++- src/Storages/MergeTree/MutateTask.cpp | 3 ++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 06f20fef613d..070073f414d3 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -890,6 +890,8 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro while (readDataNext()) { + if (!apply_squashing.header) + apply_squashing.header = state.block_for_insert; auto planned_chunk = plan_squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); if (planned_chunk.hasChunkInfo()) { @@ -899,6 +901,7 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro for (size_t j = 0; j < result_chunk.getNumColumns(); ++j) cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.block_for_insert.getDataTypes()[j], state.block_for_insert.getNames()[j])); auto result = Block(cols); + apply_squashing.header = Block(state.block_for_insert); return PushResult { .status = PushResult::TOO_MUCH_DATA, @@ -914,7 +917,8 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro ColumnsWithTypeAndName cols; if (result_chunk.hasColumns()) for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.input_header.getDataTypes()[j], state.input_header.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], apply_squashing.header.getDataTypes()[j], apply_squashing.header.getNames()[j])); + auto result = Block(cols); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 0d1fc46ec76d..e5285a970dde 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1316,7 +1316,8 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); Block block_to_squash = projection.calculate(cur_block, ctx->context); - projection_squashes[i].header = block_to_squash; + if (!projection_squashes[i].header) + projection_squashes[i].header = block_to_squash; Chunk planned_chunk = projection_squash_plannings[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); if (planned_chunk.hasChunkInfo()) From 84c8c4ca482805a762ba03152cd8f8e412cec1ca Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 30 May 2024 18:28:24 +0000 Subject: [PATCH 090/254] try to assign header unconditionally --- .../Transforms/buildPushingToViewsChain.cpp | 15 ++++++--------- src/Server/TCPHandler.cpp | 3 +-- src/Storages/MergeTree/MutateTask.cpp | 3 +-- 3 files changed, 8 insertions(+), 13 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 177d45650dde..ff1be9323f59 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -367,16 +367,13 @@ std::optional generateViewChain( bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type; out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, thread_status_holder, view_counter_ms, check_access); - if (interpreter.shouldAddSquashingFroStorage(inner_table)) - { - bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); - const auto & settings = insert_context->getSettingsRef(); + bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); + const auto & settings = insert_context->getSettingsRef(); - out.addSource(std::make_shared( - out.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); - } + out.addSource(std::make_shared( + out.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); counting->setProcessListElement(insert_context->getProcessListElement()); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 070073f414d3..1dd997967544 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -890,8 +890,7 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro while (readDataNext()) { - if (!apply_squashing.header) - apply_squashing.header = state.block_for_insert; + apply_squashing.header = state.block_for_insert; auto planned_chunk = plan_squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); if (planned_chunk.hasChunkInfo()) { diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index e5285a970dde..0d1fc46ec76d 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1316,8 +1316,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); Block block_to_squash = projection.calculate(cur_block, ctx->context); - if (!projection_squashes[i].header) - projection_squashes[i].header = block_to_squash; + projection_squashes[i].header = block_to_squash; Chunk planned_chunk = projection_squash_plannings[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); if (planned_chunk.hasChunkInfo()) From 43cb255394a4d93a1b4703f01d56ac2144c3881c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 30 May 2024 18:46:56 +0000 Subject: [PATCH 091/254] [TEMP] Change refence for 2290_async_queries_in_query_log --- .../0_stateless/02790_async_queries_in_query_log.reference | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/02790_async_queries_in_query_log.reference b/tests/queries/0_stateless/02790_async_queries_in_query_log.reference index aa18817f4e6a..af8244c06b8e 100644 --- a/tests/queries/0_stateless/02790_async_queries_in_query_log.reference +++ b/tests/queries/0_stateless/02790_async_queries_in_query_log.reference @@ -101,6 +101,13 @@ table: async_insert_landing partition_id: all rows: 3 +Row 2: +────── +database: default +table: async_insert_target +partition_id: all +rows: 3 + system.query_log Row 1: ────── From a7aa24a968fec07b72872881470878949c329738 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Thu, 30 May 2024 21:19:12 +0000 Subject: [PATCH 092/254] hilbert index analysis --- src/Common/HilbertUtils.h | 109 ++++++++++++++++++++++++ src/Storages/MergeTree/KeyCondition.cpp | 67 +++++++++++---- src/Storages/MergeTree/KeyCondition.h | 10 +++ 3 files changed, 168 insertions(+), 18 deletions(-) create mode 100644 src/Common/HilbertUtils.h diff --git a/src/Common/HilbertUtils.h b/src/Common/HilbertUtils.h new file mode 100644 index 000000000000..0adde3a0d1f7 --- /dev/null +++ b/src/Common/HilbertUtils.h @@ -0,0 +1,109 @@ +#include +#include +#include "base/types.h" +#include +#include +#include +#include + + +namespace HilbertDetails +{ + + struct Segment // represents [begin; end], all bounds are included + { + UInt64 begin; + UInt64 end; + }; + +} + + +template +void segmentBinaryPartition(UInt64 start, UInt64 finish, UInt8 current_bits, F && callback) +{ + if (current_bits == 0) + return; + + auto next_bits = current_bits - 2; + auto history = (start >> current_bits) << current_bits; + + auto start_chunk = (start >> next_bits) & 0b11; + auto finish_chunk = (finish >> next_bits) & 0b11; + + auto construct_range = [next_bits, history](UInt64 chunk) + { + return HilbertDetails::Segment{ + .begin = history + (chunk << next_bits), + .end = history + ((chunk + 1) << next_bits) - 1 + }; + }; + + if (start_chunk == finish_chunk) + { + if ((finish - start + 1) == (1 << next_bits)) // it means that [begin, end] is a range + { + callback(HilbertDetails::Segment{.begin = start, .end = finish}); + return; + } + segmentBinaryPartition(start, finish, next_bits, callback); + return; + } + + for (auto range_chunk = start_chunk + 1; range_chunk < finish_chunk; ++range_chunk) + { + callback(construct_range(range_chunk)); + } + + auto start_range = construct_range(start_chunk); + if (start == start_range.begin) + { + callback(start_range); + } + else + { + segmentBinaryPartition(start, start_range.end, next_bits, callback); + } + + auto finish_range = construct_range(finish_chunk); + if (finish == finish_range.end) + { + callback(finish_range); + } + else + { + segmentBinaryPartition(finish_range.begin, finish, next_bits, callback); + } +} + +std::array, 2> createRangeFromCorners(UInt64 x1, UInt64 y1, UInt64 x2, UInt64 y2) +{ + UInt64 dist_x = x1 > x2 ? x1 - x2 : x2 - x1; + UInt64 dist_y = y1 > y2 ? y1 - y2 : y2 - y1; + UInt64 range_size = std::max(dist_x, dist_y); + UInt64 x_min = std::min(x1, x2); + UInt64 y_min = std::min(y1, y2); + return { + std::pair{x_min, x_min + range_size}, + std::pair{y_min, y_min + range_size} + }; +} + +/** Unpack an interval of Hilbert curve to hyperrectangles covered by it across N dimensions. + */ +template +void hilbertIntervalToHyperrectangles2D(UInt64 first, UInt64 last, F && callback) +{ + segmentBinaryPartition(first, last, 64, [&](HilbertDetails::Segment range) + { + + auto interval1 = DB::FunctionHilbertDecode2DWIthLookupTableImpl<2>::decode(range.begin); + auto interval2 = DB::FunctionHilbertDecode2DWIthLookupTableImpl<2>::decode(range.end); + + std::array, 2> unpacked_range = createRangeFromCorners( + std::get<0>(interval1), std::get<1>(interval1), + std::get<0>(interval2), std::get<1>(interval2)); + + callback(unpacked_range); + }); +} diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index bd8642b9f660..667f31b36b56 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -675,6 +675,11 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( return *res; } +const std::unordered_map KeyCondition::space_filling_curve_name_to_type { + {"mortonEncode", SpaceFillingCurveType::Morton}, + {"hilbertEncode", SpaceFillingCurveType::Hilbert} +}; + ActionsDAGPtr KeyCondition::cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context) { auto res = std::make_shared(); @@ -730,15 +735,15 @@ static NameSet getAllSubexpressionNames(const ExpressionActions & key_expr) void KeyCondition::getAllSpaceFillingCurves() { - /// So far the only supported function is mortonEncode (Morton curve). - for (const auto & action : key_expr->getActions()) { + auto space_filling_curve_type_iter = space_filling_curve_name_to_type.find(action.node->function_base->getName()); if (action.node->type == ActionsDAG::ActionType::FUNCTION && action.node->children.size() >= 2 - && action.node->function_base->getName() == "mortonEncode") + && space_filling_curve_type_iter != space_filling_curve_name_to_type.end()) { SpaceFillingCurveDescription curve; + curve.type = space_filling_curve_type_iter->second; curve.function_name = action.node->function_base->getName(); curve.key_column_pos = key_columns.at(action.node->result_name); for (const auto & child : action.node->children) @@ -2649,6 +2654,15 @@ BoolMask KeyCondition::checkInHyperrectangle( const DataTypes & data_types) const { std::vector rpn_stack; + + auto curve_type = [&](size_t key_column_pos) + { + for (const auto & curve : key_space_filling_curves) + if (curve.key_column_pos == key_column_pos) + return curve.type; + return SpaceFillingCurveType::Unknown; + }; + for (const auto & element : rpn) { if (element.argument_num_of_space_filling_curve.has_value()) @@ -2748,26 +2762,43 @@ BoolMask KeyCondition::checkInHyperrectangle( UInt64 right = key_range.right.get(); BoolMask mask(false, true); - mortonIntervalToHyperrectangles<2>(left, right, - [&](std::array, 2> morton_hyperrectangle) + auto hyperrectangle_intersection_callback = [&](std::array, 2> curve_hyperrectangle) + { + BoolMask current_intersection(true, false); + for (size_t dim = 0; dim < num_dimensions; ++dim) { - BoolMask current_intersection(true, false); - for (size_t dim = 0; dim < num_dimensions; ++dim) - { - const Range & condition_arg_range = element.space_filling_curve_args_hyperrectangle[dim]; + const Range & condition_arg_range = element.space_filling_curve_args_hyperrectangle[dim]; - const Range morton_arg_range( - morton_hyperrectangle[dim].first, true, - morton_hyperrectangle[dim].second, true); + const Range curve_arg_range( + curve_hyperrectangle[dim].first, true, + curve_hyperrectangle[dim].second, true); - bool intersects = condition_arg_range.intersectsRange(morton_arg_range); - bool contains = condition_arg_range.containsRange(morton_arg_range); + bool intersects = condition_arg_range.intersectsRange(curve_arg_range); + bool contains = condition_arg_range.containsRange(curve_arg_range); - current_intersection = current_intersection & BoolMask(intersects, !contains); - } + current_intersection = current_intersection & BoolMask(intersects, !contains); + } + + mask = mask | current_intersection; + }; - mask = mask | current_intersection; - }); + switch (curve_type(element.key_column)) + { + case SpaceFillingCurveType::Hilbert: + { + hilbertIntervalToHyperrectangles2D(left, right, hyperrectangle_intersection_callback); + break; + } + case SpaceFillingCurveType::Morton: + { + mortonIntervalToHyperrectangles<2>(left, right, hyperrectangle_intersection_callback); + break; + } + case SpaceFillingCurveType::Unknown: + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "curve_type is `Unknown`. It is a bug."); + } + } rpn_stack.emplace_back(mask); } diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 6e248dd664af..c2700a52be1a 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -2,6 +2,8 @@ #include +#include + #include #include #include @@ -325,11 +327,19 @@ class KeyCondition const NameSet key_subexpr_names; /// Space-filling curves in the key + enum class SpaceFillingCurveType { + Unknown = 0, + Morton, + Hilbert + }; + static const std::unordered_map space_filling_curve_name_to_type; + struct SpaceFillingCurveDescription { size_t key_column_pos; String function_name; std::vector arguments; + SpaceFillingCurveType type; }; using SpaceFillingCurveDescriptions = std::vector; SpaceFillingCurveDescriptions key_space_filling_curves; From effc8ff0543998db85fb1b6ae51cda2d3f10a1bc Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Thu, 30 May 2024 23:04:27 +0000 Subject: [PATCH 093/254] describing functions --- src/Common/HilbertUtils.h | 58 +++++++++++++++++++++++++++++++-------- 1 file changed, 47 insertions(+), 11 deletions(-) diff --git a/src/Common/HilbertUtils.h b/src/Common/HilbertUtils.h index 0adde3a0d1f7..fe8c6bc035b0 100644 --- a/src/Common/HilbertUtils.h +++ b/src/Common/HilbertUtils.h @@ -1,3 +1,5 @@ +#pragma once + #include #include #include "base/types.h" @@ -18,18 +20,49 @@ namespace HilbertDetails } - +/* + Given the range of values of hilbert code - and this function will return segments of the Hilbert curve + such that each of them lies in a whole domain (aka square) + 0 1 + ┌────────────────────────────────┐ + │ │ │ + │ │ │ + 0 │ 00xxx │ 11xxx │ + │ | │ | │ + │ | │ | │ + │_______________│________________│ + │ | │ | │ + │ | │ | │ + │ | │ | │ + 1 │ 01xxx______│_____10xxx │ + │ │ │ + │ │ │ + └────────────────────────────────┘ + Imagine a square, one side of which is a x-axis, other is a y-axis. + First approximation of the Hilbert curve is on the picture - U curve + So we divide Hilbert Code Interval on 4 parts each of which is represented by a square: + [00xxxxxx | 01xxxxxx | 10xxxxxx | 11xxxxxx ] + 1: [ ] + start = 0010111 end = 10111110 + 2: [ ] [ ] + And look where the given interval [start, end] is located. If it contains whole sector (that represents a domain=square), + then we take this range. int he example above - it is a sector [01000000, 01111111] + Then we dig into the recursion and check the remaing ranges + Note that after first call all other ranges in the recursion will have either start or end on the end of a range, + so the comlexity of the algorithm will be O(logN), not O(N), where N is the maximum of hilbert code. +*/ template void segmentBinaryPartition(UInt64 start, UInt64 finish, UInt8 current_bits, F && callback) { if (current_bits == 0) return; - auto next_bits = current_bits - 2; - auto history = (start >> current_bits) << current_bits; + const auto next_bits = current_bits - 2; + const auto history = (start >> current_bits) << current_bits; - auto start_chunk = (start >> next_bits) & 0b11; - auto finish_chunk = (finish >> next_bits) & 0b11; + const auto chunk_mask = 0b11; + const auto start_chunk = (start >> next_bits) & chunk_mask; + const auto finish_chunk = (finish >> next_bits) & chunk_mask; auto construct_range = [next_bits, history](UInt64 chunk) { @@ -55,7 +88,7 @@ void segmentBinaryPartition(UInt64 start, UInt64 finish, UInt8 current_bits, F & callback(construct_range(range_chunk)); } - auto start_range = construct_range(start_chunk); + const auto start_range = construct_range(start_chunk); if (start == start_range.begin) { callback(start_range); @@ -65,7 +98,7 @@ void segmentBinaryPartition(UInt64 start, UInt64 finish, UInt8 current_bits, F & segmentBinaryPartition(start, start_range.end, next_bits, callback); } - auto finish_range = construct_range(finish_chunk); + const auto finish_range = construct_range(finish_chunk); if (finish == finish_range.end) { callback(finish_range); @@ -76,6 +109,8 @@ void segmentBinaryPartition(UInt64 start, UInt64 finish, UInt8 current_bits, F & } } +// Given 2 points representing ends of the range of Hilbert Curve that lies in a whole domain. +// The are neighboor corners of some square - and the function returns ranges of both sides of this square std::array, 2> createRangeFromCorners(UInt64 x1, UInt64 y1, UInt64 x2, UInt64 y2) { UInt64 dist_x = x1 > x2 ? x1 - x2 : x2 - x1; @@ -94,11 +129,12 @@ std::array, 2> createRangeFromCorners(UInt64 x1, UInt6 template void hilbertIntervalToHyperrectangles2D(UInt64 first, UInt64 last, F && callback) { - segmentBinaryPartition(first, last, 64, [&](HilbertDetails::Segment range) + const auto equal_bits_count = getLeadingZeroBits(last - first); + const auto even_equal_bits_count = equal_bits_count - equal_bits_count % 2; + segmentBinaryPartition(first, last, 64 - even_equal_bits_count, [&](HilbertDetails::Segment range) { - - auto interval1 = DB::FunctionHilbertDecode2DWIthLookupTableImpl<2>::decode(range.begin); - auto interval2 = DB::FunctionHilbertDecode2DWIthLookupTableImpl<2>::decode(range.end); + auto interval1 = DB::FunctionHilbertDecode2DWIthLookupTableImpl<3>::decode(range.begin); + auto interval2 = DB::FunctionHilbertDecode2DWIthLookupTableImpl<3>::decode(range.end); std::array, 2> unpacked_range = createRangeFromCorners( std::get<0>(interval1), std::get<1>(interval1), From c857099c3d19cce02e3ff80f3bf3faa7656eac28 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 31 May 2024 13:42:16 +0000 Subject: [PATCH 094/254] un-flaky test --- tests/queries/0_stateless/02790_async_queries_in_query_log.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02790_async_queries_in_query_log.sh b/tests/queries/0_stateless/02790_async_queries_in_query_log.sh index 1ff97031acba..a74519a76085 100755 --- a/tests/queries/0_stateless/02790_async_queries_in_query_log.sh +++ b/tests/queries/0_stateless/02790_async_queries_in_query_log.sh @@ -65,6 +65,7 @@ function print_flush_query_logs() WHERE event_date >= yesterday() AND query_id = (SELECT flush_query_id FROM system.asynchronous_insert_log WHERE event_date >= yesterday() AND query_id = '$1') + ORDER BY table FORMAT Vertical" } From 79442f226532a31cd8546af7b5fb36263c1a8f7c Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Mon, 3 Jun 2024 13:03:29 +0000 Subject: [PATCH 095/254] style --- src/Common/HilbertUtils.h | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/src/Common/HilbertUtils.h b/src/Common/HilbertUtils.h index fe8c6bc035b0..9a6ebe9fe81b 100644 --- a/src/Common/HilbertUtils.h +++ b/src/Common/HilbertUtils.h @@ -39,17 +39,18 @@ namespace HilbertDetails │ │ │ └────────────────────────────────┘ Imagine a square, one side of which is a x-axis, other is a y-axis. - First approximation of the Hilbert curve is on the picture - U curve - So we divide Hilbert Code Interval on 4 parts each of which is represented by a square: + First approximation of the Hilbert curve is on the picture - U curve. + So we divide Hilbert Code Interval on 4 parts each of which is represented by a square + and look where the given interval [start, finish] is located: [00xxxxxx | 01xxxxxx | 10xxxxxx | 11xxxxxx ] 1: [ ] start = 0010111 end = 10111110 2: [ ] [ ] - And look where the given interval [start, end] is located. If it contains whole sector (that represents a domain=square), - then we take this range. int he example above - it is a sector [01000000, 01111111] - Then we dig into the recursion and check the remaing ranges - Note that after first call all other ranges in the recursion will have either start or end on the end of a range, - so the comlexity of the algorithm will be O(logN), not O(N), where N is the maximum of hilbert code. + If it contains a whole sector (that represents a domain=square), + then we take this range. In the example above - it is a sector [01000000, 01111111] + Then we dig into the recursion and check the remaining ranges. + Note that after the first call all other ranges in the recursion will have either start or finish on the end of a range, + so the complexity of the algorithm will be O(logN), where N is the maximum of hilbert code. */ template void segmentBinaryPartition(UInt64 start, UInt64 finish, UInt8 current_bits, F && callback) @@ -110,8 +111,8 @@ void segmentBinaryPartition(UInt64 start, UInt64 finish, UInt8 current_bits, F & } // Given 2 points representing ends of the range of Hilbert Curve that lies in a whole domain. -// The are neighboor corners of some square - and the function returns ranges of both sides of this square -std::array, 2> createRangeFromCorners(UInt64 x1, UInt64 y1, UInt64 x2, UInt64 y2) +// The are neighbour corners of some square - and the function returns ranges of both sides of this square +inline std::array, 2> createRangeFromCorners(UInt64 x1, UInt64 y1, UInt64 x2, UInt64 y2) { UInt64 dist_x = x1 > x2 ? x1 - x2 : x2 - x1; UInt64 dist_y = y1 > y2 ? y1 - y2 : y2 - y1; From 5e645699d7c0f230bb510230f5528048fcbbe1b7 Mon Sep 17 00:00:00 2001 From: Artem Mustafin <24766926+Artemmm91@users.noreply.github.com> Date: Mon, 3 Jun 2024 16:05:12 +0300 Subject: [PATCH 096/254] style --- src/Storages/MergeTree/KeyCondition.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index c2700a52be1a..d1962ebfe65e 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -327,7 +327,8 @@ class KeyCondition const NameSet key_subexpr_names; /// Space-filling curves in the key - enum class SpaceFillingCurveType { + enum class SpaceFillingCurveType + { Unknown = 0, Morton, Hilbert From 711682a89169566637583434c693280e62a5ea73 Mon Sep 17 00:00:00 2001 From: Artem Mustafin <24766926+Artemmm91@users.noreply.github.com> Date: Mon, 3 Jun 2024 17:00:07 +0300 Subject: [PATCH 097/254] style --- src/Storages/MergeTree/KeyCondition.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index d1962ebfe65e..3a921987c49b 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -327,7 +327,7 @@ class KeyCondition const NameSet key_subexpr_names; /// Space-filling curves in the key - enum class SpaceFillingCurveType + enum class SpaceFillingCurveType { Unknown = 0, Morton, From e449f83cfaa548cec8b17ddfec1123e0a7b27c84 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Jun 2024 15:28:32 +0000 Subject: [PATCH 098/254] Merge consequent FilterSteps. --- .../Optimizations/mergeExpressions.cpp | 52 +++++++++++++++++++ ...plan_optimizations_merge_filters.reference | 10 ++++ ...01655_plan_optimizations_merge_filters.sql | 5 ++ 3 files changed, 67 insertions(+) create mode 100644 tests/queries/0_stateless/01655_plan_optimizations_merge_filters.reference create mode 100644 tests/queries/0_stateless/01655_plan_optimizations_merge_filters.sql diff --git a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp index a5cb5972bd87..7cd538e78b3f 100644 --- a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp +++ b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp @@ -2,10 +2,25 @@ #include #include #include +#include +#include namespace DB::QueryPlanOptimizations { +static void removeFromOutputs(ActionsDAG & dag, const ActionsDAG::Node & node) +{ + auto & outputs = dag.getOutputs(); + for (size_t i = 0; i < outputs.size(); ++i) + { + if (&node == outputs[i]) + { + outputs.erase(outputs.begin() + i); + return; + } + } +} + size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) { if (parent_node->children.size() != 1) @@ -19,6 +34,7 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) auto * parent_expr = typeid_cast(parent.get()); auto * parent_filter = typeid_cast(parent.get()); auto * child_expr = typeid_cast(child.get()); + auto * child_filter = typeid_cast(child.get()); if (parent_expr && child_expr) { @@ -60,6 +76,42 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) parent_node->children.swap(child_node->children); return 1; } + else if (parent_filter && child_filter) + { + const auto & child_actions = child_filter->getExpression(); + const auto & parent_actions = parent_filter->getExpression(); + + if (child_actions->hasArrayJoin() && parent_actions->hasStatefulFunctions()) + return 0; + + auto actions = child_actions->clone(); + const auto & child_filter_node = actions->findInOutputs(child_filter->getFilterColumnName()); + if (child_filter->removesFilterColumn()) + removeFromOutputs(*actions, child_filter_node); + + actions->mergeInplace(std::move(*parent_actions->clone())); + + const auto & parent_filter_node = actions->findInOutputs(parent_filter->getFilterColumnName()); + if (parent_filter->removesFilterColumn()) + removeFromOutputs(*actions, parent_filter_node); + + FunctionOverloadResolverPtr func_builder_and = std::make_unique(std::make_shared()); + const auto & condition = actions->addFunction(func_builder_and, {&child_filter_node, &parent_filter_node}, {}); + auto & outputs = actions->getOutputs(); + outputs.insert(outputs.begin(), &condition); + + actions->removeUnusedActions(false); + + auto filter = std::make_unique(child_filter->getInputStreams().front(), + actions, + condition.result_name, + true); + filter->setStepDescription("(" + parent_filter->getStepDescription() + " + " + child_filter->getStepDescription() + ")"); + + parent_node->step = std::move(filter); + parent_node->children.swap(child_node->children); + return 1; + } return 0; } diff --git a/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.reference b/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.reference new file mode 100644 index 000000000000..4905c7f8a713 --- /dev/null +++ b/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.reference @@ -0,0 +1,10 @@ + Filter (((WHERE + (Change column names to column identifiers + (Project names + Projection))) + HAVING)) + Filter column: and(notEquals(sum(__table2.number), 0_UInt8), equals(__table1.key, 7_UInt8)) (removed) + Aggregating + Filter (( + (Before GROUP BY + Change column names to column identifiers))) + Filter column: equals(__table1.key, 7_UInt8) (removed) + Filter (((WHERE + (Projection + Before ORDER BY)) + HAVING)) + Filter column: and(notEquals(sum(number), 0), equals(key, 7)) (removed) + Aggregating + Filter ((( + Before GROUP BY) + WHERE)) + Filter column: and(equals(bitAnd(number, 15), 7), equals(key, 7)) (removed) diff --git a/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.sql b/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.sql new file mode 100644 index 000000000000..1301135b4cb3 --- /dev/null +++ b/tests/queries/0_stateless/01655_plan_optimizations_merge_filters.sql @@ -0,0 +1,5 @@ +set allow_experimental_analyzer=1; +select explain from (explain actions = 1 select * from (select sum(number) as v, bitAnd(number, 15) as key from numbers(1e8) group by key having v != 0) where key = 7) where explain like '%Filter%' or explain like '%Aggregating%'; + +set allow_experimental_analyzer=0; +select explain from (explain actions = 1 select * from (select sum(number) as v, bitAnd(number, 15) as key from numbers(1e8) group by key having v != 0) where key = 7) where explain like '%Filter%' or explain like '%Aggregating%'; From 7d0e35bf5fb8ba0a03f34773333877b01462f302 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Jun 2024 16:53:50 +0000 Subject: [PATCH 099/254] Fix some tests. --- .../Optimizations/mergeExpressions.cpp | 2 +- .../01655_plan_optimizations.reference | 1 - .../02496_remove_redundant_sorting.reference | 13 ++++--- ...rouping_sets_predicate_push_down.reference | 36 ++++++++----------- 4 files changed, 21 insertions(+), 31 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp index 7cd538e78b3f..6ace1b3b5ce4 100644 --- a/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp +++ b/src/Processors/QueryPlan/Optimizations/mergeExpressions.cpp @@ -81,7 +81,7 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &) const auto & child_actions = child_filter->getExpression(); const auto & parent_actions = parent_filter->getExpression(); - if (child_actions->hasArrayJoin() && parent_actions->hasStatefulFunctions()) + if (child_actions->hasArrayJoin()) return 0; auto actions = child_actions->clone(); diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 1b9755a74d5d..f43c83c2c954 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -163,7 +163,6 @@ Filter column: notEquals(__table1.y, 2_UInt8) > filter is pushed down before CreatingSets CreatingSets Filter -Filter 1 3 > one condition of filter is pushed down before LEFT JOIN diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference index 77ef213b36d0..4a4e898c5bdc 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference @@ -332,13 +332,12 @@ SETTINGS optimize_aggregators_of_group_by_keys=0 -- avoid removing any() as it d Expression (Projection) Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) - Filter ((WHERE + (Projection + Before ORDER BY))) - Filter (HAVING) - Aggregating - Expression ((Before GROUP BY + Projection)) - Sorting (Sorting for ORDER BY) - Expression ((Before ORDER BY + (Projection + Before ORDER BY))) - ReadFromSystemNumbers + Filter (((WHERE + (Projection + Before ORDER BY)) + HAVING)) + Aggregating + Expression ((Before GROUP BY + Projection)) + Sorting (Sorting for ORDER BY) + Expression ((Before ORDER BY + (Projection + Before ORDER BY))) + ReadFromSystemNumbers -- execute 1 2 diff --git a/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.reference b/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.reference index 9bb0c022752b..70bcd7f255b8 100644 --- a/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.reference +++ b/tests/queries/0_stateless/02554_fix_grouping_sets_predicate_push_down.reference @@ -29,20 +29,16 @@ WHERE type_1 = \'all\' ExpressionTransform × 2 (Filter) FilterTransform × 2 - (Filter) - FilterTransform × 2 - (Filter) - FilterTransform × 2 - (Aggregating) - ExpressionTransform × 2 - AggregatingTransform × 2 - Copy 1 → 2 - (Expression) - ExpressionTransform - (Expression) - ExpressionTransform - (ReadFromMergeTree) - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 + (Aggregating) + ExpressionTransform × 2 + AggregatingTransform × 2 + Copy 1 → 2 + (Expression) + ExpressionTransform + (Expression) + ExpressionTransform + (ReadFromMergeTree) + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 (Expression) ExpressionTransform × 2 (Filter) @@ -68,14 +64,10 @@ ExpressionTransform × 2 ExpressionTransform × 2 AggregatingTransform × 2 Copy 1 → 2 - (Filter) - FilterTransform - (Filter) - FilterTransform - (Expression) - ExpressionTransform - (ReadFromMergeTree) - MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 + (Expression) + ExpressionTransform + (ReadFromMergeTree) + MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1 (Expression) ExpressionTransform × 2 (Aggregating) From ac480084a5faf6fa029bff492c09989ac9573388 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 3 Jun 2024 19:40:41 +0000 Subject: [PATCH 100/254] try to add check in mv --- .../Transforms/buildPushingToViewsChain.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index ff1be9323f59..177d45650dde 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -367,13 +367,16 @@ std::optional generateViewChain( bool check_access = !materialized_view->hasInnerTable() && materialized_view->getInMemoryMetadataPtr()->sql_security_type; out = interpreter.buildChain(inner_table, inner_metadata_snapshot, insert_columns, thread_status_holder, view_counter_ms, check_access); - bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); - const auto & settings = insert_context->getSettingsRef(); + if (interpreter.shouldAddSquashingFroStorage(inner_table)) + { + bool table_prefers_large_blocks = inner_table->prefersLargeBlocks(); + const auto & settings = insert_context->getSettingsRef(); - out.addSource(std::make_shared( - out.getInputHeader(), - table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, - table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + out.addSource(std::make_shared( + out.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL)); + } auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); counting->setProcessListElement(insert_context->getProcessListElement()); From e6f99266f60a46ec5ae56e55ecff2d753ac49919 Mon Sep 17 00:00:00 2001 From: Lee sungju Date: Tue, 4 Jun 2024 14:59:22 +0900 Subject: [PATCH 101/254] fix typo Fix typos in named collection samples --- docs/en/engines/table-engines/integrations/iceberg.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/iceberg.md b/docs/en/engines/table-engines/integrations/iceberg.md index 9d6395f73ac7..21fdbc0b1a59 100644 --- a/docs/en/engines/table-engines/integrations/iceberg.md +++ b/docs/en/engines/table-engines/integrations/iceberg.md @@ -37,7 +37,7 @@ Using named collections: http://test.s3.amazonaws.com/clickhouse-bucket/ - test + test test From 6e9343a0c754ef7a6511f81d40c393ef732f23a5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 10:34:28 +0000 Subject: [PATCH 102/254] Check what would be if disable prewhere for storage merge for old analyzer. --- src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index 8c5839a98036..b0ba4c3d40c4 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -45,6 +46,9 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) if (!source_step_with_filter) return; + if (!source_step_with_filter->getContext()->getSettingsRef().allow_experimental_analyzer && typeid_cast(source_step_with_filter))\ + return; + const auto & storage_snapshot = source_step_with_filter->getStorageSnapshot(); const auto & storage = storage_snapshot->storage; if (!storage.canMoveConditionsToPrewhere()) From aa4beb0be97ae87aea960fc642359ff20dac575a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 13:27:29 +0000 Subject: [PATCH 103/254] Fixing test. --- .../0_stateless/02156_storage_merge_prewhere.reference | 7 +++++++ tests/queries/0_stateless/02156_storage_merge_prewhere.sql | 3 ++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference index 86a36a9392c1..9bd6ca234dac 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -7,6 +7,13 @@ Prewhere info Prewhere filter Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Filter column: and(equals(k, 3), notEmpty(v)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) + Prewhere info + Prewhere filter + Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) 2 Filter column: and(equals(k, 3), notEmpty(v)) (removed) 2 diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql index ca61a8f2d571..4f010ebadfd4 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.sql +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.sql @@ -24,7 +24,8 @@ INSERT INTO t_02156_mt1 SELECT number, toString(number) FROM numbers(10000); INSERT INTO t_02156_mt2 SELECT number, toString(number) FROM numbers(10000); INSERT INTO t_02156_log SELECT number, toString(number) FROM numbers(10000); -SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count() FROM t_02156_merge1 WHERE k = 3 AND notEmpty(v)) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count() FROM t_02156_merge1 WHERE k = 3 AND notEmpty(v)) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' settings allow_experimental_analyzer=1; +SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count() FROM t_02156_merge1 WHERE k = 3 AND notEmpty(v)) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%' settings allow_experimental_analyzer=0; SELECT count() FROM t_02156_merge1 WHERE k = 3 AND notEmpty(v); SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count() FROM t_02156_merge2 WHERE k = 3 AND notEmpty(v)) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%'; From 73ba23cae7a920a30c444c71e75ed099e2a0ea9b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Jun 2024 15:55:10 +0200 Subject: [PATCH 104/254] Update optimizePrewhere.cpp --- src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp index b0ba4c3d40c4..0bf49b426e2a 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrewhere.cpp @@ -46,7 +46,8 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &) if (!source_step_with_filter) return; - if (!source_step_with_filter->getContext()->getSettingsRef().allow_experimental_analyzer && typeid_cast(source_step_with_filter))\ + /// Prewhere is broken with StorageMerge and old analyzer. + if (!source_step_with_filter->getContext()->getSettingsRef().allow_experimental_analyzer && typeid_cast(source_step_with_filter)) return; const auto & storage_snapshot = source_step_with_filter->getStorageSnapshot(); From 5b7162df6ee2c3e92c03f02b226b682debee8425 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 5 Jun 2024 04:37:59 +0200 Subject: [PATCH 105/254] Fix trash --- src/Parsers/Kusto/ParserKQLStatement.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Parsers/Kusto/ParserKQLStatement.cpp b/src/Parsers/Kusto/ParserKQLStatement.cpp index c04bf30c7aec..9c3f35ff3dd5 100644 --- a/src/Parsers/Kusto/ParserKQLStatement.cpp +++ b/src/Parsers/Kusto/ParserKQLStatement.cpp @@ -94,13 +94,17 @@ bool ParserKQLTableFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expe break; ++pos; } + if (!isValidKQLPos(pos)) + { + return false; + } --pos; kql_statement = String(pos_start->begin, pos->end); ++pos; } - Tokens token_kql(kql_statement.data(), kql_statement.data() + kql_statement.size()); - IParser::Pos pos_kql(token_kql, pos.max_depth, pos.max_backtracks); + Tokens tokens_kql(kql_statement.data(), kql_statement.data() + kql_statement.size(), 0, true); + IParser::Pos pos_kql(tokens_kql, pos.max_depth, pos.max_backtracks); Expected kql_expected; kql_expected.enable_highlighting = false; From 7089d0d84d90e36eeb2b037f3b4eb1b40fa4f56f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 5 Jun 2024 11:33:35 +0000 Subject: [PATCH 106/254] Fix fuzzed test. --- src/Storages/StorageMerge.cpp | 2 +- .../02156_storage_merge_prewhere_2.reference | 6 ++++++ .../0_stateless/02156_storage_merge_prewhere_2.sql | 13 +++++++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02156_storage_merge_prewhere_2.reference create mode 100644 tests/queries/0_stateless/02156_storage_merge_prewhere_2.sql diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 4c678a1228b6..f625b6a1820f 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1228,7 +1228,7 @@ QueryPlan ReadFromMerge::createPlanForTable( if (allow_experimental_analyzer) { - InterpreterSelectQueryAnalyzer interpreter(modified_query_info.query_tree, + InterpreterSelectQueryAnalyzer interpreter(modified_query_info.query_tree->toAST(), modified_context, SelectQueryOptions(processed_stage)); diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere_2.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere_2.reference new file mode 100644 index 000000000000..e2fa0e43e0c2 --- /dev/null +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere_2.reference @@ -0,0 +1,6 @@ +1 a +2 b +2 b +1 a +2 b +2 b diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere_2.sql b/tests/queries/0_stateless/02156_storage_merge_prewhere_2.sql new file mode 100644 index 000000000000..5c0751b7a358 --- /dev/null +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere_2.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS t_02156_ololo_1; +DROP TABLE IF EXISTS t_02156_ololo_2; +DROP TABLE IF EXISTS t_02156_ololo_dist; + +CREATE TABLE t_02156_ololo_1 (k UInt32, v Nullable(String)) ENGINE = MergeTree order by k; +CREATE TABLE t_02156_ololo_2 (k UInt32, v String) ENGINE = MergeTree order by k; +CREATE TABLE t_02156_ololo_dist (k UInt32, v String) ENGINE = Distributed(test_shard_localhost, currentDatabase(), t_02156_ololo_2); + +insert into t_02156_ololo_1 values (1, 'a'); +insert into t_02156_ololo_2 values (2, 'b'); + +select * from merge('t_02156_ololo') where k != 0 and notEmpty(v) order by k settings optimize_move_to_prewhere=0; +select * from merge('t_02156_ololo') where k != 0 and notEmpty(v) order by k settings optimize_move_to_prewhere=1; From 9920c3d17f9f64e24b46addd746c4bdcc49e6972 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 6 Jun 2024 21:19:28 +0000 Subject: [PATCH 107/254] Add uniform snowflakeID conversion functions --- docs/en/operations/settings/settings.md | 6 + .../sql-reference/functions/uuid-functions.md | 169 +++++++++++++- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + src/Functions/dateTimeToSnowflakeID.cpp | 181 +++++++++++++++ src/Functions/generateSnowflakeID.cpp | 2 +- src/Functions/snowflake.cpp | 99 +++++--- src/Functions/snowflakeIDToDateTime.cpp | 217 ++++++++++++++++++ .../0_stateless/00515_enhanced_time_zones.sql | 2 + .../0_stateless/01942_dateTimeToSnowflake.sql | 4 + .../01942_dateTimeToSnowflakeID.reference | 29 +++ .../01942_dateTimeToSnowflakeID.sql | 71 ++++++ .../01942_snowflakeIDToDateTime.reference | 27 +++ .../01942_snowflakeIDToDateTime.sql | 86 +++++++ .../0_stateless/01942_snowflakeToDateTime.sql | 5 + .../aspell-ignore/en/aspell-dict.txt | 3 + 16 files changed, 869 insertions(+), 34 deletions(-) create mode 100644 src/Functions/dateTimeToSnowflakeID.cpp create mode 100644 src/Functions/snowflakeIDToDateTime.cpp create mode 100644 tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference create mode 100644 tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql create mode 100644 tests/queries/0_stateless/01942_snowflakeIDToDateTime.reference create mode 100644 tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ffaf53085c43..a0ee2ef03992 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5398,6 +5398,12 @@ When set to `false` than all attempts are made with identical timeouts. Default value: `true`. +## uniform_snowflake_conversion_functions {#uniform_snowflake_conversion_functions} + +Controls if functions `snowflakeIDToDateTime`, `snowflakeIDToDateTime64`, `dateTimeToSnowflakeID`, and `dateTime64ToSnowflakeID` are enabled (if `true`), or functions `snowflakeToDateTime`, `snowflakeToDateTime64`, `dateTimeToSnowflake`, and `dateTime64ToSnowflake` (if `false`). + +Default value: `true` + ## allow_experimental_variant_type {#allow_experimental_variant_type} Allows creation of experimental [Variant](../../sql-reference/data-types/variant.md). diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index 0323ae728a96..24557db4ee96 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -543,12 +543,17 @@ serverUUID() Generates a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID). -The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. +The generated Snowflake ID contains the current Unix timestamp in milliseconds (41 + 1 top zero bits), followed by a machine id (10 bits), and a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function `generateSnowflakeID` guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries. +:::note +The generated Snowflake IDs are based on the UNIX epoch 1970-01-01. +While no standard or recommendation exists for the epoch of Snowflake IDs, implementations in other systems may use a different epoch, e.g. Twitter/X (2010-11-04) or Mastodon (2015-01-01). +::: + ``` 0 1 2 3 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 @@ -605,6 +610,11 @@ SELECT generateSnowflakeID(1), generateSnowflakeID(2); ## snowflakeToDateTime +:::warning +This function is deprecated and can only be used if setting [uniform_snowflake_conversion_functions](../../operations/settings/settings.md#uniform_snowflake_conversion_functions) is disabled. +The function will be removed at some point in future. +::: + Extracts the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) in [DateTime](../data-types/datetime.md) format. **Syntax** @@ -641,6 +651,11 @@ Result: ## snowflakeToDateTime64 +:::warning +This function is deprecated and can only be used if setting [uniform_snowflake_conversion_functions](../../operations/settings/settings.md#uniform_snowflake_conversion_functions) is disabled. +The function will be removed at some point in future. +::: + Extracts the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) in [DateTime64](../data-types/datetime64.md) format. **Syntax** @@ -677,6 +692,11 @@ Result: ## dateTimeToSnowflake +:::warning +This function is deprecated and can only be used if setting [uniform_snowflake_conversion_functions](../../operations/settings/settings.md#uniform_snowflake_conversion_functions) is disabled. +The function will be removed at some point in future. +::: + Converts a [DateTime](../data-types/datetime.md) value to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. **Syntax** @@ -711,6 +731,11 @@ Result: ## dateTime64ToSnowflake +:::warning +This function is deprecated and can only be used if setting [uniform_snowflake_conversion_functions](../../operations/settings/settings.md#uniform_snowflake_conversion_functions) is disabled. +The function will be removed at some point in future. +::: + Convert a [DateTime64](../data-types/datetime64.md) to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. **Syntax** @@ -743,6 +768,148 @@ Result: └─────────────────────────────┘ ``` +## snowflakeIDToDateTime + +Returns the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as a value of type [DateTime](../data-types/datetime.md). + +**Syntax** + +``` sql +snowflakeIDToDateTime(value[, epoch[, time_zone]]) +``` + +**Arguments** + +- `value` — Snowflake ID. [UInt64](../data-types/int-uint.md). +- `epoch` - Epoch of the Snowflake ID in milliseconds since 1970-01-01. Defaults to 0 (1970-01-01). For the Twitter/X epoch (2015-01-01), provide 1288834974657. Optional. [UInt*](../data-types/int-uint.md). +- `time_zone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../data-types/string.md). + +**Returned value** + +- The timestamp component of `value` as a [DateTime](../data-types/datetime.md) value. + +**Example** + +Query: + +```sql +SELECT snowflakeIDToDateTime(7204436857747984384) AS res +``` + +Result: + +``` +┌─────────────────res─┐ +│ 2024-06-06 10:59:58 │ +└─────────────────────┘ +``` + +## snowflakeIDToDateTime64 + +Returns the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as a value of type [DateTime64](../data-types/datetime64.md). + +**Syntax** + +``` sql +snowflakeIDToDateTime64(value[, epoch[, time_zone]]) +``` + +**Arguments** + +- `value` — Snowflake ID. [UInt64](../data-types/int-uint.md). +- `epoch` - Epoch of the Snowflake ID in milliseconds since 1970-01-01. Defaults to 0 (1970-01-01). For the Twitter/X epoch (2015-01-01), provide 1288834974657. Optional. [UInt*](../data-types/int-uint.md). +- `time_zone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../data-types/string.md). + +**Returned value** + +- The timestamp component of `value` as a [DateTime64](../data-types/datetime64.md) with scale = 3, i.e. millisecond precision. + +**Example** + +Query: + +```sql +SELECT snowflakeIDToDateTime64(7204436857747984384) AS res +``` + +Result: + +``` +┌─────────────────res─┐ +│ 2024-06-06 10:59:58 │ +└─────────────────────┘ +``` + +## dateTimeToSnowflakeID + +Converts a [DateTime](../data-types/datetime.md) value to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. + +**Syntax** + +``` sql +dateTimeToSnowflakeID(value[, epoch]) +``` + +**Arguments** + +- `value` — Date with time. [DateTime](../data-types/datetime.md). +- `epoch` - Epoch of the Snowflake ID in milliseconds since 1970-01-01. Defaults to 0 (1970-01-01). For the Twitter/X epoch (2015-01-01), provide 1288834974657. Optional. [UInt*](../data-types/int-uint.md). + +**Returned value** + +- Input value converted to [UInt64](../data-types/int-uint.md) as the first Snowflake ID at that time. + +**Example** + +Query: + +```sql +SELECT toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt, dateTimeToSnowflakeID(dt) AS res; +``` + +Result: + +``` +┌──────────────────dt─┬─────────────────res─┐ +│ 2021-08-15 18:57:56 │ 6832626392367104000 │ +└─────────────────────┴─────────────────────┘ +``` + +## dateTime64ToSnowflakeID + +Convert a [DateTime64](../data-types/datetime64.md) to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. + +**Syntax** + +``` sql +dateTime64ToSnowflakeID(value[, epoch]) +``` + +**Arguments** + +- `value` — Date with time. [DateTime64](../data-types/datetime64.md). +- `epoch` - Epoch of the Snowflake ID in milliseconds since 1970-01-01. Defaults to 0 (1970-01-01). For the Twitter/X epoch (2015-01-01), provide 1288834974657. Optional. [UInt*](../data-types/int-uint.md). + +**Returned value** + +- Input value converted to [UInt64](../data-types/int-uint.md) as the first Snowflake ID at that time. + +**Example** + +Query: + +```sql +SELECT toDateTime('2021-08-15 18:57:56.493', 3, 'Asia/Shanghai') AS dt, dateTime64ToSnowflakeID(dt) AS res; +``` + +Result: + +``` +┌──────────────────────dt─┬─────────────────res─┐ +│ 2021-08-15 18:57:56.493 │ 6832626394434895872 │ +└─────────────────────────┴─────────────────────┘ +``` + ## See also - [dictGetUUID](../functions/ext-dict-functions.md#ext_dict_functions-other) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 27ce54c03a71..2f85fb71e6f8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -928,6 +928,7 @@ class IColumn; M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \ M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ M(Bool, allow_deprecated_error_prone_window_functions, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)", 0) \ + M(Bool, uniform_snowflake_conversion_functions, true, "Enable functions snowflakeIDToDateTime[64] and dateTime[64]ToSnowflakeID.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 814c08c5705b..cda036c22ea8 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -97,6 +97,7 @@ static const std::map +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_FUNCTION; +} + +namespace +{ + +/// See generateSnowflakeID.cpp +constexpr int time_shift = 22; + +} + +class FunctionDateTimeToSnowflakeID : public IFunction +{ +private: + const bool uniform_snowflake_conversion_functions; + +public: + static constexpr auto name = "dateTimeToSnowflakeID"; + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + explicit FunctionDateTimeToSnowflakeID(ContextPtr context) + : uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions) + {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {"value", static_cast(&isDateTime), nullptr, "DateTime"} + }; + FunctionArgumentDescriptors optional_args{ + {"epoch", static_cast(&isNativeUInt), isColumnConst, "UInt*"} + }; + validateFunctionArgumentTypes(*this, arguments, args, optional_args); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + if (!uniform_snowflake_conversion_functions) + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "To use function {}, setting 'uniform_snowflake_conversion_functions' must be enabled", getName()); + + const auto & col_src = *arguments[0].column; + + size_t epoch = 0; + if (arguments.size() == 2 && input_rows_count != 0) + { + const auto & col_epoch = *arguments[1].column; + epoch = col_epoch.getUInt(0); + } + + auto col_res = ColumnUInt64::create(input_rows_count); + auto & res_data = col_res->getData(); + + const auto & src_data = typeid_cast(col_src).getData(); + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = (static_cast(src_data[i]) * 1000 - epoch) << time_shift; + return col_res; + } +}; + + +class FunctionDateTime64ToSnowflakeID : public IFunction +{ +private: + const bool uniform_snowflake_conversion_functions; + +public: + static constexpr auto name = "dateTime64ToSnowflakeID"; + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + explicit FunctionDateTime64ToSnowflakeID(ContextPtr context) + : uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions) + {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {"value", static_cast(&isDateTime64), nullptr, "DateTime64"} + }; + FunctionArgumentDescriptors optional_args{ + {"epoch", static_cast(&isNativeUInt), isColumnConst, "UInt*"} + }; + validateFunctionArgumentTypes(*this, arguments, args, optional_args); + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + if (!uniform_snowflake_conversion_functions) + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "To use function {}, setting 'uniform_snowflake_conversion_functions' must be enabled", getName()); + + const auto & col_src = *arguments[0].column; + const auto & src_data = typeid_cast(col_src).getData(); + + size_t epoch = 0; + if (arguments.size() == 2 && input_rows_count != 0) + { + const auto & col_epoch = *arguments[1].column; + epoch = col_epoch.getUInt(0); + } + + auto col_res = ColumnUInt64::create(input_rows_count); + auto & res_data = col_res->getData(); + + /// timestamps in snowflake-ids are millisecond-based, convert input to milliseconds + UInt32 src_scale = getDecimalScale(*arguments[0].type); + Int64 multiplier_msec = DecimalUtils::scaleMultiplier(3); + Int64 multiplier_src = DecimalUtils::scaleMultiplier(src_scale); + auto factor = multiplier_msec / static_cast(multiplier_src); + + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = static_cast(src_data[i] * factor - epoch) << time_shift; + + return col_res; + } +}; + +REGISTER_FUNCTION(DateTimeToSnowflakeID) +{ + { + FunctionDocumentation::Description description = R"(Converts a [DateTime](../data-types/datetime.md) value to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time.)"; + FunctionDocumentation::Syntax syntax = "dateTimeToSnowflakeID(value[, epoch])"; + FunctionDocumentation::Arguments arguments = { + {"value", "Date with time. [DateTime](../data-types/datetime.md)."}, + {"epoch", "Epoch of the Snowflake ID in milliseconds since 1970-01-01. Defaults to 0 (1970-01-01). For the Twitter/X epoch (2015-01-01), provide 1288834974657. Optional. [UInt*](../data-types/int-uint.md)"} + }; + FunctionDocumentation::ReturnedValue returned_value = "Input value converted to [UInt64](../data-types/int-uint.md) as the first Snowflake ID at that time."; + FunctionDocumentation::Examples examples = {{"simple", "SELECT dateTimeToSnowflakeID(toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai'))", "6832626392367104000"}}; + FunctionDocumentation::Categories categories = {"Snowflake ID"}; + + factory.registerFunction({description, syntax, arguments, returned_value, examples, categories}); + } + + { + FunctionDocumentation::Description description = R"(Converts a [DateTime64](../data-types/datetime64.md) value to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time.)"; + FunctionDocumentation::Syntax syntax = "dateTime64ToSnowflakeID(value[, epoch])"; + FunctionDocumentation::Arguments arguments = { + {"value", "Date with time. [DateTime64](../data-types/datetime.md)."}, + {"epoch", "Epoch of the Snowflake ID in milliseconds since 1970-01-01. Defaults to 0 (1970-01-01). For the Twitter/X epoch (2015-01-01), provide 1288834974657. Optional. [UInt*](../data-types/int-uint.md)"} + }; + FunctionDocumentation::ReturnedValue returned_value = "Input value converted to [UInt64](../data-types/int-uint.md) as the first Snowflake ID at that time."; + FunctionDocumentation::Examples examples = {{"simple", "SELECT dateTime64ToSnowflakeID(toDateTime64('2021-08-15 18:57:56', 3, 'Asia/Shanghai'))", "6832626394434895872"}}; + FunctionDocumentation::Categories categories = {"Snowflake ID"}; + + factory.registerFunction({description, syntax, arguments, returned_value, examples, categories}); + } +} + +} diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index f1e47ea11589..8ac010deafc6 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -207,7 +207,7 @@ class FunctionGenerateSnowflakeID : public IFunction REGISTER_FUNCTION(GenerateSnowflakeID) { - FunctionDocumentation::Description description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function generateSnowflakeID guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)"; + FunctionDocumentation::Description description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds (41 + 1 top zero bits), followed by a machine id (10 bits), and a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function generateSnowflakeID guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)"; FunctionDocumentation::Syntax syntax = "generateSnowflakeID([expression])"; FunctionDocumentation::Arguments arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}}; FunctionDocumentation::ReturnedValue returned_value = "A value of type UInt64"; diff --git a/src/Functions/snowflake.cpp b/src/Functions/snowflake.cpp index 4a2d502a31a8..801727e9eb9a 100644 --- a/src/Functions/snowflake.cpp +++ b/src/Functions/snowflake.cpp @@ -11,11 +11,17 @@ #include +/// ------------------------------------------------------------------------------------------------------------------------------ +/// The functions in this file are deprecated and should be removed in favor of functions 'snowflakeIDToDateTime[64]' and +/// 'dateTime[64]ToSnowflakeID' by summer 2025. Please also mark setting `uniform_snowflake_conversion_functions` as obsolete then. +/// ------------------------------------------------------------------------------------------------------------------------------ + namespace DB { namespace ErrorCodes { + extern const int DEPRECATED_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -34,10 +40,19 @@ constexpr int time_shift = 22; class FunctionDateTimeToSnowflake : public IFunction { private: - const char * name; + const bool uniform_snowflake_conversion_functions; public: - explicit FunctionDateTimeToSnowflake(const char * name_) : name(name_) { } + static constexpr auto name = "dateTimeToSnowflake"; + + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context); + } + + explicit FunctionDateTimeToSnowflake(ContextPtr context) + : uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions) + {} String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } @@ -56,6 +71,9 @@ class FunctionDateTimeToSnowflake : public IFunction ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { + if (uniform_snowflake_conversion_functions) + throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it disable setting 'uniform_snowflake_conversion_functions'", getName()); + const auto & src = arguments[0]; const auto & src_column = *src.column; @@ -73,13 +91,20 @@ class FunctionDateTimeToSnowflake : public IFunction class FunctionSnowflakeToDateTime : public IFunction { private: - const char * name; const bool allow_nonconst_timezone_arguments; + const bool uniform_snowflake_conversion_functions; public: - explicit FunctionSnowflakeToDateTime(const char * name_, ContextPtr context) - : name(name_) - , allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments) + static constexpr auto name = "snowflakeToDateTime"; + + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context); + } + + explicit FunctionSnowflakeToDateTime(ContextPtr context) + : allow_nonconst_timezone_arguments(context->getSettingsRef().allow_nonconst_timezone_arguments) + , uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions) {} String getName() const override { return name; } @@ -107,6 +132,9 @@ class FunctionSnowflakeToDateTime : public IFunction ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { + if (uniform_snowflake_conversion_functions) + throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it disable setting 'uniform_snowflake_conversion_functions'", getName()); + const auto & src = arguments[0]; const auto & src_column = *src.column; @@ -138,10 +166,19 @@ class FunctionSnowflakeToDateTime : public IFunction class FunctionDateTime64ToSnowflake : public IFunction { private: - const char * name; + const bool uniform_snowflake_conversion_functions; public: - explicit FunctionDateTime64ToSnowflake(const char * name_) : name(name_) { } + static constexpr auto name = "dateTime64ToSnowflake"; + + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context); + } + + explicit FunctionDateTime64ToSnowflake(ContextPtr context) + : uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions) + {} String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } @@ -160,6 +197,9 @@ class FunctionDateTime64ToSnowflake : public IFunction ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { + if (uniform_snowflake_conversion_functions) + throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it disable setting 'uniform_snowflake_conversion_functions'", getName()); + const auto & src = arguments[0]; const auto & src_column = *src.column; @@ -185,13 +225,20 @@ class FunctionDateTime64ToSnowflake : public IFunction class FunctionSnowflakeToDateTime64 : public IFunction { private: - const char * name; const bool allow_nonconst_timezone_arguments; + const bool uniform_snowflake_conversion_functions; public: - explicit FunctionSnowflakeToDateTime64(const char * name_, ContextPtr context) - : name(name_) - , allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments) + static constexpr auto name = "snowflakeToDateTime64"; + + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context); + } + + explicit FunctionSnowflakeToDateTime64(ContextPtr context) + : allow_nonconst_timezone_arguments(context->getSettingsRef().allow_nonconst_timezone_arguments) + , uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions) {} String getName() const override { return name; } @@ -219,6 +266,9 @@ class FunctionSnowflakeToDateTime64 : public IFunction ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { + if (uniform_snowflake_conversion_functions) + throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it disable setting 'uniform_snowflake_conversion_functions'", getName()); + const auto & src = arguments[0]; const auto & src_column = *src.column; @@ -246,27 +296,12 @@ class FunctionSnowflakeToDateTime64 : public IFunction } -REGISTER_FUNCTION(DateTimeToSnowflake) -{ - factory.registerFunction("dateTimeToSnowflake", - [](ContextPtr){ return std::make_shared("dateTimeToSnowflake"); }); -} - -REGISTER_FUNCTION(DateTime64ToSnowflake) -{ - factory.registerFunction("dateTime64ToSnowflake", - [](ContextPtr){ return std::make_shared("dateTime64ToSnowflake"); }); -} - -REGISTER_FUNCTION(SnowflakeToDateTime) -{ - factory.registerFunction("snowflakeToDateTime", - [](ContextPtr context){ return std::make_shared("snowflakeToDateTime", context); }); -} -REGISTER_FUNCTION(SnowflakeToDateTime64) +REGISTER_FUNCTION(LegacySnowflakeConversion) { - factory.registerFunction("snowflakeToDateTime64", - [](ContextPtr context){ return std::make_shared("snowflakeToDateTime64", context); }); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/snowflakeIDToDateTime.cpp b/src/Functions/snowflakeIDToDateTime.cpp new file mode 100644 index 000000000000..abaf09b165ba --- /dev/null +++ b/src/Functions/snowflakeIDToDateTime.cpp @@ -0,0 +1,217 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int UNKNOWN_FUNCTION; +} + +namespace +{ + +/// See generateSnowflakeID.cpp +constexpr int time_shift = 22; + +} + +class FunctionSnowflakeIDToDateTime : public IFunction +{ +private: + const bool uniform_snowflake_conversion_functions; + const bool allow_nonconst_timezone_arguments; + +public: + static constexpr auto name = "snowflakeIDToDateTime"; + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + explicit FunctionSnowflakeIDToDateTime(ContextPtr context) + : uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions) + , allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments) + {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {"value", static_cast(&isUInt64), nullptr, "UInt64"} + }; + FunctionArgumentDescriptors optional_args{ + {"epoch", static_cast(&isNativeUInt), isColumnConst, "UInt*"}, + {"time_zone", static_cast(&isString), nullptr, "String"} + }; + validateFunctionArgumentTypes(*this, arguments, args, optional_args); + + String timezone; + if (arguments.size() == 3) + timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, allow_nonconst_timezone_arguments); + + return std::make_shared(timezone); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + if (!uniform_snowflake_conversion_functions) + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "To use function {}, setting 'uniform_snowflake_conversion_functions' must be enabled", getName()); + + const auto & col_src = *arguments[0].column; + + size_t epoch = 0; + if (arguments.size() >= 2 && input_rows_count != 0) + { + const auto & col_epoch = *arguments[1].column; + epoch = col_epoch.getUInt(0); + } + + auto col_res = ColumnDateTime::create(input_rows_count); + auto & res_data = col_res->getData(); + + if (const auto * col_src_non_const = typeid_cast(&col_src)) + { + const auto & src_data = col_src_non_const->getData(); + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = static_cast(((src_data[i] >> time_shift) + epoch) / 1000); + } + else if (const auto * col_src_const = typeid_cast(&col_src)) + { + UInt64 src_val = col_src_const->getValue(); + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = static_cast(((src_val >> time_shift) + epoch) / 1000); + } + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument for function {}", name); + + return col_res; + } +}; + + +class FunctionSnowflakeIDToDateTime64 : public IFunction +{ +private: + const bool uniform_snowflake_conversion_functions; + const bool allow_nonconst_timezone_arguments; + +public: + static constexpr auto name = "snowflakeIDToDateTime64"; + + static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } + explicit FunctionSnowflakeIDToDateTime64(ContextPtr context) + : uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions) + , allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments) + {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + FunctionArgumentDescriptors args{ + {"value", static_cast(&isUInt64), nullptr, "UInt64"} + }; + FunctionArgumentDescriptors optional_args{ + {"epoch", static_cast(&isNativeUInt), isColumnConst, "UInt*"}, + {"time_zone", static_cast(&isString), nullptr, "String"} + }; + validateFunctionArgumentTypes(*this, arguments, args, optional_args); + + String timezone; + if (arguments.size() == 3) + timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, allow_nonconst_timezone_arguments); + + return std::make_shared(3, timezone); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + if (!uniform_snowflake_conversion_functions) + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "To use function {}, setting 'uniform_snowflake_conversion_functions' must be enabled", getName()); + + const auto & col_src = *arguments[0].column; + + size_t epoch = 0; + if (arguments.size() >= 2 && input_rows_count != 0) + { + const auto & col_epoch = *arguments[1].column; + epoch = col_epoch.getUInt(0); + } + + auto col_res = ColumnDateTime64::create(input_rows_count, 3); + auto & res_data = col_res->getData(); + + if (const auto * col_src_non_const = typeid_cast(&col_src)) + { + const auto & src_data = col_src_non_const->getData(); + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = (src_data[i] >> time_shift) + epoch; + } + else if (const auto * col_src_const = typeid_cast(&col_src)) + { + UInt64 src_val = col_src_const->getValue(); + for (size_t i = 0; i < input_rows_count; ++i) + res_data[i] = (src_val >> time_shift) + epoch; + } + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal argument for function {}", name); + + return col_res; + + } +}; + +REGISTER_FUNCTION(SnowflakeIDToDateTime) +{ + { + FunctionDocumentation::Description description = R"(Returns the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as a value of type [DateTime](../data-types/datetime.md).)"; + FunctionDocumentation::Syntax syntax = "snowflakeIDToDateTime(value[, epoch[, time_zone]])"; + FunctionDocumentation::Arguments arguments = { + {"value", "Snowflake ID. [UInt64](../data-types/int-uint.md)"}, + {"epoch", "Epoch of the Snowflake ID in milliseconds since 1970-01-01. Defaults to 0 (1970-01-01). For the Twitter/X epoch (2015-01-01), provide 1288834974657. Optional. [UInt*](../data-types/int-uint.md)"}, + {"time_zone", "[Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../data-types/string.md)"} + }; + FunctionDocumentation::ReturnedValue returned_value = "The timestamp component of `value` as a [DateTime](../data-types/datetime.md) value."; + FunctionDocumentation::Examples examples = {{"simple", "SELECT snowflakeIDToDateTime(7204436857747984384)", "2024-06-06 10:59:58"}}; + FunctionDocumentation::Categories categories = {"Snowflake ID"}; + + factory.registerFunction({description, syntax, arguments, returned_value, examples, categories}); + } + + { + FunctionDocumentation::Description description = R"(Returns the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) as a value of type [DateTime64](../data-types/datetime64.md).)"; + FunctionDocumentation::Syntax syntax = "snowflakeIDToDateTime64(value[, epoch[, time_zone]])"; + FunctionDocumentation::Arguments arguments = { + {"value", "Snowflake ID. [UInt64](../data-types/int-uint.md)"}, + {"epoch", "Epoch of the Snowflake ID in milliseconds since 1970-01-01. Defaults to 0 (1970-01-01). For the Twitter/X epoch (2015-01-01), provide 1288834974657. Optional. [UInt*](../data-types/int-uint.md)"}, + {"time_zone", "[Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../data-types/string.md)"} + }; + FunctionDocumentation::ReturnedValue returned_value = "The timestamp component of `value` as a [DateTime64](../data-types/datetime64.md) with scale = 3, i.e. millisecond precision."; + FunctionDocumentation::Examples examples = {{"simple", "SELECT snowflakeIDToDateTime64(7204436857747984384)", "2024-06-06 10:59:58"}}; + FunctionDocumentation::Categories categories = {"Snowflake ID"}; + + factory.registerFunction({description, syntax, arguments, returned_value, examples, categories}); + } +} + +} diff --git a/tests/queries/0_stateless/00515_enhanced_time_zones.sql b/tests/queries/0_stateless/00515_enhanced_time_zones.sql index 837b0b4be203..e39b618b670c 100644 --- a/tests/queries/0_stateless/00515_enhanced_time_zones.sql +++ b/tests/queries/0_stateless/00515_enhanced_time_zones.sql @@ -1,3 +1,5 @@ +SET uniform_snowflake_conversion_functions = 0; + SELECT addMonths(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), 1, 'Asia/Kolkata'); SELECT addMonths(toDateTime('2017-11-05 10:37:47', 'Asia/Kolkata'), 1); SELECT addMonths(toTimeZone(toDateTime('2017-11-05 08:07:47', 'Asia/Istanbul'), 'Asia/Kolkata'), 1); diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql index 1090179bb67e..0386717c933e 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql @@ -1,3 +1,4 @@ +SET uniform_snowflake_conversion_functions = 0; -- Force-disable uniform snowflake conversion functions (in case this is randomized in CI) SET session_timezone = 'Africa/Juba'; -- Error cases @@ -10,6 +11,9 @@ SELECT dateTime64ToSnowflake('abc'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} SELECT dateTimeToSnowflake('abc', 123); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} SELECT dateTime64ToSnowflake('abc', 123); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT dateTimeToSnowflake(now()) SETTINGS uniform_snowflake_conversion_functions = 1; -- { serverError DEPRECATED_FUNCTION } +SELECT dateTime64ToSnowflake(now64()) SETTINGS uniform_snowflake_conversion_functions = 1; -- { serverError DEPRECATED_FUNCTION } + SELECT '-- const / non-const inputs'; WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference new file mode 100644 index 000000000000..ab4e67701233 --- /dev/null +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference @@ -0,0 +1,29 @@ +-- Negative tests +-- Return type +UInt64 +UInt64 +-- Standard and twitter epoch +Row 1: +────── +dt: 2021-08-15 18:57:56 +dt64: 2021-08-15 18:57:56.492 +dateTimeToSnowflakeID(dt): 6832747188322304000 +dateTime64ToSnowflakeID(dt64): 6832747190385901568 +dateTimeToSnowflakeID(dt, twitter_epoch): 1426981498778550272 +dateTime64ToSnowflakeID(dt64, twitter_epoch): 1426981500842147840 +-- Different DateTime64 scales +Row 1: +────── +dateTime64ToSnowflakeID(dt64_0): 6832747188322304000 +dateTime64ToSnowflakeID(dt64_1): 6832747190000025600 +dateTime64ToSnowflakeID(dt64_2): 6832747190377512960 +dateTime64ToSnowflakeID(dt64_3): 6832747190385901568 +dateTime64ToSnowflakeID(dt64_4): 6832747190385901568 +-- Idempotency +Row 1: +────── +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_0)), dt64_0): 1 +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_1)), dt64_1): 1 +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_2)), dt64_2): 1 +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_3)), dt64_3): 1 +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_4)), dt64_4): 0 diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql new file mode 100644 index 000000000000..d4ea1d7efd08 --- /dev/null +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql @@ -0,0 +1,71 @@ +SET session_timezone = 'UTC'; -- disable timezone randomization +SET allow_experimental_analyzer = 1; -- The old path formats the result with different whitespaces +SET uniform_snowflake_conversion_functions = 1; -- Force-enable uniform snowflake conversion functions (in case this is randomized in CI) + +SELECT '-- Negative tests'; +SELECT dateTimeToSnowflakeID(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT dateTime64ToSnowflakeID(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT dateTimeToSnowflakeID('invalid_dt'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT dateTime64ToSnowflakeID('invalid_dt'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT dateTimeToSnowflakeID(now(), 'invalid_epoch'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT dateTime64ToSnowflakeID(now64(), 'invalid_epoch'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT dateTimeToSnowflakeID(now(), 42, 'too_many_args'); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT dateTime64ToSnowflakeID(now64(), 42, 'too_many_args'); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} + +SELECT dateTimeToSnowflakeID(now()) SETTINGS uniform_snowflake_conversion_functions = 0; -- { serverError UNKNOWN_FUNCTION } +SELECT dateTime64ToSnowflakeID(now64()) SETTINGS uniform_snowflake_conversion_functions = 0; -- { serverError UNKNOWN_FUNCTION } + +SELECT '-- Return type'; +SELECT toTypeName(dateTimeToSnowflakeID(now())); +SELECT toTypeName(dateTime64ToSnowflakeID(now64())); + +SELECT '-- Standard and twitter epoch'; + +WITH + toDateTime('2021-08-15 18:57:56') AS dt, + toDateTime64('2021-08-15 18:57:56.492', 3) AS dt64, + 1288834974657 AS twitter_epoch +SELECT + dt, + dt64, + dateTimeToSnowflakeID(dt), + dateTime64ToSnowflakeID(dt64), + dateTimeToSnowflakeID(dt, twitter_epoch), + dateTime64ToSnowflakeID(dt64, twitter_epoch) +FORMAT + Vertical; + +SELECT '-- Different DateTime64 scales'; + +WITH + toDateTime64('2021-08-15 18:57:56.492', 0, 'UTC') AS dt64_0, + toDateTime64('2021-08-15 18:57:56.492', 1, 'UTC') AS dt64_1, + toDateTime64('2021-08-15 18:57:56.492', 2, 'UTC') AS dt64_2, + toDateTime64('2021-08-15 18:57:56.492', 3, 'UTC') AS dt64_3, + toDateTime64('2021-08-15 18:57:56.492', 4, 'UTC') AS dt64_4 +SELECT + dateTime64ToSnowflakeID(dt64_0), + dateTime64ToSnowflakeID(dt64_1), + dateTime64ToSnowflakeID(dt64_2), + dateTime64ToSnowflakeID(dt64_3), + dateTime64ToSnowflakeID(dt64_4) +Format + Vertical; + +SELECT '-- Idempotency'; + + -- DateTime64-to-SnowflakeID-to-DateTime64 is idempotent if the scale is <=3 (millisecond precision) +WITH + now64(0) AS dt64_0, + now64(1) AS dt64_1, + now64(2) AS dt64_2, + now64(3) AS dt64_3, + now64(4) AS dt64_4 +SELECT + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_0)) == dt64_0, + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_1)) == dt64_1, + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_2)) == dt64_2, + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_3)) == dt64_3, + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_4)) == dt64_4 +FORMAT + Vertical; diff --git a/tests/queries/0_stateless/01942_snowflakeIDToDateTime.reference b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.reference new file mode 100644 index 000000000000..9ed8c1dd3e5b --- /dev/null +++ b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.reference @@ -0,0 +1,27 @@ +-- Negative tests +-- Return type +DateTime +DateTime64(3) +-- Non-const path +Row 1: +────── +sf: 7204436857747984384 +dt: 2024-06-06 10:59:58 +dt64: 2024-06-06 10:59:58.851 +Row 1: +────── +sf: 1426981498778550272 +dt: 2021-08-15 18:57:56 +dt64: 2021-08-15 18:57:56.000 +Row 1: +────── +sf: 7204436857747984384 +dt: 2024-06-06 18:59:58 +dt64: 2024-06-06 18:59:58.851 +-- Const path +Row 1: +────── +sf: 7204436857747984384 +dt: 2024-06-06 10:59:58 +dt64: 2024-06-06 10:59:58.851 +-- Can be combined with generateSnowflakeID diff --git a/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql new file mode 100644 index 000000000000..b0e244ef8149 --- /dev/null +++ b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql @@ -0,0 +1,86 @@ +SET session_timezone = 'UTC'; -- disable timezone randomization +SET allow_experimental_analyzer = 1; -- The old path formats the result with different whitespaces +SET uniform_snowflake_conversion_functions = 1; -- Force-enable uniform snowflake conversion functions (in case this is randomized in CI) + +SELECT '-- Negative tests'; +SELECT snowflakeIDToDateTime(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT snowflakeIDToDateTime64(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT snowflakeIDToDateTime('invalid_snowflake'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeIDToDateTime64('invalid_snowflake'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeIDToDateTime(123::UInt64, 'invalid_epoch'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeIDToDateTime64(123::UInt64, 'invalid_epoch'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeIDToDateTime(123::UInt64, materialize(42)); -- {serverError ILLEGAL_COLUMN} +SELECT snowflakeIDToDateTime64(123::UInt64, materialize(42)); -- {serverError ILLEGAL_COLUMN} +SELECT snowflakeIDToDateTime(123::UInt64, 42, 42); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeIDToDateTime64(123::UInt64, 42, 42); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeIDToDateTime(123::UInt64, 42, 'UTC', 'too_many_args'); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT snowflakeIDToDateTime64(123::UInt64, 42, 'UTC', 'too_many_args'); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} + +SELECT snowflakeIDToDateTime(123::UInt64) SETTINGS uniform_snowflake_conversion_functions = 0; -- { serverError UNKNOWN_FUNCTION } +SELECT snowflakeIDToDateTime64(123::UInt64) SETTINGS uniform_snowflake_conversion_functions = 0; -- { serverError UNKNOWN_FUNCTION } + +SELECT '-- Return type'; +SELECT toTypeName(snowflakeIDToDateTime(123::UInt64)); +SELECT toTypeName(snowflakeIDToDateTime64(123::UInt64)); + +SELECT '-- Non-const path'; +-- Two const arguments are mapped to two non-const arguments ('getDefaultImplementationForConstants'), the non-const path is taken + +WITH + 7204436857747984384 AS sf +SELECT + sf, + snowflakeIDToDateTime(sf) as dt, + snowflakeIDToDateTime64(sf) as dt64 +FORMAT + Vertical; + +-- With Twitter Snowflake ID and Twitter epoch +WITH + 1426981498778550272 AS sf, + 1288834974657 AS epoch +SELECT + sf, + snowflakeIDToDateTime(sf, epoch) as dt, + snowflakeIDToDateTime64(sf, epoch) as dt64 +FORMAT + Vertical; + +-- non-default timezone +WITH + 7204436857747984384 AS sf, + 0 AS epoch, -- default epoch + 'Asia/Shanghai' AS tz +SELECT + sf, + snowflakeIDToDateTime(sf, epoch, tz) as dt, + snowflakeIDToDateTime64(sf, epoch, tz) as dt64 +FORMAT + Vertical; + +SELECT '-- Const path'; + +-- The const path can only be tested by const snowflake + const epoch + non-const time-zone. The latter requires a special setting. +WITH + 7204436857747984384 AS sf, + 0 AS epoch, -- default epoch + materialize('Asia/Shanghai') AS tz +SELECT + sf, + snowflakeIDToDateTime(sf, epoch, tz) as dt, + snowflakeIDToDateTime64(sf, epoch, tz) as dt64 +FORMAT + Vertical +SETTINGS + allow_nonconst_timezone_arguments = 1; + + +SELECT '-- Can be combined with generateSnowflakeID'; + +WITH + generateSnowflakeID() AS snowflake +SELECT + snowflakeIDToDateTime(snowflake), + snowflakeIDToDateTime64(snowflake) +FORMAT + Null; diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql index f1a50dd370da..1729a50ae44f 100644 --- a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql @@ -1,3 +1,5 @@ +SET uniform_snowflake_conversion_functions = 0; -- Force-disable uniform snowflake conversion functions (in case this is randomized in CI) + -- -- Error cases SELECT snowflakeToDateTime(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} SELECT snowflakeToDateTime64(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} @@ -8,6 +10,9 @@ SELECT snowflakeToDateTime64('abc'); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} SELECT snowflakeToDateTime('abc', 123); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} SELECT snowflakeToDateTime64('abc', 123); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +SELECT snowflakeToDateTime(123::Int64) SETTINGS uniform_snowflake_conversion_functions = 1; -- { serverError DEPRECATED_FUNCTION } +SELECT snowflakeToDateTime64(123::Int64) SETTINGS uniform_snowflake_conversion_functions = 1; -- { serverError DEPRECATED_FUNCTION } + SELECT 'const column'; WITH CAST(1426860704886947840 AS Int64) AS i64, diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 49f43615c7e4..0025214762ed 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -960,6 +960,7 @@ ToGeoBoundary ToIPv ToParent ToSnowflake +ToSnowflakeID ToString ToUnicode Toolset @@ -1453,6 +1454,7 @@ datatypes dateName dateTime dateTimeToSnowflake +dateTimeToSnowflakeID datetime datetimes dayofyear @@ -2468,6 +2470,7 @@ skewpop skewsamp skippingerrors sleepEachRow +snowflakeIDToDateTime snowflakeToDateTime socketcache soundex From c02cb392bfebeff6cae375eede4990c5fff20fa4 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 7 Jun 2024 13:32:08 +0000 Subject: [PATCH 108/254] faster processing of scheduler queue activations --- src/Common/Scheduler/ISchedulerNode.h | 338 ++++++++++-------- .../Nodes/tests/gtest_event_queue.cpp | 143 ++++++++ .../tests/gtest_throttler_constraint.cpp | 2 - .../System/StorageSystemScheduler.cpp | 1 - 4 files changed, 341 insertions(+), 143 deletions(-) create mode 100644 src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index df8d86f379c0..37a85666f3a7 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -11,10 +11,10 @@ #include #include +#include #include #include -#include #include #include #include @@ -30,6 +30,7 @@ namespace ErrorCodes } class ISchedulerNode; +class EventQueue; inline const Poco::Util::AbstractConfiguration & emptyConfig() { @@ -82,6 +83,115 @@ struct SchedulerNodeInfo } }; + +/* + * Node of hierarchy for scheduling requests for resource. Base class for all + * kinds of scheduling elements (queues, policies, constraints and schedulers). + * + * Root node is a scheduler, which has it's thread to dequeue requests, + * execute requests (see ResourceRequest) and process events in a thread-safe manner. + * Immediate children of the scheduler represent independent resources. + * Each resource has it's own hierarchy to achieve required scheduling policies. + * Non-leaf nodes do not hold requests, but keep scheduling state + * (e.g. consumption history, amount of in-flight requests, etc). + * Leafs of hierarchy are queues capable of holding pending requests. + * + * scheduler (SchedulerRoot) + * / \ + * constraint constraint (SemaphoreConstraint) + * | | + * policy policy (PriorityPolicy) + * / \ / \ + * q1 q2 q3 q4 (FifoQueue) + * + * Dequeueing request from an inner node will dequeue request from one of active leaf-queues in its subtree. + * Node is considered to be active iff: + * - it has at least one pending request in one of leaves of it's subtree; + * - and enforced constraints, if any, are satisfied + * (e.g. amount of concurrent requests is not greater than some number). + * + * All methods must be called only from scheduler thread for thread-safety. + */ +class ISchedulerNode : public boost::intrusive::list_base_hook<>, private boost::noncopyable +{ +public: + explicit ISchedulerNode(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) + : event_queue(event_queue_) + , info(config, config_prefix) + {} + + virtual ~ISchedulerNode() = default; + + /// Checks if two nodes configuration is equal + virtual bool equals(ISchedulerNode * other) + { + return info.equals(other->info); + } + + /// Attach new child + virtual void attachChild(const std::shared_ptr & child) = 0; + + /// Detach and destroy child + virtual void removeChild(ISchedulerNode * child) = 0; + + /// Get attached child by name + virtual ISchedulerNode * getChild(const String & child_name) = 0; + + /// Activation of child due to the first pending request + /// Should be called on leaf node (i.e. queue) to propagate activation signal through chain to the root + virtual void activateChild(ISchedulerNode * child) = 0; + + /// Returns true iff node is active + virtual bool isActive() = 0; + + /// Returns number of active children + virtual size_t activeChildren() = 0; + + /// Returns the first request to be executed as the first component of resulting pair. + /// The second pair component is `true` iff node is still active after dequeueing. + virtual std::pair dequeueRequest() = 0; + + /// Returns full path string using names of every parent + String getPath() + { + String result; + ISchedulerNode * ptr = this; + while (ptr->parent) + { + result = "/" + ptr->basename + result; + ptr = ptr->parent; + } + return result.empty() ? "/" : result; + } + + /// Attach to a parent (used by attachChild) + virtual void setParent(ISchedulerNode * parent_) + { + parent = parent_; + } + +protected: + /// Notify parents about the first pending request or constraint becoming satisfied. + /// Postponed to be handled in scheduler thread, so it is intended to be called from outside. + void scheduleActivation(); + +public: + EventQueue * const event_queue; + String basename; + SchedulerNodeInfo info; + ISchedulerNode * parent = nullptr; + UInt64 activation_event_id = 0; // Valid for `ISchedulerNode` placed in EventQueue::activations + + /// Introspection + std::atomic dequeued_requests{0}; + std::atomic canceled_requests{0}; + std::atomic dequeued_cost{0}; + std::atomic canceled_cost{0}; + std::atomic busy_periods{0}; +}; + +using SchedulerNodePtr = std::shared_ptr; + /* * Simple waitable thread-safe FIFO task queue. * Intended to hold postponed events for later handling (usually by scheduler thread). @@ -89,57 +199,71 @@ struct SchedulerNodeInfo class EventQueue { public: - using Event = std::function; + using Task = std::function; + + using EventId = UInt64; + static constexpr EventId not_postponed = 0; + using TimePoint = std::chrono::system_clock::time_point; using Duration = std::chrono::system_clock::duration; - static constexpr UInt64 not_postponed = 0; + + struct Event + { + const EventId event_id; + std::function task; + + Event(EventId event_id_, std::function && task_) + : event_id(event_id_) + , task(std::move(task_)) + {} + }; struct Postponed { TimePoint key; - UInt64 id; // for canceling - std::unique_ptr event; + EventId event_id; // for canceling + std::unique_ptr task; - Postponed(TimePoint key_, UInt64 id_, Event && event_) + Postponed(TimePoint key_, EventId event_id_, Task && task_) : key(key_) - , id(id_) - , event(std::make_unique(std::move(event_))) + , event_id(event_id_) + , task(std::make_unique(std::move(task_))) {} bool operator<(const Postponed & rhs) const { - return std::tie(key, id) > std::tie(rhs.key, rhs.id); // reversed for min-heap + return std::tie(key, event_id) > std::tie(rhs.key, rhs.event_id); // reversed for min-heap } }; /// Add an `event` to be processed after `until` time point. - /// Returns a unique id for canceling. - [[nodiscard]] UInt64 postpone(TimePoint until, Event && event) + /// Returns a unique event id for canceling. + [[nodiscard]] EventId postpone(TimePoint until, Task && task) { std::unique_lock lock{mutex}; if (postponed.empty() || until < postponed.front().key) pending.notify_one(); - auto id = ++last_id; - postponed.emplace_back(until, id, std::move(event)); + auto event_id = ++last_event_id; + postponed.emplace_back(until, event_id, std::move(task)); std::push_heap(postponed.begin(), postponed.end()); - return id; + return event_id; } /// Cancel a postponed event using its unique id. /// NOTE: Only postponed events can be canceled. /// NOTE: If you need to cancel enqueued event, consider doing your actions inside another enqueued /// NOTE: event instead. This ensures that all previous events are processed. - bool cancelPostponed(UInt64 postponed_id) + bool cancelPostponed(EventId postponed_event_id) { - if (postponed_id == not_postponed) + if (postponed_event_id == not_postponed) return false; std::unique_lock lock{mutex}; for (auto i = postponed.begin(), e = postponed.end(); i != e; ++i) { - if (i->id == postponed_id) + if (i->event_id == postponed_event_id) { postponed.erase(i); - // It is O(n), but we do not expect either big heaps or frequent cancels. So it is fine. + // It is O(n), but we do not expect neither big heaps nor frequent cancels. So it is fine. std::make_heap(postponed.begin(), postponed.end()); return true; } @@ -148,11 +272,23 @@ class EventQueue } /// Add an `event` for immediate processing - void enqueue(Event && event) + void enqueue(Task && task) { std::unique_lock lock{mutex}; - bool was_empty = queue.empty(); - queue.emplace_back(event); + bool was_empty = events.empty() && activations.empty(); + auto event_id = ++last_event_id; + events.emplace_back(event_id, std::move(task)); + if (was_empty) + pending.notify_one(); + } + + /// Add an activation `event` for immediate processing. Activations use a separate queue for performance reasons. + void enqueueActivation(ISchedulerNode * node) + { + std::unique_lock lock{mutex}; + bool was_empty = events.empty() && activations.empty(); + node->activation_event_id = ++last_event_id; + activations.push_back(*node); if (was_empty) pending.notify_one(); } @@ -163,7 +299,7 @@ class EventQueue bool forceProcess() { std::unique_lock lock{mutex}; - if (!queue.empty()) + if (!events.empty() || !activations.empty()) { processQueue(std::move(lock)); return true; @@ -181,7 +317,7 @@ class EventQueue bool tryProcess() { std::unique_lock lock{mutex}; - if (!queue.empty()) + if (!events.empty() || !activations.empty()) { processQueue(std::move(lock)); return true; @@ -205,7 +341,7 @@ class EventQueue std::unique_lock lock{mutex}; while (true) { - if (!queue.empty()) + if (!events.empty() || !activations.empty()) { processQueue(std::move(lock)); return; @@ -269,141 +405,63 @@ class EventQueue void processQueue(std::unique_lock && lock) { - Event event = std::move(queue.front()); - queue.pop_front(); + if (events.empty()) + return processActivation(std::move(lock)); + if (activations.empty()) + return processEvent(std::move(lock)); + if (activations.front().activation_event_id < events.front().event_id) + return processActivation(std::move(lock)); + else + return processEvent(std::move(lock)); + } + + void processActivation(std::unique_lock && lock) + { + ISchedulerNode * node = &activations.front(); + activations.pop_front(); + node->activation_event_id = 0; lock.unlock(); // do not hold queue mutex while processing events - event(); + node->parent->activateChild(node); + } + + void processEvent(std::unique_lock && lock) + { + Task task = std::move(events.front().task); + events.pop_front(); + lock.unlock(); // do not hold queue mutex while processing events + task(); } void processPostponed(std::unique_lock && lock) { - Event event = std::move(*postponed.front().event); + Task task = std::move(*postponed.front().task); std::pop_heap(postponed.begin(), postponed.end()); postponed.pop_back(); lock.unlock(); // do not hold queue mutex while processing events - event(); + task(); } std::mutex mutex; std::condition_variable pending; - std::deque queue; + + // `events` and `activations` logically represent one ordered queue. To preserve the common order we use `EventId` + // Activations are stored in a separate queue for performance reasons (mostly to avoid any allocations) + std::deque events; + boost::intrusive::list activations; + std::vector postponed; - UInt64 last_id = 0; + EventId last_event_id = 0; std::atomic manual_time{TimePoint()}; // for tests only }; -/* - * Node of hierarchy for scheduling requests for resource. Base class for all - * kinds of scheduling elements (queues, policies, constraints and schedulers). - * - * Root node is a scheduler, which has it's thread to dequeue requests, - * execute requests (see ResourceRequest) and process events in a thread-safe manner. - * Immediate children of the scheduler represent independent resources. - * Each resource has it's own hierarchy to achieve required scheduling policies. - * Non-leaf nodes do not hold requests, but keep scheduling state - * (e.g. consumption history, amount of in-flight requests, etc). - * Leafs of hierarchy are queues capable of holding pending requests. - * - * scheduler (SchedulerRoot) - * / \ - * constraint constraint (SemaphoreConstraint) - * | | - * policy policy (PriorityPolicy) - * / \ / \ - * q1 q2 q3 q4 (FifoQueue) - * - * Dequeueing request from an inner node will dequeue request from one of active leaf-queues in its subtree. - * Node is considered to be active iff: - * - it has at least one pending request in one of leaves of it's subtree; - * - and enforced constraints, if any, are satisfied - * (e.g. amount of concurrent requests is not greater than some number). - * - * All methods must be called only from scheduler thread for thread-safety. - */ -class ISchedulerNode : private boost::noncopyable +inline void ISchedulerNode::scheduleActivation() { -public: - explicit ISchedulerNode(EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) - : event_queue(event_queue_) - , info(config, config_prefix) - {} - - virtual ~ISchedulerNode() = default; - - /// Checks if two nodes configuration is equal - virtual bool equals(ISchedulerNode * other) + if (likely(parent)) { - return info.equals(other->info); + // The same as `enqueue([this] { parent->activateChild(this); });` but faster + event_queue->enqueueActivation(this); } - - /// Attach new child - virtual void attachChild(const std::shared_ptr & child) = 0; - - /// Detach and destroy child - virtual void removeChild(ISchedulerNode * child) = 0; - - /// Get attached child by name - virtual ISchedulerNode * getChild(const String & child_name) = 0; - - /// Activation of child due to the first pending request - /// Should be called on leaf node (i.e. queue) to propagate activation signal through chain to the root - virtual void activateChild(ISchedulerNode * child) = 0; - - /// Returns true iff node is active - virtual bool isActive() = 0; - - /// Returns number of active children - virtual size_t activeChildren() = 0; - - /// Returns the first request to be executed as the first component of resulting pair. - /// The second pair component is `true` iff node is still active after dequeueing. - virtual std::pair dequeueRequest() = 0; - - /// Returns full path string using names of every parent - String getPath() - { - String result; - ISchedulerNode * ptr = this; - while (ptr->parent) - { - result = "/" + ptr->basename + result; - ptr = ptr->parent; - } - return result.empty() ? "/" : result; - } - - /// Attach to a parent (used by attachChild) - virtual void setParent(ISchedulerNode * parent_) - { - parent = parent_; - } - -protected: - /// Notify parents about the first pending request or constraint becoming satisfied. - /// Postponed to be handled in scheduler thread, so it is intended to be called from outside. - void scheduleActivation() - { - if (likely(parent)) - { - event_queue->enqueue([this] { parent->activateChild(this); }); - } - } - -public: - EventQueue * const event_queue; - String basename; - SchedulerNodeInfo info; - ISchedulerNode * parent = nullptr; - - /// Introspection - std::atomic dequeued_requests{0}; - std::atomic canceled_requests{0}; - std::atomic dequeued_cost{0}; - std::atomic canceled_cost{0}; - std::atomic busy_periods{0}; -}; - -using SchedulerNodePtr = std::shared_ptr; +} } diff --git a/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp b/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp new file mode 100644 index 000000000000..0e281607bc2e --- /dev/null +++ b/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp @@ -0,0 +1,143 @@ +#include +#include + +#include + +using namespace DB; + +class FakeSchedulerNode : public ISchedulerNode +{ +public: + explicit FakeSchedulerNode(String & log_, EventQueue * event_queue_, const Poco::Util::AbstractConfiguration & config = emptyConfig(), const String & config_prefix = {}) + : ISchedulerNode(event_queue_, config, config_prefix) + , log(log_) + {} + + void attachChild(const SchedulerNodePtr & child) override + { + log += " +" + child->basename; + } + + void removeChild(ISchedulerNode * child) override + { + log += " -" + child->basename; + } + + ISchedulerNode * getChild(const String & /* child_name */) override + { + return nullptr; + } + + void activateChild(ISchedulerNode * child) override + { + log += " A" + child->basename; + } + + bool isActive() override + { + return false; + } + + size_t activeChildren() override + { + return 0; + } + + std::pair dequeueRequest() override + { + log += " D"; + return {nullptr, false}; + } + +private: + String & log; +}; + +struct QueueTest { + String log; + EventQueue event_queue; + FakeSchedulerNode root_node; + + QueueTest() + : root_node(log, &event_queue) + {} + + SchedulerNodePtr makeNode(const String & name) + { + auto node = std::make_shared(log, &event_queue); + node->basename = name; + node->setParent(&root_node); + return std::static_pointer_cast(node); + } + + void process(EventQueue::TimePoint now, const String & expected_log, size_t limit = size_t(-1)) + { + event_queue.setManualTime(now); + for (;limit > 0; limit--) + { + if (!event_queue.tryProcess()) + break; + } + EXPECT_EQ(log, expected_log); + log.clear(); + } + + void activate(const SchedulerNodePtr & node) + { + event_queue.enqueueActivation(node.get()); + } + + void event(const String & text) + { + event_queue.enqueue([this, text] { log += " " + text; }); + } + + EventQueue::EventId postpone(EventQueue::TimePoint until, const String & text) + { + return event_queue.postpone(until, [this, text] { log += " " + text; }); + } + + void cancel(EventQueue::EventId event_id) + { + event_queue.cancelPostponed(event_id); + } +}; + +TEST(SchedulerEventQueue, Smoke) +{ + QueueTest t; + + using namespace std::chrono_literals; + + EventQueue::TimePoint start = std::chrono::system_clock::now(); + t.process(start, "", 0); + + // Activations + auto node1 = t.makeNode("1"); + auto node2 = t.makeNode("2"); + t.activate(node2); + t.activate(node1); + t.process(start + 42s, " A2 A1"); + + // Events + t.event("E1"); + t.event("E2"); + t.process(start + 100s, " E1 E2"); + + // Postponed events + t.postpone(start + 200s, "P200"); + auto p190 = t.postpone(start + 200s, "P190"); + t.postpone(start + 150s, "P150"); + t.postpone(start + 175s, "P175"); + t.process(start + 180s, " P150 P175"); + t.event("E3"); + t.cancel(p190); + t.process(start + 300s, " E3 P200"); + + // Ordering of events and activations + t.event("E1"); + t.activate(node1); + t.event("E2"); + t.activate(node2); + t.process(start + 300s, " E1 A1 E2 A2"); +} diff --git a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp index 9703227ccfc4..6cfccb252fa6 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_throttler_constraint.cpp @@ -5,8 +5,6 @@ #include #include -#include "Common/Scheduler/ISchedulerNode.h" -#include "Common/Scheduler/ResourceRequest.h" using namespace DB; diff --git a/src/Storages/System/StorageSystemScheduler.cpp b/src/Storages/System/StorageSystemScheduler.cpp index 651ca815420a..339a59e88a5d 100644 --- a/src/Storages/System/StorageSystemScheduler.cpp +++ b/src/Storages/System/StorageSystemScheduler.cpp @@ -12,7 +12,6 @@ #include #include #include -#include "Common/Scheduler/ResourceRequest.h" namespace DB From a12fec7f41203049ea6a454d6ac8832499ef7958 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 7 Jun 2024 14:22:23 +0000 Subject: [PATCH 109/254] Try to fix flaky test --- .../0_stateless/01942_dateTimeToSnowflakeID.reference | 10 +++++----- .../0_stateless/01942_dateTimeToSnowflakeID.sql | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference index ab4e67701233..aaf5a2e35436 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference @@ -22,8 +22,8 @@ dateTime64ToSnowflakeID(dt64_4): 6832747190385901568 -- Idempotency Row 1: ────── -equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_0)), dt64_0): 1 -equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_1)), dt64_1): 1 -equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_2)), dt64_2): 1 -equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_3)), dt64_3): 1 -equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_4)), dt64_4): 0 +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_0), 0, 'UTC'), dt64_0): 1 +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_1), 0, 'UTC'), dt64_1): 1 +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_2), 0, 'UTC'), dt64_2): 1 +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_3), 0, 'UTC'), dt64_3): 1 +equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_4), 0, 'UTC'), dt64_4): 0 diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql index d4ea1d7efd08..ae8f73766971 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql @@ -62,10 +62,10 @@ WITH now64(3) AS dt64_3, now64(4) AS dt64_4 SELECT - snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_0)) == dt64_0, - snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_1)) == dt64_1, - snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_2)) == dt64_2, - snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_3)) == dt64_3, - snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_4)) == dt64_4 + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_0), 0, 'UTC') == dt64_0, + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_1), 0, 'UTC') == dt64_1, + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_2), 0, 'UTC') == dt64_2, + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_3), 0, 'UTC') == dt64_3, + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_4), 0, 'UTC') == dt64_4 FORMAT Vertical; From 9a7f5d1e2071255451224e16ea80eb69e8ee4658 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 7 Jun 2024 20:17:48 +0000 Subject: [PATCH 110/254] Try to fix flaky test, pt. II --- .../01942_dateTimeToSnowflakeID.reference | 5 ++++- .../0_stateless/01942_dateTimeToSnowflakeID.sql | 15 +++++++++++---- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference index aaf5a2e35436..5dcd0c9dfcdc 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.reference @@ -26,4 +26,7 @@ equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_0), 0, 'UTC'), dt64_ equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_1), 0, 'UTC'), dt64_1): 1 equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_2), 0, 'UTC'), dt64_2): 1 equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_3), 0, 'UTC'), dt64_3): 1 -equals(snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_4), 0, 'UTC'), dt64_4): 0 +Row 1: +────── +dt64_4: 2023-11-11 11:11:11.1231 +snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_4)): 2023-11-11 11:11:11.123 diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql index ae8f73766971..33bac8aaa35f 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql @@ -59,13 +59,20 @@ WITH now64(0) AS dt64_0, now64(1) AS dt64_1, now64(2) AS dt64_2, - now64(3) AS dt64_3, - now64(4) AS dt64_4 + now64(3) AS dt64_3 SELECT snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_0), 0, 'UTC') == dt64_0, snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_1), 0, 'UTC') == dt64_1, snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_2), 0, 'UTC') == dt64_2, - snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_3), 0, 'UTC') == dt64_3, - snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_4), 0, 'UTC') == dt64_4 + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_3), 0, 'UTC') == dt64_3 +FORMAT + Vertical; + +-- not idempotent +WITH + toDateTime64('2023-11-11 11:11:11.1231', 4, 'UTC') AS dt64_4 +SELECT + dt64_4, + snowflakeIDToDateTime64(dateTime64ToSnowflakeID(dt64_4)) FORMAT Vertical; From ab79addee492172440724df16de1c6a417ccd04f Mon Sep 17 00:00:00 2001 From: serxa Date: Sun, 9 Jun 2024 12:55:31 +0000 Subject: [PATCH 111/254] use 1MB HTTP buffers to avoid frequnet send syscalls --- base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h b/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h index c87719b63a49..3178306363c8 100644 --- a/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h +++ b/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h @@ -26,7 +26,7 @@ namespace Poco { namespace Net { - constexpr size_t HTTP_DEFAULT_BUFFER_SIZE = 8 * 1024; + constexpr size_t HTTP_DEFAULT_BUFFER_SIZE = 1024 * 1024; typedef Poco::BasicBufferedStreamBuf> HTTPBasicStreamBuf; From b8fff4bd7e10eeb6b74d3e31e0104054f8c75869 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Sun, 9 Jun 2024 13:27:25 +0000 Subject: [PATCH 112/254] bug fixed --- src/Storages/MergeTree/KeyCondition.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 4f1f0d337c0e..443c39ec88bc 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -749,16 +749,17 @@ static NameSet getAllSubexpressionNames(const ExpressionActions & key_expr) void KeyCondition::getAllSpaceFillingCurves() { + /// So far the only supported function is mortonEncode and hilbertEncode (Morton and Hilbert curves). + for (const auto & action : key_expr->getActions()) { - auto space_filling_curve_type_iter = space_filling_curve_name_to_type.find(action.node->function_base->getName()); if (action.node->type == ActionsDAG::ActionType::FUNCTION && action.node->children.size() >= 2 - && space_filling_curve_type_iter != space_filling_curve_name_to_type.end()) + && space_filling_curve_name_to_type.count(action.node->function_base->getName()) > 0) { SpaceFillingCurveDescription curve; - curve.type = space_filling_curve_type_iter->second; curve.function_name = action.node->function_base->getName(); + curve.type = space_filling_curve_name_to_type.at(curve.function_name); curve.key_column_pos = key_columns.at(action.node->result_name); for (const auto & child : action.node->children) { From 7ecbce97e77f525bd3b4322ad0fd7f99cf91dd7b Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 10 Jun 2024 12:37:06 +0200 Subject: [PATCH 113/254] empty commit From 6fb4514293a83ea92575deee4ae0809f4c97ec8d Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 10 Jun 2024 14:27:45 +0000 Subject: [PATCH 114/254] Fix host_id in DatabaseReplicated when cluster_secure_connection is enabled --- src/Databases/DatabaseReplicated.cpp | 21 ++++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index b91596a555db..6f2ea51ef0f8 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -73,9 +73,10 @@ zkutil::ZooKeeperPtr DatabaseReplicated::getZooKeeper() const return getContext()->getZooKeeper(); } -static inline String getHostID(ContextPtr global_context, const UUID & db_uuid) +static inline String getHostID(ContextPtr global_context, const UUID & db_uuid, bool secure) { - return Cluster::Address::toString(getFQDNOrHostName(), global_context->getTCPPort()) + ':' + toString(db_uuid); + UInt16 port = secure ? global_context->getTCPPortSecure().value_or(DBMS_DEFAULT_SECURE_PORT) : global_context->getTCPPort(); + return Cluster::Address::toString(getFQDNOrHostName(), port) + ':' + toString(db_uuid); } static inline UInt64 getMetadataHash(const String & table_name, const String & metadata) @@ -369,8 +370,10 @@ void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(LoadingStrictnessL return; } - String host_id = getHostID(getContext(), db_uuid); - if (is_create_query || replica_host_id != host_id) + String host_id = getHostID(getContext(), db_uuid, cluster_auth_info.cluster_secure_connection); + String host_id_default = getHostID(getContext(), db_uuid, false); + + if (is_create_query || (replica_host_id != host_id && replica_host_id != host_id_default)) { throw Exception( ErrorCodes::REPLICA_ALREADY_EXISTS, @@ -378,6 +381,14 @@ void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(LoadingStrictnessL replica_name, shard_name, zookeeper_path, replica_host_id, host_id); } + /// Before 24.6 we always created host_id with unsecure port, even if cluster_auth_info.cluster_secure_connection was true. + /// So not to break compatibility, we need to update host_id to secure one if cluster_auth_info.cluster_secure_connection is true. + if (host_id != host_id_default && replica_host_id == host_id_default) + { + current_zookeeper->set(replica_path, host_id, -1); + createEmptyLogEntry(current_zookeeper); + } + /// Check that replica_group_name in ZooKeeper matches the local one and change it if necessary. String zk_replica_group_name; if (!current_zookeeper->tryGet(replica_path + "/replica_group", zk_replica_group_name)) @@ -504,7 +515,7 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt "already contains some data and it does not look like Replicated database path.", zookeeper_path); /// Write host name to replica_path, it will protect from multiple replicas with the same name - auto host_id = getHostID(getContext(), db_uuid); + auto host_id = getHostID(getContext(), db_uuid, cluster_auth_info.cluster_secure_connection); for (int attempts = 10; attempts > 0; --attempts) { From ec5fa482e40d5602134f4ead167b63efd08d1cc4 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 10 Jun 2024 15:26:36 +0000 Subject: [PATCH 115/254] Fix style --- src/Databases/DatabaseReplicated.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 6f2ea51ef0f8..98b3c9f70026 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -381,7 +381,7 @@ void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(LoadingStrictnessL replica_name, shard_name, zookeeper_path, replica_host_id, host_id); } - /// Before 24.6 we always created host_id with unsecure port, even if cluster_auth_info.cluster_secure_connection was true. + /// Before 24.6 we always created host_id with insecure port, even if cluster_auth_info.cluster_secure_connection was true. /// So not to break compatibility, we need to update host_id to secure one if cluster_auth_info.cluster_secure_connection is true. if (host_id != host_id_default && replica_host_id == host_id_default) { From 68e1d8701c973def035f0b9ea17f6ca3d224d73c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 10 Jun 2024 18:09:07 +0000 Subject: [PATCH 116/254] fixes due to review --- src/Interpreters/InterpreterInsertQuery.cpp | 10 +- src/Interpreters/Squashing.cpp | 119 +++++++++--------- src/Interpreters/Squashing.h | 38 ++---- .../Transforms/ApplySquashingTransform.h | 10 +- .../Transforms/PlanSquashingTransform.cpp | 62 +++------ .../Transforms/PlanSquashingTransform.h | 7 +- .../Transforms/SquashingTransform.cpp | 22 ++-- .../Transforms/SquashingTransform.h | 6 +- src/Server/TCPHandler.cpp | 17 ++- src/Storages/MergeTree/MutateTask.cpp | 16 ++- src/Storages/ProjectionsDescription.cpp | 2 +- 11 files changed, 128 insertions(+), 181 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index e632886778f1..d735fb8a55cc 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -632,7 +632,10 @@ BlockIO InterpreterInsertQuery::execute() pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { - return std::make_shared(in_header); + return std::make_shared( + in_header, + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); }); } @@ -685,7 +688,10 @@ BlockIO InterpreterInsertQuery::execute() { bool table_prefers_large_blocks = table->prefersLargeBlocks(); - auto squashing = std::make_shared(chain.getInputHeader()); + auto squashing = std::make_shared( + chain.getInputHeader(), + table_prefers_large_blocks ? settings.min_insert_block_size_rows : settings.max_block_size, + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL); chain.addSource(std::move(squashing)); diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 47add4954219..a05c5853ce34 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -10,77 +10,30 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -ApplySquashing::ApplySquashing(Block header_) +Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) : header(header_) + , min_block_size_rows(min_block_size_rows_) + , min_block_size_bytes(min_block_size_bytes_) +{ +} + +Chunk Squashing::flush() { + return convertToChunk(std::move(chunks_to_merge_vec)); } -Chunk ApplySquashing::add(Chunk && input_chunk) +Chunk Squashing::squash(Chunk && input_chunk) { if (!input_chunk.hasChunkInfo()) return Chunk(); const auto *info = getInfoFromChunk(input_chunk); - append(info->chunks); + squash(info->chunks); return std::move(accumulated_chunk); } -void ApplySquashing::append(std::vector & input_chunks) -{ - accumulated_chunk = {}; - std::vector mutable_columns = {}; - size_t rows = 0; - for (const Chunk & chunk : input_chunks) - rows += chunk.getNumRows(); - - { - auto & first_chunk = input_chunks[0]; - Columns columns = first_chunk.detachColumns(); - for (size_t i = 0; i < columns.size(); ++i) - { - mutable_columns.push_back(IColumn::mutate(std::move(columns[i]))); - mutable_columns[i]->reserve(rows); - } - } - - for (size_t i = 1; i < input_chunks.size(); ++i) // We've already processed the first chunk above - { - Columns columns = input_chunks[i].detachColumns(); - for (size_t j = 0, size = mutable_columns.size(); j < size; ++j) - { - const auto source_column = columns[j]; - - mutable_columns[j]->insertRangeFrom(*source_column, 0, source_column->size()); - } - } - accumulated_chunk.setColumns(std::move(mutable_columns), rows); -} - -const ChunksToSquash* ApplySquashing::getInfoFromChunk(const Chunk & chunk) -{ - const auto& info = chunk.getChunkInfo(); - const auto * agg_info = typeid_cast(info.get()); - - if (!agg_info) - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no ChunksToSquash in ChunkInfoPtr"); - - return agg_info; -} - -PlanSquashing::PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) - : min_block_size_rows(min_block_size_rows_) - , min_block_size_bytes(min_block_size_bytes_) - , header(header_) -{ -} - -Chunk PlanSquashing::flush() -{ - return convertToChunk(std::move(chunks_to_merge_vec)); -} - -Chunk PlanSquashing::add(Chunk && input_chunk) +Chunk Squashing::add(Chunk && input_chunk) { if (!input_chunk) return {}; @@ -131,7 +84,7 @@ Chunk PlanSquashing::add(Chunk && input_chunk) return {}; } -Chunk PlanSquashing::convertToChunk(std::vector && chunks) +Chunk Squashing::convertToChunk(std::vector && chunks) const { if (chunks.empty()) return {}; @@ -144,19 +97,61 @@ Chunk PlanSquashing::convertToChunk(std::vector && chunks) return Chunk(header.cloneEmptyColumns(), 0, info); } -void PlanSquashing::expandCurrentSize(size_t rows, size_t bytes) +void Squashing::squash(std::vector & input_chunks) +{ + accumulated_chunk = {}; + std::vector mutable_columns = {}; + size_t rows = 0; + for (const Chunk & chunk : input_chunks) + rows += chunk.getNumRows(); + + { + auto & first_chunk = input_chunks[0]; + Columns columns = first_chunk.detachColumns(); + for (size_t i = 0; i < columns.size(); ++i) + { + mutable_columns.push_back(IColumn::mutate(std::move(columns[i]))); + mutable_columns[i]->reserve(rows); + } + } + + for (size_t i = 1; i < input_chunks.size(); ++i) // We've already processed the first chunk above + { + Columns columns = input_chunks[i].detachColumns(); + for (size_t j = 0, size = mutable_columns.size(); j < size; ++j) + { + const auto source_column = columns[j]; + + mutable_columns[j]->insertRangeFrom(*source_column, 0, source_column->size()); + } + } + accumulated_chunk.setColumns(std::move(mutable_columns), rows); +} + +const ChunksToSquash* Squashing::getInfoFromChunk(const Chunk & chunk) +{ + const auto& info = chunk.getChunkInfo(); + const auto * agg_info = typeid_cast(info.get()); + + if (!agg_info) + throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no ChunksToSquash in ChunkInfoPtr"); + + return agg_info; +} + +void Squashing::expandCurrentSize(size_t rows, size_t bytes) { accumulated_size.rows += rows; accumulated_size.bytes += bytes; } -void PlanSquashing::changeCurrentSize(size_t rows, size_t bytes) +void Squashing::changeCurrentSize(size_t rows, size_t bytes) { accumulated_size.rows = rows; accumulated_size.bytes = bytes; } -bool PlanSquashing::isEnoughSize(size_t rows, size_t bytes) const +bool Squashing::isEnoughSize(size_t rows, size_t bytes) const { return (!min_block_size_rows && !min_block_size_bytes) || (min_block_size_rows && rows >= min_block_size_rows) diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index 77191e630500..760b7d7475f4 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -26,39 +26,23 @@ struct ChunksToSquash : public ChunkInfo * Order of data is kept. */ -class ApplySquashing +class Squashing { public: - explicit ApplySquashing(Block header_); - - Chunk add(Chunk && input_chunk); - - Block header; - -private: - Chunk accumulated_chunk; - - const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); - - void append(std::vector & input_chunks); - - bool isEnoughSize(const Block & block); - bool isEnoughSize(size_t rows, size_t bytes) const; -}; - -class PlanSquashing -{ -public: - explicit PlanSquashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); - PlanSquashing(PlanSquashing && other) = default; + explicit Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); + Squashing(Squashing && other) = default; Chunk add(Chunk && input_chunk); + Chunk squash(Chunk && input_chunk); Chunk flush(); + bool isDataLeft() { return !chunks_to_merge_vec.empty(); } + Block header; + private: struct CurrentSize { @@ -70,14 +54,18 @@ class PlanSquashing size_t min_block_size_rows; size_t min_block_size_bytes; - const Block header; CurrentSize accumulated_size; + Chunk accumulated_chunk; + + const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); + + void squash(std::vector & input_chunks); void expandCurrentSize(size_t rows, size_t bytes); void changeCurrentSize(size_t rows, size_t bytes); bool isEnoughSize(size_t rows, size_t bytes) const; - Chunk convertToChunk(std::vector && chunks); + Chunk convertToChunk(std::vector && chunks) const; }; } diff --git a/src/Processors/Transforms/ApplySquashingTransform.h b/src/Processors/Transforms/ApplySquashingTransform.h index e63691fcc6a0..7bf1f32340be 100644 --- a/src/Processors/Transforms/ApplySquashingTransform.h +++ b/src/Processors/Transforms/ApplySquashingTransform.h @@ -9,9 +9,9 @@ namespace DB class ApplySquashingTransform : public ExceptionKeepingTransform { public: - explicit ApplySquashingTransform(const Block & header) + explicit ApplySquashingTransform(const Block & header, const size_t min_block_size_rows, const size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(header) + , squashing(header, min_block_size_rows, min_block_size_bytes) { } @@ -37,7 +37,7 @@ class ApplySquashingTransform : public ExceptionKeepingTransform protected: void onConsume(Chunk chunk) override { - if (auto res_chunk = squashing.add(std::move(chunk))) + if (auto res_chunk = squashing.squash(std::move(chunk))) cur_chunk.setColumns(res_chunk.getColumns(), res_chunk.getNumRows()); } @@ -50,12 +50,12 @@ class ApplySquashingTransform : public ExceptionKeepingTransform } void onFinish() override { - auto chunk = squashing.add({}); + auto chunk = squashing.squash({}); finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); } private: - ApplySquashing squashing; + Squashing squashing; Chunk cur_chunk; Chunk finish_chunk; }; diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 7945bd97e04e..f8d5143493f0 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes } PlanSquashingTransform::PlanSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) - : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), balance(header, min_block_size_rows, min_block_size_bytes) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), squashing(header, min_block_size_rows, min_block_size_bytes) { } @@ -29,9 +29,6 @@ IProcessor::Status PlanSquashingTransform::prepare() case READ_IF_CAN: status = prepareConsume(); break; - case WAIT_IN: - planning_status = PlanningStatus::READ_IF_CAN; - return Status::NeedData; case PUSH: return sendOrFlush(); case FLUSH: @@ -64,17 +61,21 @@ void PlanSquashingTransform::init() IProcessor::Status PlanSquashingTransform::prepareConsume() { - bool inputs_have_no_data = true, all_finished = true; + bool all_finished = true; for (auto & input : inputs) { if (!input.isFinished()) all_finished = false; + else + { + input.setNeeded(); + continue; + } if (input.hasData()) { - inputs_have_no_data = false; chunk = input.pull(); - transform(chunk); + chunk = transform(std::move(chunk)); if (chunk.hasChunkInfo()) { @@ -86,62 +87,27 @@ IProcessor::Status PlanSquashingTransform::prepareConsume() if (all_finished) /// If all inputs are closed, we check if we have data in balancing { - if (balance.isDataLeft()) /// If we have data in balancing, we process this data + if (squashing.isDataLeft()) /// If we have data in balancing, we process this data { planning_status = PlanningStatus::FLUSH; flushChunk(); return Status::Ready; } - planning_status = PlanningStatus::PUSH; - return Status::Ready; - } - - if (inputs_have_no_data) - planning_status = PlanningStatus::WAIT_IN; - - return Status::Ready; -} - -IProcessor::Status PlanSquashingTransform::waitForDataIn() -{ - bool all_finished = true; - bool inputs_have_no_data = true; - for (auto & input : inputs) - { - if (input.isFinished()) - continue; - - all_finished = false; - - if (input.hasData()) - inputs_have_no_data = false; - - } - if (all_finished) - { - planning_status = PlanningStatus::READ_IF_CAN; - return Status::Ready; - } - - if (!inputs_have_no_data) - { - planning_status = PlanningStatus::READ_IF_CAN; + planning_status = PlanningStatus::FINISH; return Status::Ready; } return Status::NeedData; } -void PlanSquashingTransform::transform(Chunk & chunk_) +Chunk PlanSquashingTransform::transform(Chunk && chunk_) { - Chunk res_chunk = balance.add(std::move(chunk_)); - std::swap(res_chunk, chunk_); + return squashing.add(std::move(chunk_)); } -void PlanSquashingTransform::flushChunk() +Chunk PlanSquashingTransform::flushChunk() { - Chunk res_chunk = balance.flush(); - std::swap(res_chunk, chunk); + return squashing.flush(); } IProcessor::Status PlanSquashingTransform::sendOrFlush() diff --git a/src/Processors/Transforms/PlanSquashingTransform.h b/src/Processors/Transforms/PlanSquashingTransform.h index 7afc942a7f23..a9152d9dbe94 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.h +++ b/src/Processors/Transforms/PlanSquashingTransform.h @@ -8,7 +8,6 @@ enum PlanningStatus { INIT, READ_IF_CAN, - WAIT_IN, PUSH, FLUSH, FINISH @@ -36,12 +35,12 @@ class PlanSquashingTransform : public IProcessor Status waitForDataIn(); Status finish(); - void transform(Chunk & chunk); - void flushChunk(); + Chunk transform(Chunk && chunk); + Chunk flushChunk(); private: Chunk chunk; - PlanSquashing balance; + Squashing squashing; PlanningStatus planning_status = PlanningStatus::INIT; }; } diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index a516811bf456..c1f8a9f25139 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -12,16 +12,15 @@ extern const int LOGICAL_ERROR; SquashingTransform::SquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , planSquashing(header, min_block_size_rows, min_block_size_bytes) - , applySquashing(header) + , squashing(header, min_block_size_rows, min_block_size_bytes) { } void SquashingTransform::onConsume(Chunk chunk) { - Chunk planned_chunk = planSquashing.add(std::move(chunk)); + Chunk planned_chunk = squashing.add(std::move(chunk)); if (planned_chunk.hasChunkInfo()) - cur_chunk = applySquashing.add(std::move(planned_chunk)); + cur_chunk = squashing.squash(std::move(planned_chunk)); } SquashingTransform::GenerateResult SquashingTransform::onGenerate() @@ -34,9 +33,9 @@ SquashingTransform::GenerateResult SquashingTransform::onGenerate() void SquashingTransform::onFinish() { - Chunk chunk = planSquashing.flush(); + Chunk chunk = squashing.flush(); if (chunk.hasChunkInfo()) - chunk = applySquashing.add(std::move(chunk)); + chunk = squashing.squash(std::move(chunk)); finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); } @@ -60,8 +59,7 @@ void SquashingTransform::work() SimpleSquashingTransform::SimpleSquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ISimpleTransform(header, header, false) - , planSquashing(header, min_block_size_rows, min_block_size_bytes) - , applySquashing(header) + , squashing(header, min_block_size_rows, min_block_size_bytes) { } @@ -69,18 +67,18 @@ void SimpleSquashingTransform::transform(Chunk & chunk) { if (!finished) { - Chunk planned_chunk = planSquashing.add(std::move(chunk)); + Chunk planned_chunk = squashing.add(std::move(chunk)); if (planned_chunk.hasChunkInfo()) - chunk = applySquashing.add(std::move(planned_chunk)); + chunk = squashing.squash(std::move(planned_chunk)); } else { if (chunk.hasRows()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk expected to be empty, otherwise it will be lost"); - chunk = planSquashing.flush(); + chunk = squashing.flush(); if (chunk.hasChunkInfo()) - chunk = applySquashing.add(std::move(chunk)); + chunk = squashing.squash(std::move(chunk)); } } diff --git a/src/Processors/Transforms/SquashingTransform.h b/src/Processors/Transforms/SquashingTransform.h index b5b3c6616d2c..c5b727ac6ec7 100644 --- a/src/Processors/Transforms/SquashingTransform.h +++ b/src/Processors/Transforms/SquashingTransform.h @@ -24,8 +24,7 @@ class SquashingTransform : public ExceptionKeepingTransform void onFinish() override; private: - PlanSquashing planSquashing; - ApplySquashing applySquashing; + Squashing squashing; Chunk cur_chunk; Chunk finish_chunk; }; @@ -44,8 +43,7 @@ class SimpleSquashingTransform : public ISimpleTransform IProcessor::Status prepare() override; private: - PlanSquashing planSquashing; - ApplySquashing applySquashing; + Squashing squashing; bool finished = false; }; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 1dd997967544..2be4e8d56654 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -885,22 +885,21 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro using PushResult = AsynchronousInsertQueue::PushResult; startInsertQuery(); - PlanSquashing plan_squashing(state.input_header, 0, query_context->getSettingsRef().async_insert_max_data_size); - ApplySquashing apply_squashing(state.input_header); + Squashing squashing(state.input_header, 0, query_context->getSettingsRef().async_insert_max_data_size); while (readDataNext()) { - apply_squashing.header = state.block_for_insert; - auto planned_chunk = plan_squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); + squashing.header = state.block_for_insert; + auto planned_chunk = squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); if (planned_chunk.hasChunkInfo()) { - Chunk result_chunk = apply_squashing.add(std::move(planned_chunk)); + Chunk result_chunk = squashing.squash(std::move(planned_chunk)); ColumnsWithTypeAndName cols; if (result_chunk.hasColumns() && state.block_for_insert) for (size_t j = 0; j < result_chunk.getNumColumns(); ++j) cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.block_for_insert.getDataTypes()[j], state.block_for_insert.getNames()[j])); auto result = Block(cols); - apply_squashing.header = Block(state.block_for_insert); + squashing.header = Block(state.block_for_insert); return PushResult { .status = PushResult::TOO_MUCH_DATA, @@ -909,14 +908,14 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro } } - auto planned_chunk = plan_squashing.flush(); + auto planned_chunk = squashing.flush(); Chunk result_chunk; if (planned_chunk.hasChunkInfo()) - result_chunk = apply_squashing.add(std::move(planned_chunk)); + result_chunk = squashing.squash(std::move(planned_chunk)); ColumnsWithTypeAndName cols; if (result_chunk.hasColumns()) for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], apply_squashing.header.getDataTypes()[j], apply_squashing.header.getNames()[j])); + cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], squashing.header.getDataTypes()[j], squashing.header.getNames()[j])); auto result = Block(cols); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 0d1fc46ec76d..fad195d6a364 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1267,8 +1267,7 @@ class PartMergerWriter ProjectionNameToItsBlocks projection_parts; std::move_iterator projection_parts_iterator; - std::vector projection_squash_plannings; - std::vector projection_squashes; + std::vector projection_squashes; const ProjectionsDescription & projections; ExecutableTaskPtr merge_projection_parts_task_ptr; @@ -1286,10 +1285,9 @@ void PartMergerWriter::prepare() for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { - PlanSquashing plan_squashing(ctx->updated_header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + Squashing squashing(ctx->updated_header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); // We split the materialization into multiple stages similar to the process of INSERT SELECT query. - projection_squash_plannings.emplace_back(ctx->updated_header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); - projection_squashes.emplace_back(ctx->updated_header); + projection_squashes.emplace_back(ctx->updated_header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); } existing_rows_count = 0; @@ -1317,11 +1315,11 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); Block block_to_squash = projection.calculate(cur_block, ctx->context); projection_squashes[i].header = block_to_squash; - Chunk planned_chunk = projection_squash_plannings[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); + Chunk planned_chunk = projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); if (planned_chunk.hasChunkInfo()) { - Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); + Chunk projection_chunk = projection_squashes[i].squash(std::move(planned_chunk)); ColumnsWithTypeAndName cols; if (projection_chunk.hasColumns()) for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) @@ -1345,11 +1343,11 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { const auto & projection = *ctx->projections_to_build[i]; - auto & projection_squash_plan = projection_squash_plannings[i]; + auto & projection_squash_plan = projection_squashes[i]; auto planned_chunk = projection_squash_plan.flush(); if (planned_chunk.hasChunkInfo()) { - Chunk projection_chunk = projection_squashes[i].add(std::move(planned_chunk)); + Chunk projection_chunk = projection_squashes[i].squash(std::move(planned_chunk)); ColumnsWithTypeAndName cols; if (projection_chunk.hasColumns()) for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index c88582a8a1ac..37ea3f274b61 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -313,7 +313,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) // There should be only one output block after this transformation. builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, 1)); - builder.addTransform(std::make_shared(builder.getHeader())); + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); From 8be3957f104d84ab28051c0e4953bc4d9338bf50 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 10 Jun 2024 20:20:46 +0200 Subject: [PATCH 117/254] style check --- src/Interpreters/Squashing.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index a05c5853ce34..05d99d89b15e 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -84,7 +84,7 @@ Chunk Squashing::add(Chunk && input_chunk) return {}; } -Chunk Squashing::convertToChunk(std::vector && chunks) const +Chunk Squashing::convertToChunk(std::vector && chunks) const { if (chunks.empty()) return {}; From d2be2ce70544564e33d33b33fa38db5d84d082fd Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 10 Jun 2024 19:01:48 +0000 Subject: [PATCH 118/254] fixes in planner --- src/Interpreters/Squashing.cpp | 9 ++++----- src/Interpreters/Squashing.h | 3 +-- src/Processors/Transforms/PlanSquashingTransform.cpp | 5 ++--- 3 files changed, 7 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index a05c5853ce34..588b75d33ded 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -28,9 +28,7 @@ Chunk Squashing::squash(Chunk && input_chunk) return Chunk(); const auto *info = getInfoFromChunk(input_chunk); - squash(info->chunks); - - return std::move(accumulated_chunk); + return squash(info->chunks); } Chunk Squashing::add(Chunk && input_chunk) @@ -97,9 +95,9 @@ Chunk Squashing::convertToChunk(std::vector && chunks) const return Chunk(header.cloneEmptyColumns(), 0, info); } -void Squashing::squash(std::vector & input_chunks) +Chunk Squashing::squash(std::vector & input_chunks) { - accumulated_chunk = {}; + Chunk accumulated_chunk; std::vector mutable_columns = {}; size_t rows = 0; for (const Chunk & chunk : input_chunks) @@ -126,6 +124,7 @@ void Squashing::squash(std::vector & input_chunks) } } accumulated_chunk.setColumns(std::move(mutable_columns), rows); + return accumulated_chunk; } const ChunksToSquash* Squashing::getInfoFromChunk(const Chunk & chunk) diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index 760b7d7475f4..15b543be236d 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -55,11 +55,10 @@ class Squashing size_t min_block_size_bytes; CurrentSize accumulated_size; - Chunk accumulated_chunk; const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); - void squash(std::vector & input_chunks); + Chunk squash(std::vector & input_chunks); void expandCurrentSize(size_t rows, size_t bytes); void changeCurrentSize(size_t rows, size_t bytes); diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index f8d5143493f0..5ae605cc51a1 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -27,8 +27,7 @@ IProcessor::Status PlanSquashingTransform::prepare() init(); break; case READ_IF_CAN: - status = prepareConsume(); - break; + return prepareConsume(); case PUSH: return sendOrFlush(); case FLUSH: @@ -90,7 +89,7 @@ IProcessor::Status PlanSquashingTransform::prepareConsume() if (squashing.isDataLeft()) /// If we have data in balancing, we process this data { planning_status = PlanningStatus::FLUSH; - flushChunk(); + chunk = flushChunk(); return Status::Ready; } planning_status = PlanningStatus::FINISH; From 7b239f110245c53dbfc6d296eea4d975d019867e Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 11 Jun 2024 17:36:49 +0000 Subject: [PATCH 119/254] fixes due to review --- src/Interpreters/Squashing.cpp | 16 ++++++------ src/Interpreters/Squashing.h | 10 +++----- src/Processors/Chunk.h | 2 +- .../Transforms/ApplySquashingTransform.h | 6 ++--- .../Transforms/PlanSquashingTransform.cpp | 2 +- .../Transforms/SquashingTransform.cpp | 12 ++++----- src/Server/TCPHandler.cpp | 22 ++++++---------- src/Storages/MergeTree/MutateTask.cpp | 25 ++++++++----------- 8 files changed, 41 insertions(+), 54 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 2c0e7859d442..7964379a35d9 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -10,9 +10,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) - : header(header_) - , min_block_size_rows(min_block_size_rows_) +Squashing::Squashing(size_t min_block_size_rows_, size_t min_block_size_bytes_) + : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) { } @@ -87,12 +86,15 @@ Chunk Squashing::convertToChunk(std::vector && chunks) const if (chunks.empty()) return {}; + auto chunk = Chunk(chunks.back().cloneEmptyColumns(), 0); + auto info = std::make_shared(); info->chunks = std::move(chunks); chunks.clear(); - return Chunk(header.cloneEmptyColumns(), 0, info); + chunk.setChunkInfo(info); + return chunk; } Chunk Squashing::squash(std::vector & input_chunks) @@ -106,10 +108,10 @@ Chunk Squashing::squash(std::vector & input_chunks) { auto & first_chunk = input_chunks[0]; Columns columns = first_chunk.detachColumns(); - for (size_t i = 0; i < columns.size(); ++i) + for (auto & column : columns) { - mutable_columns.push_back(IColumn::mutate(std::move(columns[i]))); - mutable_columns[i]->reserve(rows); + mutable_columns.push_back(IColumn::mutate(std::move(column))); + mutable_columns.back()->reserve(rows); } } diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index 15b543be236d..ea991d6dc852 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -29,11 +29,11 @@ struct ChunksToSquash : public ChunkInfo class Squashing { public: - explicit Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); + explicit Squashing(size_t min_block_size_rows_, size_t min_block_size_bytes_); Squashing(Squashing && other) = default; Chunk add(Chunk && input_chunk); - Chunk squash(Chunk && input_chunk); + static Chunk squash(Chunk && input_chunk); Chunk flush(); bool isDataLeft() @@ -41,8 +41,6 @@ class Squashing return !chunks_to_merge_vec.empty(); } - Block header; - private: struct CurrentSize { @@ -56,9 +54,9 @@ class Squashing CurrentSize accumulated_size; - const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); + static const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); - Chunk squash(std::vector & input_chunks); + static Chunk squash(std::vector & input_chunks); void expandCurrentSize(size_t rows, size_t bytes); void changeCurrentSize(size_t rows, size_t bytes); diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 4f753798eaa8..3c0952f7aedc 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -89,7 +89,7 @@ class Chunk UInt64 getNumColumns() const { return columns.size(); } bool hasRows() const { return num_rows > 0; } bool hasColumns() const { return !columns.empty(); } - bool empty() const { return !hasRows() && !hasColumns(); } + bool empty() const { return !hasRows() && !hasColumns() && !hasChunkInfo(); } explicit operator bool() const { return !empty(); } void addColumn(ColumnPtr column); diff --git a/src/Processors/Transforms/ApplySquashingTransform.h b/src/Processors/Transforms/ApplySquashingTransform.h index 7bf1f32340be..2de614b8dbd5 100644 --- a/src/Processors/Transforms/ApplySquashingTransform.h +++ b/src/Processors/Transforms/ApplySquashingTransform.h @@ -11,7 +11,7 @@ class ApplySquashingTransform : public ExceptionKeepingTransform public: explicit ApplySquashingTransform(const Block & header, const size_t min_block_size_rows, const size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(header, min_block_size_rows, min_block_size_bytes) + , squashing(min_block_size_rows, min_block_size_bytes) { } @@ -37,7 +37,7 @@ class ApplySquashingTransform : public ExceptionKeepingTransform protected: void onConsume(Chunk chunk) override { - if (auto res_chunk = squashing.squash(std::move(chunk))) + if (auto res_chunk = DB::Squashing::squash(std::move(chunk))) cur_chunk.setColumns(res_chunk.getColumns(), res_chunk.getNumRows()); } @@ -50,7 +50,7 @@ class ApplySquashingTransform : public ExceptionKeepingTransform } void onFinish() override { - auto chunk = squashing.squash({}); + auto chunk = Chunk(); finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); } diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 5ae605cc51a1..71e4ee15eccc 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes } PlanSquashingTransform::PlanSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) - : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), squashing(header, min_block_size_rows, min_block_size_bytes) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), squashing(min_block_size_rows, min_block_size_bytes) { } diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index c1f8a9f25139..34c5b1a7202b 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -12,7 +12,7 @@ extern const int LOGICAL_ERROR; SquashingTransform::SquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(header, min_block_size_rows, min_block_size_bytes) + , squashing(min_block_size_rows, min_block_size_bytes) { } @@ -20,7 +20,7 @@ void SquashingTransform::onConsume(Chunk chunk) { Chunk planned_chunk = squashing.add(std::move(chunk)); if (planned_chunk.hasChunkInfo()) - cur_chunk = squashing.squash(std::move(planned_chunk)); + cur_chunk = DB::Squashing::squash(std::move(planned_chunk)); } SquashingTransform::GenerateResult SquashingTransform::onGenerate() @@ -35,7 +35,7 @@ void SquashingTransform::onFinish() { Chunk chunk = squashing.flush(); if (chunk.hasChunkInfo()) - chunk = squashing.squash(std::move(chunk)); + chunk = DB::Squashing::squash(std::move(chunk)); finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); } @@ -59,7 +59,7 @@ void SquashingTransform::work() SimpleSquashingTransform::SimpleSquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ISimpleTransform(header, header, false) - , squashing(header, min_block_size_rows, min_block_size_bytes) + , squashing(min_block_size_rows, min_block_size_bytes) { } @@ -69,7 +69,7 @@ void SimpleSquashingTransform::transform(Chunk & chunk) { Chunk planned_chunk = squashing.add(std::move(chunk)); if (planned_chunk.hasChunkInfo()) - chunk = squashing.squash(std::move(planned_chunk)); + chunk = DB::Squashing::squash(std::move(planned_chunk)); } else { @@ -78,7 +78,7 @@ void SimpleSquashingTransform::transform(Chunk & chunk) chunk = squashing.flush(); if (chunk.hasChunkInfo()) - chunk = squashing.squash(std::move(chunk)); + chunk = DB::Squashing::squash(std::move(chunk)); } } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index cfabc9cef34b..940fa94e182c 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -886,21 +886,17 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro using PushResult = AsynchronousInsertQueue::PushResult; startInsertQuery(); - Squashing squashing(state.input_header, 0, query_context->getSettingsRef().async_insert_max_data_size); + Squashing squashing(0, query_context->getSettingsRef().async_insert_max_data_size); + Block header; while (readDataNext()) { - squashing.header = state.block_for_insert; + header = state.block_for_insert.cloneWithoutColumns(); auto planned_chunk = squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); if (planned_chunk.hasChunkInfo()) { - Chunk result_chunk = squashing.squash(std::move(planned_chunk)); - ColumnsWithTypeAndName cols; - if (result_chunk.hasColumns() && state.block_for_insert) - for (size_t j = 0; j < result_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], state.block_for_insert.getDataTypes()[j], state.block_for_insert.getNames()[j])); - auto result = Block(cols); - squashing.header = Block(state.block_for_insert); + Chunk result_chunk = DB::Squashing::squash(std::move(planned_chunk)); + auto result = state.block_for_insert.cloneWithColumns(result_chunk.getColumns()); return PushResult { .status = PushResult::TOO_MUCH_DATA, @@ -912,13 +908,9 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro auto planned_chunk = squashing.flush(); Chunk result_chunk; if (planned_chunk.hasChunkInfo()) - result_chunk = squashing.squash(std::move(planned_chunk)); - ColumnsWithTypeAndName cols; - if (result_chunk.hasColumns()) - for (size_t j = 0; j < result_chunk.getNumColumns(); ++ j) - cols.push_back(ColumnWithTypeAndName(result_chunk.getColumns()[j], squashing.header.getDataTypes()[j], squashing.header.getNames()[j])); + result_chunk = DB::Squashing::squash(std::move(planned_chunk)); - auto result = Block(cols); + auto result = header.cloneWithColumns(result_chunk.getColumns()); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 1d0c24c0f08f..6fcc8f4ad924 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1286,9 +1286,8 @@ void PartMergerWriter::prepare() for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { - Squashing squashing(ctx->updated_header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); // We split the materialization into multiple stages similar to the process of INSERT SELECT query. - projection_squashes.emplace_back(ctx->updated_header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + projection_squashes.emplace_back(settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); } existing_rows_count = 0; @@ -1298,6 +1297,7 @@ void PartMergerWriter::prepare() bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Block cur_block; + Block header; if (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor->pull(cur_block)) { if (ctx->minmax_idx) @@ -1315,18 +1315,16 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); Block block_to_squash = projection.calculate(cur_block, ctx->context); - projection_squashes[i].header = block_to_squash; + header = block_to_squash.cloneWithoutColumns(); Chunk planned_chunk = projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); if (planned_chunk.hasChunkInfo()) { - Chunk projection_chunk = projection_squashes[i].squash(std::move(planned_chunk)); - ColumnsWithTypeAndName cols; - if (projection_chunk.hasColumns()) - for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], block_to_squash.getDataTypes()[j], block_to_squash.getNames()[j])); + Chunk projection_chunk = DB::Squashing::squash(std::move(planned_chunk)); + + auto result = block_to_squash.cloneWithColumns(projection_chunk.getColumns()); auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( - *ctx->data, ctx->log, Block(cols), projection, ctx->new_data_part.get(), ++block_num); + *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); tmp_part.finalize(); tmp_part.part->getDataPartStorage().commitTransaction(); projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); @@ -1348,14 +1346,11 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() auto planned_chunk = projection_squash_plan.flush(); if (planned_chunk.hasChunkInfo()) { - Chunk projection_chunk = projection_squashes[i].squash(std::move(planned_chunk)); - ColumnsWithTypeAndName cols; - if (projection_chunk.hasColumns()) - for (size_t j = 0; j < projection_chunk.getNumColumns(); ++j) - cols.push_back(ColumnWithTypeAndName(projection_chunk.getColumns()[j], projection_squashes[i].header.getDataTypes()[j], projection_squashes[i].header.getNames()[j])); + Chunk projection_chunk = DB::Squashing::squash(std::move(planned_chunk)); + auto result = header.cloneWithColumns(projection_chunk.getColumns()); auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( - *ctx->data, ctx->log, Block(cols), projection, ctx->new_data_part.get(), ++block_num); + *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); temp_part.finalize(); temp_part.part->getDataPartStorage().commitTransaction(); projection_parts[projection.name].emplace_back(std::move(temp_part.part)); From 85289219f12219f74efd6ebc932707c7b3d1f2ce Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 11 Jun 2024 18:03:44 +0000 Subject: [PATCH 120/254] Remove flags drom ActionsDAG (part 1). --- src/Interpreters/ActionsDAG.cpp | 3 - src/Interpreters/ActionsDAG.h | 17 +- src/Interpreters/ActionsVisitor.cpp | 56 +++---- src/Interpreters/ActionsVisitor.h | 15 +- src/Interpreters/ExpressionActions.cpp | 19 +-- src/Interpreters/ExpressionActions.h | 36 ++-- src/Interpreters/ExpressionAnalyzer.cpp | 214 ++++++++++++------------ src/Interpreters/ExpressionAnalyzer.h | 56 +++---- src/Interpreters/WindowDescription.h | 4 +- 9 files changed, 206 insertions(+), 214 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 54db8980e12e..271f8f7474e2 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1150,8 +1150,6 @@ void ActionsDAG::project(const NamesWithAliases & projection) } removeUnusedActions(); - projectInput(); - projected_output = true; } bool ActionsDAG::tryRestoreColumn(const std::string & column_name) @@ -1526,7 +1524,6 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( actions_dag->outputs.swap(projection); actions_dag->removeUnusedActions(); - actions_dag->projectInput(); return actions_dag; } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 150fa84762f0..ae2cb4129d55 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -103,13 +103,11 @@ class ActionsDAG NodeRawConstPtrs inputs; NodeRawConstPtrs outputs; - bool project_input = false; - bool projected_output = false; - public: ActionsDAG() = default; ActionsDAG(ActionsDAG &&) = default; ActionsDAG(const ActionsDAG &) = delete; + ActionsDAG & operator=(ActionsDAG &&) = default; ActionsDAG & operator=(const ActionsDAG &) = delete; explicit ActionsDAG(const NamesAndTypesList & inputs_); explicit ActionsDAG(const ColumnsWithTypeAndName & inputs_); @@ -179,10 +177,6 @@ class ActionsDAG /// Return true if column was removed from inputs. bool removeUnusedResult(const std::string & column_name); - void projectInput(bool project = true) { project_input = project; } - bool isInputProjected() const { return project_input; } - bool isOutputProjected() const { return projected_output; } - /// Remove actions that are not needed to compute output nodes void removeUnusedActions(bool allow_remove_inputs = true, bool allow_constant_folding = true); @@ -510,4 +504,13 @@ struct ActionDAGNodes ActionsDAG::NodeRawConstPtrs nodes; }; +struct ActionsAndFlags +{ + ActionsDAG actions; + bool project_input = false; + bool projected_output = false; +}; + +using ActionsAndFlagsPtr = std::shared_ptr; + } diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 0bdd4c089f1d..b41c4509bd36 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -486,16 +486,12 @@ class ScopeStack::Index } }; -ScopeStack::Level::~Level() = default; -ScopeStack::Level::Level() = default; -ScopeStack::Level::Level(Level &&) noexcept = default; - ActionsMatcher::Data::Data( ContextPtr context_, SizeLimits set_size_limit_, size_t subquery_depth_, std::reference_wrapper source_columns_, - ActionsDAGPtr actions_dag, + ActionsDAG actions_dag, PreparedSetsPtr prepared_sets_, bool no_subqueries_, bool no_makeset_, @@ -531,13 +527,12 @@ std::vector ActionsMatcher::Data::getAllColumnNames() const return index.getAllNames(); } -ScopeStack::ScopeStack(ActionsDAGPtr actions_dag, ContextPtr context_) : WithContext(context_) +ScopeStack::ScopeStack(ActionsDAG actions_dag, ContextPtr context_) : WithContext(context_) { - auto & level = stack.emplace_back(); - level.actions_dag = std::move(actions_dag); - level.index = std::make_unique(level.actions_dag->getOutputs()); + auto & level = stack.emplace_back(ScopeStack::Level{std::move(actions_dag), {}, {}}); + level.index = std::make_unique(level.actions_dag.getOutputs()); - for (const auto & node : level.actions_dag->getOutputs()) + for (const auto & node : level.actions_dag.getOutputs()) if (node->type == ActionsDAG::ActionType::INPUT) level.inputs.emplace(node->result_name); } @@ -545,22 +540,21 @@ ScopeStack::ScopeStack(ActionsDAGPtr actions_dag, ContextPtr context_) : WithCon void ScopeStack::pushLevel(const NamesAndTypesList & input_columns) { auto & level = stack.emplace_back(); - level.actions_dag = std::make_shared(); - level.index = std::make_unique(level.actions_dag->getOutputs()); + level.index = std::make_unique(level.actions_dag.getOutputs()); const auto & prev = stack[stack.size() - 2]; for (const auto & input_column : input_columns) { - const auto & node = level.actions_dag->addInput(input_column.name, input_column.type); + const auto & node = level.actions_dag.addInput(input_column.name, input_column.type); level.index->addNode(&node); level.inputs.emplace(input_column.name); } - for (const auto & node : prev.actions_dag->getOutputs()) + for (const auto & node : prev.actions_dag.getOutputs()) { if (!level.index->contains(node->result_name)) { - const auto & input = level.actions_dag->addInput({node->column, node->result_type, node->result_name}); + const auto & input = level.actions_dag.addInput({node->column, node->result_type, node->result_name}); level.index->addNode(&input); } } @@ -585,12 +579,12 @@ size_t ScopeStack::getColumnLevel(const std::string & name) void ScopeStack::addColumn(ColumnWithTypeAndName column) { - const auto & node = stack[0].actions_dag->addColumn(std::move(column)); + const auto & node = stack[0].actions_dag.addColumn(std::move(column)); stack[0].index->addNode(&node); for (size_t j = 1; j < stack.size(); ++j) { - const auto & input = stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name}); + const auto & input = stack[j].actions_dag.addInput({node.column, node.result_type, node.result_name}); stack[j].index->addNode(&input); } } @@ -599,12 +593,12 @@ void ScopeStack::addAlias(const std::string & name, std::string alias) { auto level = getColumnLevel(name); const auto & source = stack[level].index->getNode(name); - const auto & node = stack[level].actions_dag->addAlias(source, std::move(alias)); + const auto & node = stack[level].actions_dag.addAlias(source, std::move(alias)); stack[level].index->addNode(&node); for (size_t j = level + 1; j < stack.size(); ++j) { - const auto & input = stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name}); + const auto & input = stack[j].actions_dag.addInput({node.column, node.result_type, node.result_name}); stack[j].index->addNode(&input); } } @@ -618,12 +612,12 @@ void ScopeStack::addArrayJoin(const std::string & source_name, std::string resul throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression with arrayJoin cannot depend on lambda argument: {}", source_name); - const auto & node = stack.front().actions_dag->addArrayJoin(*source_node, std::move(result_name)); + const auto & node = stack.front().actions_dag.addArrayJoin(*source_node, std::move(result_name)); stack.front().index->addNode(&node); for (size_t j = 1; j < stack.size(); ++j) { - const auto & input = stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name}); + const auto & input = stack[j].actions_dag.addInput({node.column, node.result_type, node.result_name}); stack[j].index->addNode(&input); } } @@ -642,17 +636,17 @@ void ScopeStack::addFunction( for (const auto & argument : argument_names) children.push_back(&stack[level].index->getNode(argument)); - const auto & node = stack[level].actions_dag->addFunction(function, std::move(children), std::move(result_name)); + const auto & node = stack[level].actions_dag.addFunction(function, std::move(children), std::move(result_name)); stack[level].index->addNode(&node); for (size_t j = level + 1; j < stack.size(); ++j) { - const auto & input = stack[j].actions_dag->addInput({node.column, node.result_type, node.result_name}); + const auto & input = stack[j].actions_dag.addInput({node.column, node.result_type, node.result_name}); stack[j].index->addNode(&input); } } -ActionsDAGPtr ScopeStack::popLevel() +ActionsDAG ScopeStack::popLevel() { auto res = std::move(stack.back().actions_dag); stack.pop_back(); @@ -661,12 +655,12 @@ ActionsDAGPtr ScopeStack::popLevel() std::string ScopeStack::dumpNames() const { - return stack.back().actions_dag->dumpNames(); + return stack.back().actions_dag.dumpNames(); } const ActionsDAG & ScopeStack::getLastActions() const { - return *stack.back().actions_dag; + return stack.back().actions_dag; } const ScopeStack::Index & ScopeStack::getLastActionsIndex() const @@ -989,7 +983,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data.set_size_limit, data.subquery_depth, data.source_columns, - std::make_shared(data.source_columns), + ActionsDAG(data.source_columns), data.prepared_sets, data.no_subqueries, data.no_makeset, @@ -1008,10 +1002,10 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & } auto dag = index_hint_data.getActions(); - dag->project(args); + dag.project(args); auto index_hint = std::make_shared(); - index_hint->setActions(std::move(dag)); + index_hint->setActions(std::make_shared(std::move(dag))); // Arguments are removed. We add function instead of constant column to avoid constant folding. data.addFunction(std::make_unique(index_hint), {}, column_name); @@ -1271,10 +1265,10 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & auto lambda_dag = data.actions_stack.popLevel(); String result_name = lambda->arguments->children.at(1)->getColumnName(); - lambda_dag->removeUnusedActions(Names(1, result_name)); + lambda_dag.removeUnusedActions(Names(1, result_name)); auto lambda_actions = std::make_shared( - lambda_dag, + std::make_shared(lambda_dag), ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes)); DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 046c7387ee83..a0064637939b 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB { @@ -43,20 +44,16 @@ struct ScopeStack : WithContext struct Level { - ActionsDAGPtr actions_dag; + ActionsDAG actions_dag; IndexPtr index; NameSet inputs; - - Level(); - Level(Level &&) noexcept; - ~Level(); }; using Levels = std::vector; Levels stack; - ScopeStack(ActionsDAGPtr actions_dag, ContextPtr context_); + ScopeStack(ActionsDAG actions_dag, ContextPtr context_); void pushLevel(const NamesAndTypesList & input_columns); @@ -67,7 +64,7 @@ struct ScopeStack : WithContext void addArrayJoin(const std::string & source_name, std::string result_name); void addFunction(const FunctionOverloadResolverPtr & function, const Names & argument_names, std::string result_name); - ActionsDAGPtr popLevel(); + ActionsDAG popLevel(); const ActionsDAG & getLastActions() const; const Index & getLastActionsIndex() const; @@ -147,7 +144,7 @@ class ActionsMatcher SizeLimits set_size_limit_, size_t subquery_depth_, std::reference_wrapper source_columns_, - ActionsDAGPtr actions_dag, + ActionsDAG actions_dag, PreparedSetsPtr prepared_sets_, bool no_subqueries_, bool no_makeset_, @@ -182,7 +179,7 @@ class ActionsMatcher actions_stack.addFunction(function, argument_names, std::move(result_name)); } - ActionsDAGPtr getActions() + ActionsDAG getActions() { return actions_stack.popLevel(); } diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 04f29f35c3c4..3ba448a0da0f 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -757,11 +757,7 @@ void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run, } } - if (actions_dag->isInputProjected()) - { - block.clear(); - } - else if (allow_duplicates_in_input) + if (allow_duplicates_in_input) { /// This case is the same as when the input is projected /// since we do not need any input columns. @@ -862,7 +858,7 @@ std::string ExpressionActions::dumpActions() const for (const auto & output_column : output_columns) ss << output_column.name << " " << output_column.type->getName() << "\n"; - ss << "\nproject input: " << actions_dag->isInputProjected() << "\noutput positions:"; + ss << "\noutput positions:"; for (auto pos : result_positions) ss << " " << pos; ss << "\n"; @@ -926,7 +922,6 @@ JSONBuilder::ItemPtr ExpressionActions::toTree() const map->add("Actions", std::move(actions_array)); map->add("Outputs", std::move(outputs_array)); map->add("Positions", std::move(positions_array)); - map->add("Project Input", actions_dag->isInputProjected()); return map; } @@ -980,7 +975,7 @@ void ExpressionActionsChain::addStep(NameSet non_constant_inputs) if (column.column && isColumnConst(*column.column) && non_constant_inputs.contains(column.name)) column.column = nullptr; - steps.push_back(std::make_unique(std::make_shared(columns))); + steps.push_back(std::make_unique(std::make_shared(ActionsDAG(columns), false, false))); } void ExpressionActionsChain::finalize() @@ -1129,14 +1124,14 @@ void ExpressionActionsChain::JoinStep::finalize(const NameSet & required_output_ std::swap(result_columns, new_result_columns); } -ActionsDAGPtr & ExpressionActionsChain::Step::actions() +ActionsAndFlagsPtr & ExpressionActionsChain::Step::actions() { - return typeid_cast(*this).actions_dag; + return typeid_cast(*this).actions_and_flags; } -const ActionsDAGPtr & ExpressionActionsChain::Step::actions() const +const ActionsAndFlagsPtr & ExpressionActionsChain::Step::actions() const { - return typeid_cast(*this).actions_dag; + return typeid_cast(*this).actions_and_flags; } } diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index cb467004d29f..b8ac10fd80e4 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -173,48 +173,48 @@ struct ExpressionActionsChain : WithContext /// Remove unused result and update required columns virtual void finalize(const NameSet & required_output_) = 0; /// Add projections to expression - virtual void prependProjectInput() const = 0; + virtual void prependProjectInput() = 0; virtual std::string dump() const = 0; /// Only for ExpressionActionsStep - ActionsDAGPtr & actions(); - const ActionsDAGPtr & actions() const; + ActionsAndFlagsPtr & actions(); + const ActionsAndFlagsPtr & actions() const; }; struct ExpressionActionsStep : public Step { - ActionsDAGPtr actions_dag; + ActionsAndFlagsPtr actions_and_flags; - explicit ExpressionActionsStep(ActionsDAGPtr actions_dag_, Names required_output_ = Names()) + explicit ExpressionActionsStep(ActionsAndFlagsPtr actiactions_and_flags_, Names required_output_ = Names()) : Step(std::move(required_output_)) - , actions_dag(std::move(actions_dag_)) + , actions_and_flags(std::move(actiactions_and_flags_)) { } NamesAndTypesList getRequiredColumns() const override { - return actions_dag->getRequiredColumns(); + return actions_and_flags->actions.getRequiredColumns(); } ColumnsWithTypeAndName getResultColumns() const override { - return actions_dag->getResultColumns(); + return actions_and_flags->actions.getResultColumns(); } void finalize(const NameSet & required_output_) override { - if (!actions_dag->isOutputProjected()) - actions_dag->removeUnusedActions(required_output_); + if (!actions_and_flags->projected_output) + actions_and_flags->actions.removeUnusedActions(required_output_); } - void prependProjectInput() const override + void prependProjectInput() override { - actions_dag->projectInput(); + actions_and_flags->project_input = true; } std::string dump() const override { - return actions_dag->dumpDAG(); + return actions_and_flags->actions.dumpDAG(); } }; @@ -229,7 +229,7 @@ struct ExpressionActionsChain : WithContext NamesAndTypesList getRequiredColumns() const override { return required_columns; } ColumnsWithTypeAndName getResultColumns() const override { return result_columns; } void finalize(const NameSet & required_output_) override; - void prependProjectInput() const override {} /// TODO: remove unused columns before ARRAY JOIN ? + void prependProjectInput() override {} /// TODO: remove unused columns before ARRAY JOIN ? std::string dump() const override { return "ARRAY JOIN"; } }; @@ -245,7 +245,7 @@ struct ExpressionActionsChain : WithContext NamesAndTypesList getRequiredColumns() const override { return required_columns; } ColumnsWithTypeAndName getResultColumns() const override { return result_columns; } void finalize(const NameSet & required_output_) override; - void prependProjectInput() const override {} /// TODO: remove unused columns before JOIN ? + void prependProjectInput() override {} /// TODO: remove unused columns before JOIN ? std::string dump() const override { return "JOIN"; } }; @@ -263,7 +263,7 @@ struct ExpressionActionsChain : WithContext steps.clear(); } - ActionsDAGPtr getLastActions(bool allow_empty = false) + ActionsAndFlagsPtr getLastActions(bool allow_empty = false) { if (steps.empty()) { @@ -272,7 +272,7 @@ struct ExpressionActionsChain : WithContext throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty ExpressionActionsChain"); } - return typeid_cast(steps.back().get())->actions_dag; + return typeid_cast(steps.back().get())->actions_and_flags; } Step & getLastStep() @@ -286,7 +286,7 @@ struct ExpressionActionsChain : WithContext Step & lastStep(const NamesAndTypesList & columns) { if (steps.empty()) - steps.emplace_back(std::make_unique(std::make_shared(columns))); + steps.emplace_back(std::make_unique(std::make_shared(ActionsDAG(columns), false, false))); return *steps.back(); } diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index d80d5cd5b938..530bace3b1e0 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -186,7 +186,7 @@ ExpressionAnalyzer::ExpressionAnalyzer( /// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers. initGlobalSubqueriesAndExternalTables(do_global, is_explain); - auto temp_actions = std::make_shared(sourceColumns()); + ActionsDAG temp_actions(sourceColumns()); columns_after_array_join = getColumnsAfterArrayJoin(temp_actions, sourceColumns()); columns_after_join = analyzeJoin(temp_actions, columns_after_array_join); /// has_aggregation, aggregation_keys, aggregate_descriptions, aggregated_columns. @@ -199,7 +199,7 @@ ExpressionAnalyzer::ExpressionAnalyzer( analyzeAggregation(temp_actions); } -NamesAndTypesList ExpressionAnalyzer::getColumnsAfterArrayJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns) +NamesAndTypesList ExpressionAnalyzer::getColumnsAfterArrayJoin(ActionsDAG & actions, const NamesAndTypesList & src_columns) { const auto * select_query = query->as(); if (!select_query) @@ -213,14 +213,14 @@ NamesAndTypesList ExpressionAnalyzer::getColumnsAfterArrayJoin(ActionsDAGPtr & a getRootActionsNoMakeSet(array_join_expression_list, actions, false); auto array_join = addMultipleArrayJoinAction(actions, is_array_join_left); - auto sample_columns = actions->getResultColumns(); + auto sample_columns = actions.getResultColumns(); array_join->prepare(sample_columns); - actions = std::make_shared(sample_columns); + actions = ActionsDAG(sample_columns); NamesAndTypesList new_columns_after_array_join; NameSet added_columns; - for (auto & column : actions->getResultColumns()) + for (auto & column : actions.getResultColumns()) { if (syntax->array_join_result_to_source.contains(column.name)) { @@ -236,7 +236,7 @@ NamesAndTypesList ExpressionAnalyzer::getColumnsAfterArrayJoin(ActionsDAGPtr & a return new_columns_after_array_join; } -NamesAndTypesList ExpressionAnalyzer::analyzeJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns) +NamesAndTypesList ExpressionAnalyzer::analyzeJoin(ActionsDAG & actions, const NamesAndTypesList & src_columns) { const auto * select_query = query->as(); if (!select_query) @@ -246,9 +246,9 @@ NamesAndTypesList ExpressionAnalyzer::analyzeJoin(ActionsDAGPtr & actions, const if (join) { getRootActionsNoMakeSet(analyzedJoin().leftKeysList(), actions, false); - auto sample_columns = actions->getNamesAndTypesList(); + auto sample_columns = actions.getNamesAndTypesList(); syntax->analyzed_join->addJoinedColumnsAndCorrectTypes(sample_columns, true); - actions = std::make_shared(sample_columns); + actions = ActionsDAG(sample_columns); } NamesAndTypesList result_columns = src_columns; @@ -256,7 +256,7 @@ NamesAndTypesList ExpressionAnalyzer::analyzeJoin(ActionsDAGPtr & actions, const return result_columns; } -void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) +void ExpressionAnalyzer::analyzeAggregation(ActionsDAG & temp_actions) { /** Find aggregation keys (aggregation_keys), information about aggregate functions (aggregate_descriptions), * as well as a set of columns obtained after the aggregation, if any, @@ -272,7 +272,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) if (!has_aggregation) { - aggregated_columns = temp_actions->getNamesAndTypesList(); + aggregated_columns = temp_actions.getNamesAndTypesList(); return; } @@ -321,7 +321,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) ssize_t group_size = group_elements_ast.size(); const auto & column_name = group_elements_ast[j]->getColumnName(); - const auto * node = temp_actions->tryFindInOutputs(column_name); + const auto * node = temp_actions.tryFindInOutputs(column_name); if (!node) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier (in GROUP BY): {}", column_name); @@ -375,7 +375,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) getRootActionsNoMakeSet(group_asts[i], temp_actions, false); const auto & column_name = group_asts[i]->getColumnName(); - const auto * node = temp_actions->tryFindInOutputs(column_name); + const auto * node = temp_actions.tryFindInOutputs(column_name); if (!node) throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown identifier (in GROUP BY): {}", column_name); @@ -434,7 +434,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) has_const_aggregation_keys = select_query->group_by_with_constant_keys; } else - aggregated_columns = temp_actions->getNamesAndTypesList(); + aggregated_columns = temp_actions.getNamesAndTypesList(); for (const auto & desc : aggregate_descriptions) aggregated_columns.emplace_back(desc.column_name, desc.function->getResultType()); @@ -465,7 +465,7 @@ SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_o return storage_set->getSet(); } -void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts) +void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAG & actions, bool only_consts) { LogAST log; ActionsVisitor::Data visitor_data( @@ -485,7 +485,7 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_ actions = visitor_data.getActions(); } -void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAGPtr & actions, bool only_consts) +void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAG & actions, bool only_consts) { LogAST log; ActionsVisitor::Data visitor_data( @@ -507,7 +507,7 @@ void ExpressionAnalyzer::getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAGP void ExpressionAnalyzer::getRootActionsForHaving( - const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts) + const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAG & actions, bool only_consts) { LogAST log; ActionsVisitor::Data visitor_data( @@ -528,7 +528,7 @@ void ExpressionAnalyzer::getRootActionsForHaving( } -void ExpressionAnalyzer::getRootActionsForWindowFunctions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions) +void ExpressionAnalyzer::getRootActionsForWindowFunctions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAG & actions) { LogAST log; ActionsVisitor::Data visitor_data( @@ -548,7 +548,7 @@ void ExpressionAnalyzer::getRootActionsForWindowFunctions(const ASTPtr & ast, bo } -void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, AggregateDescriptions & descriptions) +void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAG & actions, AggregateDescriptions & descriptions) { for (const ASTPtr & ast : aggregates()) { @@ -567,7 +567,7 @@ void ExpressionAnalyzer::makeAggregateDescriptions(ActionsDAGPtr & actions, Aggr for (size_t i = 0; i < arguments.size(); ++i) { const std::string & name = arguments[i]->getColumnName(); - const auto * dag_node = actions->tryFindInOutputs(name); + const auto * dag_node = actions.tryFindInOutputs(name); if (!dag_node) { throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, @@ -658,7 +658,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, with_alias->getColumnName(), 1 /* direction */, 1 /* nulls_direction */)); - auto actions_dag = std::make_shared(aggregated_columns); + ActionsDAG actions_dag(aggregated_columns); getRootActions(column_ast, false, actions_dag); desc.partition_by_actions.push_back(std::move(actions_dag)); } @@ -679,7 +679,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, order_by_element.direction, order_by_element.nulls_direction)); - auto actions_dag = std::make_shared(aggregated_columns); + ActionsDAG actions_dag(aggregated_columns); getRootActions(column_ast, false, actions_dag); desc.order_by_actions.push_back(std::move(actions_dag)); } @@ -720,7 +720,7 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, } } -void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) +void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAG & actions) { auto current_context = getContext(); @@ -737,13 +737,13 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) desc, elem.definition.get()); auto [it, inserted] = window_descriptions.insert( - {desc.window_name, desc}); + {elem.name, std::move(desc)}); if (!inserted) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "Window '{}' is defined twice in the WINDOW clause", - desc.window_name); + elem.name); } } } @@ -776,7 +776,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) for (size_t i = 0; i < arguments.size(); ++i) { const std::string & name = arguments[i]->getColumnName(); - const auto * node = actions->tryFindInOutputs(name); + const auto * node = actions.tryFindInOutputs(name); if (!node) { @@ -817,13 +817,14 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAGPtr actions) { const auto & definition = function_node.window_definition->as< const ASTWindowDefinition &>(); + auto default_window_name = definition.getDefaultWindowName(); WindowDescription desc; - desc.window_name = definition.getDefaultWindowName(); + desc.window_name = default_window_name; makeWindowDescriptionFromAST(*current_context, window_descriptions, desc, &definition); auto [it, inserted] = window_descriptions.insert( - {desc.window_name, desc}); + {default_window_name, std::move(desc)}); if (!inserted) { @@ -871,7 +872,7 @@ const ASTSelectQuery * SelectQueryExpressionAnalyzer::getAggregatingQuery() cons } /// "Big" ARRAY JOIN. -ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAGPtr & actions, bool array_join_is_left) const +ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAG & actions, bool array_join_is_left) const { NameSet result_columns; for (const auto & result_source : syntax->array_join_result_to_source) @@ -879,8 +880,8 @@ ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAGPtr /// Assign new names to columns, if needed. if (result_source.first != result_source.second) { - const auto & node = actions->findInOutputs(result_source.second); - actions->getOutputs().push_back(&actions->addAlias(node, result_source.first)); + const auto & node = actions.findInOutputs(result_source.second); + actions.getOutputs().push_back(&actions.addAlias(node, result_source.first)); } /// Make ARRAY JOIN (replace arrays with their insides) for the columns in these new names. @@ -890,7 +891,7 @@ ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAGPtr return std::make_shared(result_columns, array_join_is_left, getContext()); } -ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, ActionsDAGPtr & before_array_join, bool only_types) +ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, ActionsAndFlagsPtr & before_array_join, bool only_types) { const auto * select_query = getSelectQuery(); @@ -900,9 +901,9 @@ ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActi ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns()); - getRootActions(array_join_expression_list, only_types, step.actions()); + getRootActions(array_join_expression_list, only_types, step.actions()->actions); - auto array_join = addMultipleArrayJoinAction(step.actions(), is_array_join_left); + auto array_join = addMultipleArrayJoinAction(step.actions()->actions, is_array_join_left); before_array_join = chain.getLastActions(); chain.steps.push_back(std::make_unique(array_join, step.getResultColumns())); @@ -916,17 +917,19 @@ bool SelectQueryExpressionAnalyzer::appendJoinLeftKeys(ExpressionActionsChain & { ExpressionActionsChain::Step & step = chain.lastStep(columns_after_array_join); - getRootActions(analyzedJoin().leftKeysList(), only_types, step.actions()); + getRootActions(analyzedJoin().leftKeysList(), only_types, step.actions()->actions); return true; } JoinPtr SelectQueryExpressionAnalyzer::appendJoin( ExpressionActionsChain & chain, - ActionsDAGPtr & converting_join_columns) + ActionsAndFlagsPtr & converting_join_columns) { const ColumnsWithTypeAndName & left_sample_columns = chain.getLastStep().getResultColumns(); - JoinPtr join = makeJoin(*syntax->ast_join, left_sample_columns, converting_join_columns); + auto tmp_actions = std::make_shared(std::move(converting_join_columns->actions)); + JoinPtr join = makeJoin(*syntax->ast_join, left_sample_columns, tmp_actions); + converting_join_columns->actions = std::move(*tmp_actions); if (converting_join_columns) { @@ -1065,7 +1068,6 @@ static std::unique_ptr buildJoinedPlan( rename_dag->getOutputs()[pos] = &alias; } } - rename_dag->projectInput(); auto rename_step = std::make_unique(joined_plan->getCurrentDataStream(), std::move(rename_dag)); rename_step->setStepDescription("Rename joined columns"); joined_plan->addStep(std::move(rename_step)); @@ -1166,45 +1168,45 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin( return join; } -ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( +ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendPrewhere( ExpressionActionsChain & chain, bool only_types) { const auto * select_query = getSelectQuery(); if (!select_query->prewhere()) - return nullptr; + return {}; Names first_action_names; if (!chain.steps.empty()) first_action_names = chain.steps.front()->getRequiredColumns().getNames(); auto & step = chain.lastStep(sourceColumns()); - getRootActions(select_query->prewhere(), only_types, step.actions()); + getRootActions(select_query->prewhere(), only_types, step.actions()->actions); String prewhere_column_name = select_query->prewhere()->getColumnName(); step.addRequiredOutput(prewhere_column_name); - const auto & node = step.actions()->findInOutputs(prewhere_column_name); + const auto & node = step.actions()->actions.findInOutputs(prewhere_column_name); auto filter_type = node.result_type; if (!filter_type->canBeUsedInBooleanContext()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in PREWHERE: {}", filter_type->getName()); - ActionsDAGPtr prewhere_actions; + ActionsAndFlagsPtr prewhere_actions; { /// Remove unused source_columns from prewhere actions. - auto tmp_actions_dag = std::make_shared(sourceColumns()); + ActionsDAG tmp_actions_dag(sourceColumns()); getRootActions(select_query->prewhere(), only_types, tmp_actions_dag); /// Constants cannot be removed since they can be used in other parts of the query. /// And if they are not used anywhere, except PREWHERE, they will be removed on the next step. - tmp_actions_dag->removeUnusedActions( + tmp_actions_dag.removeUnusedActions( NameSet{prewhere_column_name}, /* allow_remove_inputs= */ true, /* allow_constant_folding= */ false); - auto required_columns = tmp_actions_dag->getRequiredColumnsNames(); + auto required_columns = tmp_actions_dag.getRequiredColumnsNames(); NameSet required_source_columns(required_columns.begin(), required_columns.end()); required_source_columns.insert(first_action_names.begin(), first_action_names.end()); - auto names = step.actions()->getNames(); + auto names = step.actions()->actions.getNames(); NameSet name_set(names.begin(), names.end()); for (const auto & column : sourceColumns()) @@ -1213,13 +1215,13 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( Names required_output(name_set.begin(), name_set.end()); prewhere_actions = chain.getLastActions(); - prewhere_actions->removeUnusedActions(required_output); + prewhere_actions->actions.removeUnusedActions(required_output); } { - ActionsDAGPtr actions; + ActionsDAG actions; - auto required_columns = prewhere_actions->getRequiredColumns(); + auto required_columns = prewhere_actions->actions.getRequiredColumns(); NameSet prewhere_input_names; for (const auto & col : required_columns) prewhere_input_names.insert(col.name); @@ -1263,11 +1265,11 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( } } - actions = std::make_shared(std::move(required_columns)); + actions = ActionsDAG(required_columns); } else { - ColumnsWithTypeAndName columns = prewhere_actions->getResultColumns(); + ColumnsWithTypeAndName columns = prewhere_actions->actions.getResultColumns(); for (const auto & column : sourceColumns()) { @@ -1278,7 +1280,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( } } - actions = std::make_shared(std::move(columns)); + actions = ActionsDAG(columns); } chain.steps.emplace_back( @@ -1300,12 +1302,12 @@ bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(columns_after_join); - getRootActions(select_query->where(), only_types, step.actions()); + getRootActions(select_query->where(), only_types, step.actions()->actions); auto where_column_name = select_query->where()->getColumnName(); step.addRequiredOutput(where_column_name); - const auto & node = step.actions()->findInOutputs(where_column_name); + const auto & node = step.actions()->actions.findInOutputs(where_column_name); auto filter_type = node.result_type; if (!filter_type->canBeUsedInBooleanContext()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in WHERE: {}", @@ -1332,7 +1334,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain for (const auto & ast_element : ast->children) { step.addRequiredOutput(ast_element->getColumnName()); - getRootActions(ast_element, only_types, step.actions()); + getRootActions(ast_element, only_types, step.actions()->actions); } } } @@ -1341,7 +1343,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain for (const auto & ast : asts) { step.addRequiredOutput(ast->getColumnName()); - getRootActions(ast, only_types, step.actions()); + getRootActions(ast, only_types, step.actions()->actions); } } @@ -1349,7 +1351,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain { for (auto & child : asts) { - auto actions_dag = std::make_shared(columns_after_join); + ActionsDAG actions_dag(columns_after_join); getRootActions(child, only_types, actions_dag); group_by_elements_actions.emplace_back( std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); @@ -1387,7 +1389,7 @@ void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(Expression const ASTFunction & node = typeid_cast(*ast); if (node.arguments) for (auto & argument : node.arguments->children) - getRootActions(argument, only_types, step.actions()); + getRootActions(argument, only_types, step.actions()->actions); } } @@ -1409,7 +1411,7 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments( // recursively together with (1b) as ASTFunction::window_definition. if (getSelectQuery()->window()) { - getRootActionsNoMakeSet(getSelectQuery()->window(), step.actions()); + getRootActionsNoMakeSet(getSelectQuery()->window(), step.actions()->actions); } for (const auto & [_, w] : window_descriptions) @@ -1420,7 +1422,7 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments( // definitions (1a). // Requiring a constant reference to a shared pointer to non-const AST // doesn't really look sane, but the visitor does indeed require it. - getRootActionsNoMakeSet(f.function_node->clone(), step.actions()); + getRootActionsNoMakeSet(f.function_node->clone(), step.actions()->actions); // (2b) Required function argument columns. for (const auto & a : f.function_node->arguments->children) @@ -1442,17 +1444,17 @@ void SelectQueryExpressionAnalyzer::appendExpressionsAfterWindowFunctions(Expres ExpressionActionsChain::Step & step = chain.lastStep(columns_after_window); for (const auto & expression : syntax->expressions_with_window_function) - getRootActionsForWindowFunctions(expression->clone(), true, step.actions()); + getRootActionsForWindowFunctions(expression->clone(), true, step.actions()->actions); } -void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAGPtr & before_aggregation, ExpressionActionsChain & chain, bool /* only_types */) +void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAG & before_aggregation, ExpressionActionsChain & chain, bool /* only_types */) { const auto * select_query = getAggregatingQuery(); if (!select_query->groupBy() || !(select_query->group_by_with_rollup || select_query->group_by_with_cube)) return; - auto source_columns = before_aggregation->getResultColumns(); + auto source_columns = before_aggregation.getResultColumns(); ColumnsWithTypeAndName result_columns; for (const auto & source_column : source_columns) @@ -1462,9 +1464,9 @@ void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAGPtr & befor else result_columns.push_back(source_column); } - ExpressionActionsChain::Step & step = chain.lastStep(before_aggregation->getNamesAndTypesList()); + ExpressionActionsChain::Step & step = chain.lastStep(before_aggregation.getNamesAndTypesList()); - step.actions() = ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position); + step.actions()->actions = std::move(*ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position)); } void SelectQueryExpressionAnalyzer::appendSelectSkipWindowExpressions(ExpressionActionsChain::Step & step, ASTPtr const & node) @@ -1495,7 +1497,7 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); - getRootActionsForHaving(select_query->having(), only_types, step.actions()); + getRootActionsForHaving(select_query->having(), only_types, step.actions()->actions); step.addRequiredOutput(select_query->having()->getColumnName()); @@ -1508,13 +1510,13 @@ void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); - getRootActions(select_query->select(), only_types, step.actions()); + getRootActions(select_query->select(), only_types, step.actions()->actions); for (const auto & child : select_query->select()->children) appendSelectSkipWindowExpressions(step, child); } -ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, +ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions & order_by_elements_actions) { const auto * select_query = getSelectQuery(); @@ -1538,7 +1540,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai replaceForPositionalArguments(ast->children.at(0), select_query, ASTSelectQuery::Expression::ORDER_BY); } - getRootActions(select_query->orderBy(), only_types, step.actions()); + getRootActions(select_query->orderBy(), only_types, step.actions()->actions); bool with_fill = false; @@ -1601,7 +1603,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChai for (const auto & child : select_query->orderBy()->children) { auto actions_dag = std::make_shared(columns_after_join); - getRootActions(child, only_types, actions_dag); + getRootActions(child, only_types, *actions_dag); order_by_elements_actions.emplace_back( std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); } @@ -1628,7 +1630,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); - getRootActions(select_query->limitBy(), only_types, step.actions()); + getRootActions(select_query->limitBy(), only_types, step.actions()->actions); NameSet existing_column_names; for (const auto & column : aggregated_columns) @@ -1657,7 +1659,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain return true; } -ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const +ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const { const auto * select_query = getSelectQuery(); @@ -1706,14 +1708,15 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio } auto actions = chain.getLastActions(); - actions->project(result_columns); + actions->actions.project(result_columns); if (!required_result_columns.empty()) { result_columns.clear(); for (const auto & column : required_result_columns) result_columns.emplace_back(column, std::string{}); - actions->project(result_columns); + actions->actions.project(result_columns); + actions->project_input = actions->projected_output = true; } return actions; @@ -1723,14 +1726,14 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActio void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const ASTPtr & expr, bool only_types) { ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns()); - getRootActions(expr, only_types, step.actions()); + getRootActions(expr, only_types, step.actions()->actions); step.addRequiredOutput(expr->getColumnName()); } - -ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_result) +ActionsAndFlagsPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_result) { - auto actions_dag = std::make_shared(aggregated_columns); + ActionsAndFlagsPtr res; + ActionsDAG actions_dag(aggregated_columns); NamesWithAliases result_columns; Names result_names; @@ -1757,9 +1760,12 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_r if (add_aliases) { if (project_result) - actions_dag->project(result_columns); + { + actions_dag.project(result_columns); + res->project_input = res->projected_output = true; + } else - actions_dag->addAliases(result_columns); + actions_dag.addAliases(result_columns); } if (!(add_aliases && project_result)) @@ -1775,10 +1781,11 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_r } } - actions_dag->removeUnusedActions(name_set); + actions_dag.removeUnusedActions(name_set); } - return actions_dag; + res->actions = std::move(actions_dag); + return res; } ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result, CompileExpressions compile_expressions) @@ -1790,7 +1797,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool proje ActionsDAGPtr ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs) { auto actions = std::make_shared(constant_inputs); - getRootActions(query, true /* no_makeset_for_subqueries */, actions, true /* only_consts */); + getRootActions(query, true /* no_makeset_for_subqueries */, *actions, true /* only_consts */); return actions; } @@ -1805,7 +1812,7 @@ std::unique_ptr SelectQueryExpressionAnalyzer::getJoinedPlan() return std::move(joined_plan); } -ActionsDAGPtr SelectQueryExpressionAnalyzer::simpleSelectActions() +ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::simpleSelectActions() { ExpressionActionsChain new_chain(getContext()); appendSelect(new_chain, false); @@ -1950,7 +1957,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( { where_step_num = chain.steps.size() - 1; before_where = chain.getLastActions(); - if (allowEarlyConstantFolding(*before_where, settings)) + if (allowEarlyConstantFolding(before_where->actions, settings)) { Block before_where_sample; if (chain.steps.size() > 1) @@ -1960,7 +1967,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (sanitizeBlock(before_where_sample)) { ExpressionActions( - before_where, + std::make_shared(before_where->actions.clone()), ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample); auto & column_elem @@ -1986,7 +1993,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( before_aggregation = chain.getLastActions(); if (settings.group_by_use_nulls) - query_analyzer.appendGroupByModifiers(before_aggregation, chain, only_types); + query_analyzer.appendGroupByModifiers(before_aggregation->actions, chain, only_types); auto columns_before_aggregation = finalize_chain(chain); @@ -2033,8 +2040,8 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( true); auto & step = chain.lastStep(query_analyzer.aggregated_columns); - auto & actions = step.actions(); - actions = ActionsDAG::merge(std::move(*actions), std::move(*converting)); + auto & actions = step.actions()->actions; + actions = std::move(*ActionsDAG::merge(std::move(actions), std::move(*converting))); } } @@ -2070,13 +2077,13 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( // the main SELECT, similar to what we do for aggregate functions. if (has_window) { - query_analyzer.makeWindowDescriptions(chain.getLastActions()); + query_analyzer.makeWindowDescriptions(chain.getLastActions()->actions); query_analyzer.appendWindowFunctionsArguments(chain, only_types || !first_stage); // Build a list of output columns of the window step. // 1) We need the columns that are the output of ExpressionActions. - for (const auto & x : chain.getLastActions()->getNamesAndTypesList()) + for (const auto & x : chain.getLastActions()->actions.getNamesAndTypesList()) { query_analyzer.columns_after_window.push_back(x); } @@ -2113,7 +2120,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( finalize_chain(chain); query_analyzer.appendExpressionsAfterWindowFunctions(chain, only_types || !first_stage); - for (const auto & x : chain.getLastActions()->getNamesAndTypesList()) + for (const auto & x : chain.getLastActions()->actions.getNamesAndTypesList()) { query_analyzer.columns_after_window.push_back(x); } @@ -2173,7 +2180,6 @@ void ExpressionAnalysisResult::finalize( if (prewhere_step_num >= 0) { const ExpressionActionsChain::Step & step = *chain.steps.at(prewhere_step_num); - prewhere_info->prewhere_actions->projectInput(false); NameSet columns_to_remove; for (const auto & [name, can_remove] : step.required_output) @@ -2206,9 +2212,9 @@ void ExpressionAnalysisResult::finalize( void ExpressionAnalysisResult::removeExtraColumns() const { if (hasWhere()) - before_where->projectInput(); + before_where->project_input = true; if (hasHaving()) - before_having->projectInput(); + before_having->project_input = true; } void ExpressionAnalysisResult::checkActions() const @@ -2238,7 +2244,7 @@ std::string ExpressionAnalysisResult::dump() const if (before_array_join) { - ss << "before_array_join " << before_array_join->dumpDAG() << "\n"; + ss << "before_array_join " << before_array_join->actions.dumpDAG() << "\n"; } if (array_join) @@ -2248,12 +2254,12 @@ std::string ExpressionAnalysisResult::dump() const if (before_join) { - ss << "before_join " << before_join->dumpDAG() << "\n"; + ss << "before_join " << before_join->actions.dumpDAG() << "\n"; } if (before_where) { - ss << "before_where " << before_where->dumpDAG() << "\n"; + ss << "before_where " << before_where->actions.dumpDAG() << "\n"; } if (prewhere_info) @@ -2268,32 +2274,32 @@ std::string ExpressionAnalysisResult::dump() const if (before_aggregation) { - ss << "before_aggregation " << before_aggregation->dumpDAG() << "\n"; + ss << "before_aggregation " << before_aggregation->actions.dumpDAG() << "\n"; } if (before_having) { - ss << "before_having " << before_having->dumpDAG() << "\n"; + ss << "before_having " << before_having->actions.dumpDAG() << "\n"; } if (before_window) { - ss << "before_window " << before_window->dumpDAG() << "\n"; + ss << "before_window " << before_window->actions.dumpDAG() << "\n"; } if (before_order_by) { - ss << "before_order_by " << before_order_by->dumpDAG() << "\n"; + ss << "before_order_by " << before_order_by->actions.dumpDAG() << "\n"; } if (before_limit_by) { - ss << "before_limit_by " << before_limit_by->dumpDAG() << "\n"; + ss << "before_limit_by " << before_limit_by->actions.dumpDAG() << "\n"; } if (final_projection) { - ss << "final_projection " << final_projection->dumpDAG() << "\n"; + ss << "final_projection " << final_projection->actions.dumpDAG() << "\n"; } if (!selected_columns.empty()) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 941194e69ff0..43608cab4121 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -117,7 +117,7 @@ class ExpressionAnalyzer : protected ExpressionAnalyzerData, private boost::nonc /// If add_aliases, only the calculated values in the desired order and add aliases. /// If also project_result, than only aliases remain in the output block. /// Otherwise, only temporary columns will be deleted from the block. - ActionsDAGPtr getActionsDAG(bool add_aliases, bool project_result = true); + ActionsAndFlagsPtr getActionsDAG(bool add_aliases, bool project_result); ExpressionActionsPtr getActions(bool add_aliases, bool project_result = true, CompileExpressions compile_expressions = CompileExpressions::no); /// Get actions to evaluate a constant expression. The function adds constants and applies functions that depend only on constants. @@ -139,7 +139,7 @@ class ExpressionAnalyzer : protected ExpressionAnalyzerData, private boost::nonc const WindowDescriptions & windowDescriptions() const { return window_descriptions; } void makeWindowDescriptionFromAST(const Context & context, const WindowDescriptions & existing_descriptions, WindowDescription & desc, const IAST * ast); - void makeWindowDescriptions(ActionsDAGPtr actions); + void makeWindowDescriptions(ActionsDAG & actions); /** Checks if subquery is not a plain StorageSet. * Because while making set we will read data from StorageSet which is not allowed. @@ -172,34 +172,34 @@ class ExpressionAnalyzer : protected ExpressionAnalyzerData, private boost::nonc /// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables. void initGlobalSubqueriesAndExternalTables(bool do_global, bool is_explain); - ArrayJoinActionPtr addMultipleArrayJoinAction(ActionsDAGPtr & actions, bool is_left) const; + ArrayJoinActionPtr addMultipleArrayJoinAction(ActionsDAG & actions, bool is_left) const; - void getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts = false); + void getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAG & actions, bool only_consts = false); /** Similar to getRootActions but do not make sets when analyzing IN functions. It's used in * analyzeAggregation which happens earlier than analyzing PREWHERE and WHERE. If we did, the * prepared sets would not be applicable for MergeTree index optimization. */ - void getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAGPtr & actions, bool only_consts = false); + void getRootActionsNoMakeSet(const ASTPtr & ast, ActionsDAG & actions, bool only_consts = false); - void getRootActionsForHaving(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts = false); + void getRootActionsForHaving(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAG & actions, bool only_consts = false); - void getRootActionsForWindowFunctions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions); + void getRootActionsForWindowFunctions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAG & actions); /** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions, * Create a set of columns aggregated_columns resulting after the aggregation, if any, * or after all the actions that are normally performed before aggregation. * Set has_aggregation = true if there is GROUP BY or at least one aggregate function. */ - void analyzeAggregation(ActionsDAGPtr & temp_actions); - void makeAggregateDescriptions(ActionsDAGPtr & actions, AggregateDescriptions & descriptions); + void analyzeAggregation(ActionsDAG & temp_actions); + void makeAggregateDescriptions(ActionsDAG & actions, AggregateDescriptions & descriptions); const ASTSelectQuery * getSelectQuery() const; bool isRemoteStorage() const; - NamesAndTypesList getColumnsAfterArrayJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns); - NamesAndTypesList analyzeJoin(ActionsDAGPtr & actions, const NamesAndTypesList & src_columns); + NamesAndTypesList getColumnsAfterArrayJoin(ActionsDAG & actions, const NamesAndTypesList & src_columns); + NamesAndTypesList analyzeJoin(ActionsDAG & actions, const NamesAndTypesList & src_columns); AggregationKeysInfo getAggregationKeysInfo() const noexcept { @@ -231,20 +231,20 @@ struct ExpressionAnalysisResult bool use_grouping_set_key = false; - ActionsDAGPtr before_array_join; + ActionsAndFlagsPtr before_array_join; ArrayJoinActionPtr array_join; - ActionsDAGPtr before_join; - ActionsDAGPtr converting_join_columns; + ActionsAndFlagsPtr before_join; + ActionsAndFlagsPtr converting_join_columns; JoinPtr join; - ActionsDAGPtr before_where; - ActionsDAGPtr before_aggregation; - ActionsDAGPtr before_having; + ActionsAndFlagsPtr before_where; + ActionsAndFlagsPtr before_aggregation; + ActionsAndFlagsPtr before_having; String having_column_name; bool remove_having_filter = false; - ActionsDAGPtr before_window; - ActionsDAGPtr before_order_by; - ActionsDAGPtr before_limit_by; - ActionsDAGPtr final_projection; + ActionsAndFlagsPtr before_window; + ActionsAndFlagsPtr before_order_by; + ActionsAndFlagsPtr before_limit_by; + ActionsAndFlagsPtr final_projection; /// Columns from the SELECT list, before renaming them to aliases. Used to /// perform SELECT DISTINCT. @@ -351,12 +351,12 @@ class SelectQueryExpressionAnalyzer : public ExpressionAnalyzer /// Tables that will need to be sent to remote servers for distributed query processing. const TemporaryTablesMapping & getExternalTables() const { return external_tables; } - ActionsDAGPtr simpleSelectActions(); + ActionsAndFlagsPtr simpleSelectActions(); /// These appends are public only for tests void appendSelect(ExpressionActionsChain & chain, bool only_types); /// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases. - ActionsDAGPtr appendProjectResult(ExpressionActionsChain & chain) const; + ActionsAndFlagsPtr appendProjectResult(ExpressionActionsChain & chain) const; private: StorageMetadataPtr metadata_snapshot; @@ -386,13 +386,13 @@ class SelectQueryExpressionAnalyzer : public ExpressionAnalyzer */ /// Before aggregation: - ArrayJoinActionPtr appendArrayJoin(ExpressionActionsChain & chain, ActionsDAGPtr & before_array_join, bool only_types); + ArrayJoinActionPtr appendArrayJoin(ExpressionActionsChain & chain, ActionsAndFlagsPtr & before_array_join, bool only_types); bool appendJoinLeftKeys(ExpressionActionsChain & chain, bool only_types); - JoinPtr appendJoin(ExpressionActionsChain & chain, ActionsDAGPtr & converting_join_columns); + JoinPtr appendJoin(ExpressionActionsChain & chain, ActionsAndFlagsPtr & converting_join_columns); /// remove_filter is set in ExpressionActionsChain::finalize(); /// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier). - ActionsDAGPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types); + ActionsAndFlagsPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types); bool appendWhere(ExpressionActionsChain & chain, bool only_types); bool appendGroupBy(ExpressionActionsChain & chain, bool only_types, bool optimize_aggregation_in_order, ManyExpressionActions &); void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types); @@ -401,12 +401,12 @@ class SelectQueryExpressionAnalyzer : public ExpressionAnalyzer void appendExpressionsAfterWindowFunctions(ExpressionActionsChain & chain, bool only_types); void appendSelectSkipWindowExpressions(ExpressionActionsChain::Step & step, ASTPtr const & node); - void appendGroupByModifiers(ActionsDAGPtr & before_aggregation, ExpressionActionsChain & chain, bool only_types); + void appendGroupByModifiers(ActionsDAG & before_aggregation, ExpressionActionsChain & chain, bool only_types); /// After aggregation: bool appendHaving(ExpressionActionsChain & chain, bool only_types); /// appendSelect - ActionsDAGPtr appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions &); + ActionsAndFlagsPtr appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions &); bool appendLimitBy(ExpressionActionsChain & chain, bool only_types); /// appendProjectResult }; diff --git a/src/Interpreters/WindowDescription.h b/src/Interpreters/WindowDescription.h index c26e4517c9a3..15004189f77f 100644 --- a/src/Interpreters/WindowDescription.h +++ b/src/Interpreters/WindowDescription.h @@ -93,8 +93,8 @@ struct WindowDescription // then by ORDER BY. This field holds this combined sort order. SortDescription full_sort_description; - std::vector partition_by_actions; - std::vector order_by_actions; + std::vector partition_by_actions; + std::vector order_by_actions; WindowFrame frame; From a54d6793578f0dbe14e2d33b5d64fcc931d1c0b1 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 11 Jun 2024 18:46:09 +0000 Subject: [PATCH 121/254] roll back some changes --- src/Interpreters/Squashing.cpp | 10 ++++------ src/Interpreters/Squashing.h | 3 ++- src/Processors/Transforms/ApplySquashingTransform.h | 4 ++-- src/Processors/Transforms/PlanSquashingTransform.cpp | 2 +- src/Processors/Transforms/SquashingTransform.cpp | 4 ++-- src/Server/TCPHandler.cpp | 8 +++----- src/Storages/MergeTree/MutateTask.cpp | 7 +++---- 7 files changed, 17 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 7964379a35d9..f8b6a6542ccd 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -10,8 +10,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -Squashing::Squashing(size_t min_block_size_rows_, size_t min_block_size_bytes_) - : min_block_size_rows(min_block_size_rows_) +Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) + : header(header_) + , min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) { } @@ -86,15 +87,12 @@ Chunk Squashing::convertToChunk(std::vector && chunks) const if (chunks.empty()) return {}; - auto chunk = Chunk(chunks.back().cloneEmptyColumns(), 0); - auto info = std::make_shared(); info->chunks = std::move(chunks); chunks.clear(); - chunk.setChunkInfo(info); - return chunk; + return Chunk(header.cloneEmptyColumns(), 0, info); } Chunk Squashing::squash(std::vector & input_chunks) diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index ea991d6dc852..d76cca60e41e 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -29,7 +29,7 @@ struct ChunksToSquash : public ChunkInfo class Squashing { public: - explicit Squashing(size_t min_block_size_rows_, size_t min_block_size_bytes_); + explicit Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); Squashing(Squashing && other) = default; Chunk add(Chunk && input_chunk); @@ -41,6 +41,7 @@ class Squashing return !chunks_to_merge_vec.empty(); } + Block header; private: struct CurrentSize { diff --git a/src/Processors/Transforms/ApplySquashingTransform.h b/src/Processors/Transforms/ApplySquashingTransform.h index 2de614b8dbd5..965a084bb135 100644 --- a/src/Processors/Transforms/ApplySquashingTransform.h +++ b/src/Processors/Transforms/ApplySquashingTransform.h @@ -11,7 +11,7 @@ class ApplySquashingTransform : public ExceptionKeepingTransform public: explicit ApplySquashingTransform(const Block & header, const size_t min_block_size_rows, const size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(min_block_size_rows, min_block_size_bytes) + , squashing(header, min_block_size_rows, min_block_size_bytes) { } @@ -50,7 +50,7 @@ class ApplySquashingTransform : public ExceptionKeepingTransform } void onFinish() override { - auto chunk = Chunk(); + auto chunk = DB::Squashing::squash({}); finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); } diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 71e4ee15eccc..5ae605cc51a1 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes } PlanSquashingTransform::PlanSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) - : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), squashing(min_block_size_rows, min_block_size_bytes) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), squashing(header, min_block_size_rows, min_block_size_bytes) { } diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 34c5b1a7202b..34b733cde5ed 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -12,7 +12,7 @@ extern const int LOGICAL_ERROR; SquashingTransform::SquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(min_block_size_rows, min_block_size_bytes) + , squashing(header, min_block_size_rows, min_block_size_bytes) { } @@ -59,7 +59,7 @@ void SquashingTransform::work() SimpleSquashingTransform::SimpleSquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ISimpleTransform(header, header, false) - , squashing(min_block_size_rows, min_block_size_bytes) + , squashing(header, min_block_size_rows, min_block_size_bytes) { } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 940fa94e182c..f9b24139d86d 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -886,12 +885,11 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro using PushResult = AsynchronousInsertQueue::PushResult; startInsertQuery(); - Squashing squashing(0, query_context->getSettingsRef().async_insert_max_data_size); - Block header; + Squashing squashing(state.input_header, 0, query_context->getSettingsRef().async_insert_max_data_size); while (readDataNext()) { - header = state.block_for_insert.cloneWithoutColumns(); + squashing.header = state.block_for_insert; auto planned_chunk = squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); if (planned_chunk.hasChunkInfo()) { @@ -910,7 +908,7 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro if (planned_chunk.hasChunkInfo()) result_chunk = DB::Squashing::squash(std::move(planned_chunk)); - auto result = header.cloneWithColumns(result_chunk.getColumns()); + auto result = squashing.header.cloneWithColumns(result_chunk.getColumns()); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 6fcc8f4ad924..4ea0e9ece4f6 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1287,7 +1287,7 @@ void PartMergerWriter::prepare() for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { // We split the materialization into multiple stages similar to the process of INSERT SELECT query. - projection_squashes.emplace_back(settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + projection_squashes.emplace_back(ctx->updated_header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); } existing_rows_count = 0; @@ -1297,7 +1297,6 @@ void PartMergerWriter::prepare() bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Block cur_block; - Block header; if (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor->pull(cur_block)) { if (ctx->minmax_idx) @@ -1315,7 +1314,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); Block block_to_squash = projection.calculate(cur_block, ctx->context); - header = block_to_squash.cloneWithoutColumns(); + projection_squashes[i].header = block_to_squash; Chunk planned_chunk = projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); if (planned_chunk.hasChunkInfo()) @@ -1348,7 +1347,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Chunk projection_chunk = DB::Squashing::squash(std::move(planned_chunk)); - auto result = header.cloneWithColumns(projection_chunk.getColumns()); + auto result = projection_squash_plan.header.cloneWithColumns(projection_chunk.getColumns()); auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); temp_part.finalize(); From 66a714e3ee2488883a800692b4d02e89b272b668 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 11 Jun 2024 22:45:02 +0000 Subject: [PATCH 122/254] Revert "roll back some changes" This reverts commit a54d6793578f0dbe14e2d33b5d64fcc931d1c0b1. --- src/Interpreters/Squashing.cpp | 10 ++++++---- src/Interpreters/Squashing.h | 3 +-- src/Processors/Transforms/ApplySquashingTransform.h | 4 ++-- src/Processors/Transforms/PlanSquashingTransform.cpp | 2 +- src/Processors/Transforms/SquashingTransform.cpp | 4 ++-- src/Server/TCPHandler.cpp | 8 +++++--- src/Storages/MergeTree/MutateTask.cpp | 7 ++++--- 7 files changed, 21 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index f8b6a6542ccd..7964379a35d9 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -10,9 +10,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) - : header(header_) - , min_block_size_rows(min_block_size_rows_) +Squashing::Squashing(size_t min_block_size_rows_, size_t min_block_size_bytes_) + : min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) { } @@ -87,12 +86,15 @@ Chunk Squashing::convertToChunk(std::vector && chunks) const if (chunks.empty()) return {}; + auto chunk = Chunk(chunks.back().cloneEmptyColumns(), 0); + auto info = std::make_shared(); info->chunks = std::move(chunks); chunks.clear(); - return Chunk(header.cloneEmptyColumns(), 0, info); + chunk.setChunkInfo(info); + return chunk; } Chunk Squashing::squash(std::vector & input_chunks) diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index d76cca60e41e..ea991d6dc852 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -29,7 +29,7 @@ struct ChunksToSquash : public ChunkInfo class Squashing { public: - explicit Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); + explicit Squashing(size_t min_block_size_rows_, size_t min_block_size_bytes_); Squashing(Squashing && other) = default; Chunk add(Chunk && input_chunk); @@ -41,7 +41,6 @@ class Squashing return !chunks_to_merge_vec.empty(); } - Block header; private: struct CurrentSize { diff --git a/src/Processors/Transforms/ApplySquashingTransform.h b/src/Processors/Transforms/ApplySquashingTransform.h index 965a084bb135..2de614b8dbd5 100644 --- a/src/Processors/Transforms/ApplySquashingTransform.h +++ b/src/Processors/Transforms/ApplySquashingTransform.h @@ -11,7 +11,7 @@ class ApplySquashingTransform : public ExceptionKeepingTransform public: explicit ApplySquashingTransform(const Block & header, const size_t min_block_size_rows, const size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(header, min_block_size_rows, min_block_size_bytes) + , squashing(min_block_size_rows, min_block_size_bytes) { } @@ -50,7 +50,7 @@ class ApplySquashingTransform : public ExceptionKeepingTransform } void onFinish() override { - auto chunk = DB::Squashing::squash({}); + auto chunk = Chunk(); finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); } diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 5ae605cc51a1..71e4ee15eccc 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes } PlanSquashingTransform::PlanSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) - : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), squashing(header, min_block_size_rows, min_block_size_bytes) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), squashing(min_block_size_rows, min_block_size_bytes) { } diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 34b733cde5ed..34c5b1a7202b 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -12,7 +12,7 @@ extern const int LOGICAL_ERROR; SquashingTransform::SquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(header, min_block_size_rows, min_block_size_bytes) + , squashing(min_block_size_rows, min_block_size_bytes) { } @@ -59,7 +59,7 @@ void SquashingTransform::work() SimpleSquashingTransform::SimpleSquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ISimpleTransform(header, header, false) - , squashing(header, min_block_size_rows, min_block_size_bytes) + , squashing(min_block_size_rows, min_block_size_bytes) { } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index f9b24139d86d..940fa94e182c 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -885,11 +886,12 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro using PushResult = AsynchronousInsertQueue::PushResult; startInsertQuery(); - Squashing squashing(state.input_header, 0, query_context->getSettingsRef().async_insert_max_data_size); + Squashing squashing(0, query_context->getSettingsRef().async_insert_max_data_size); + Block header; while (readDataNext()) { - squashing.header = state.block_for_insert; + header = state.block_for_insert.cloneWithoutColumns(); auto planned_chunk = squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); if (planned_chunk.hasChunkInfo()) { @@ -908,7 +910,7 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro if (planned_chunk.hasChunkInfo()) result_chunk = DB::Squashing::squash(std::move(planned_chunk)); - auto result = squashing.header.cloneWithColumns(result_chunk.getColumns()); + auto result = header.cloneWithColumns(result_chunk.getColumns()); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 4ea0e9ece4f6..6fcc8f4ad924 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1287,7 +1287,7 @@ void PartMergerWriter::prepare() for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { // We split the materialization into multiple stages similar to the process of INSERT SELECT query. - projection_squashes.emplace_back(ctx->updated_header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + projection_squashes.emplace_back(settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); } existing_rows_count = 0; @@ -1297,6 +1297,7 @@ void PartMergerWriter::prepare() bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Block cur_block; + Block header; if (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor->pull(cur_block)) { if (ctx->minmax_idx) @@ -1314,7 +1315,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); Block block_to_squash = projection.calculate(cur_block, ctx->context); - projection_squashes[i].header = block_to_squash; + header = block_to_squash.cloneWithoutColumns(); Chunk planned_chunk = projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); if (planned_chunk.hasChunkInfo()) @@ -1347,7 +1348,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Chunk projection_chunk = DB::Squashing::squash(std::move(planned_chunk)); - auto result = projection_squash_plan.header.cloneWithColumns(projection_chunk.getColumns()); + auto result = header.cloneWithColumns(projection_chunk.getColumns()); auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); temp_part.finalize(); From 9923d38227e81216aef3c4323dd68becdf0ff0d3 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 11 Jun 2024 22:45:48 +0000 Subject: [PATCH 123/254] revert changes for empty chunk --- src/Processors/Chunk.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 3c0952f7aedc..4f753798eaa8 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -89,7 +89,7 @@ class Chunk UInt64 getNumColumns() const { return columns.size(); } bool hasRows() const { return num_rows > 0; } bool hasColumns() const { return !columns.empty(); } - bool empty() const { return !hasRows() && !hasColumns() && !hasChunkInfo(); } + bool empty() const { return !hasRows() && !hasColumns(); } explicit operator bool() const { return !empty(); } void addColumn(ColumnPtr column); From 603176ef7f6f4de861d152993dd11e41b6502fd8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Jun 2024 01:22:51 +0200 Subject: [PATCH 124/254] Fix bad error message --- src/IO/S3/PocoHTTPClient.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 1cef43530e07..aab7a39534dd 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -535,7 +535,7 @@ void PocoHTTPClient::makeRequestInternalImpl( const static std::string_view needle = ""; if (auto it = std::search(response_string.begin(), response_string.end(), std::default_searcher(needle.begin(), needle.end())); it != response_string.end()) { - LOG_WARNING(log, "Response for request contain tag in body, settings internal server error (500 code)"); + LOG_WARNING(log, "Response for the request contains an tag in the body, will treat it as an internal server error (code 500)"); response->SetResponseCode(Aws::Http::HttpResponseCode::INTERNAL_SERVER_ERROR); addMetric(request, S3MetricType::Errors); From 4a72b36f287a4588b5e608a49b3ae4c824c8e8de Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 11 Jun 2024 23:26:23 +0000 Subject: [PATCH 125/254] Revert "Revert "roll back some changes"" This reverts commit 66a714e3ee2488883a800692b4d02e89b272b668. --- src/Interpreters/Squashing.cpp | 10 ++++------ src/Interpreters/Squashing.h | 3 ++- src/Processors/Transforms/ApplySquashingTransform.h | 4 ++-- src/Processors/Transforms/PlanSquashingTransform.cpp | 2 +- src/Processors/Transforms/SquashingTransform.cpp | 4 ++-- src/Server/TCPHandler.cpp | 8 +++----- src/Storages/MergeTree/MutateTask.cpp | 7 +++---- 7 files changed, 17 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 7964379a35d9..f8b6a6542ccd 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -10,8 +10,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -Squashing::Squashing(size_t min_block_size_rows_, size_t min_block_size_bytes_) - : min_block_size_rows(min_block_size_rows_) +Squashing::Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_) + : header(header_) + , min_block_size_rows(min_block_size_rows_) , min_block_size_bytes(min_block_size_bytes_) { } @@ -86,15 +87,12 @@ Chunk Squashing::convertToChunk(std::vector && chunks) const if (chunks.empty()) return {}; - auto chunk = Chunk(chunks.back().cloneEmptyColumns(), 0); - auto info = std::make_shared(); info->chunks = std::move(chunks); chunks.clear(); - chunk.setChunkInfo(info); - return chunk; + return Chunk(header.cloneEmptyColumns(), 0, info); } Chunk Squashing::squash(std::vector & input_chunks) diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index ea991d6dc852..d76cca60e41e 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -29,7 +29,7 @@ struct ChunksToSquash : public ChunkInfo class Squashing { public: - explicit Squashing(size_t min_block_size_rows_, size_t min_block_size_bytes_); + explicit Squashing(Block header_, size_t min_block_size_rows_, size_t min_block_size_bytes_); Squashing(Squashing && other) = default; Chunk add(Chunk && input_chunk); @@ -41,6 +41,7 @@ class Squashing return !chunks_to_merge_vec.empty(); } + Block header; private: struct CurrentSize { diff --git a/src/Processors/Transforms/ApplySquashingTransform.h b/src/Processors/Transforms/ApplySquashingTransform.h index 2de614b8dbd5..965a084bb135 100644 --- a/src/Processors/Transforms/ApplySquashingTransform.h +++ b/src/Processors/Transforms/ApplySquashingTransform.h @@ -11,7 +11,7 @@ class ApplySquashingTransform : public ExceptionKeepingTransform public: explicit ApplySquashingTransform(const Block & header, const size_t min_block_size_rows, const size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(min_block_size_rows, min_block_size_bytes) + , squashing(header, min_block_size_rows, min_block_size_bytes) { } @@ -50,7 +50,7 @@ class ApplySquashingTransform : public ExceptionKeepingTransform } void onFinish() override { - auto chunk = Chunk(); + auto chunk = DB::Squashing::squash({}); finish_chunk.setColumns(chunk.getColumns(), chunk.getNumRows()); } diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 71e4ee15eccc..5ae605cc51a1 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes } PlanSquashingTransform::PlanSquashingTransform(const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes, size_t num_ports) - : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), squashing(min_block_size_rows, min_block_size_bytes) + : IProcessor(InputPorts(num_ports, header), OutputPorts(num_ports, header)), squashing(header, min_block_size_rows, min_block_size_bytes) { } diff --git a/src/Processors/Transforms/SquashingTransform.cpp b/src/Processors/Transforms/SquashingTransform.cpp index 34c5b1a7202b..34b733cde5ed 100644 --- a/src/Processors/Transforms/SquashingTransform.cpp +++ b/src/Processors/Transforms/SquashingTransform.cpp @@ -12,7 +12,7 @@ extern const int LOGICAL_ERROR; SquashingTransform::SquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ExceptionKeepingTransform(header, header, false) - , squashing(min_block_size_rows, min_block_size_bytes) + , squashing(header, min_block_size_rows, min_block_size_bytes) { } @@ -59,7 +59,7 @@ void SquashingTransform::work() SimpleSquashingTransform::SimpleSquashingTransform( const Block & header, size_t min_block_size_rows, size_t min_block_size_bytes) : ISimpleTransform(header, header, false) - , squashing(min_block_size_rows, min_block_size_bytes) + , squashing(header, min_block_size_rows, min_block_size_bytes) { } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 940fa94e182c..f9b24139d86d 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -886,12 +885,11 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro using PushResult = AsynchronousInsertQueue::PushResult; startInsertQuery(); - Squashing squashing(0, query_context->getSettingsRef().async_insert_max_data_size); - Block header; + Squashing squashing(state.input_header, 0, query_context->getSettingsRef().async_insert_max_data_size); while (readDataNext()) { - header = state.block_for_insert.cloneWithoutColumns(); + squashing.header = state.block_for_insert; auto planned_chunk = squashing.add({state.block_for_insert.getColumns(), state.block_for_insert.rows()}); if (planned_chunk.hasChunkInfo()) { @@ -910,7 +908,7 @@ AsynchronousInsertQueue::PushResult TCPHandler::processAsyncInsertQuery(Asynchro if (planned_chunk.hasChunkInfo()) result_chunk = DB::Squashing::squash(std::move(planned_chunk)); - auto result = header.cloneWithColumns(result_chunk.getColumns()); + auto result = squashing.header.cloneWithColumns(result_chunk.getColumns()); return insert_queue.pushQueryWithBlock(state.parsed_query, std::move(result), query_context); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 6fcc8f4ad924..4ea0e9ece4f6 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1287,7 +1287,7 @@ void PartMergerWriter::prepare() for (size_t i = 0, size = ctx->projections_to_build.size(); i < size; ++i) { // We split the materialization into multiple stages similar to the process of INSERT SELECT query. - projection_squashes.emplace_back(settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); + projection_squashes.emplace_back(ctx->updated_header, settings.min_insert_block_size_rows, settings.min_insert_block_size_bytes); } existing_rows_count = 0; @@ -1297,7 +1297,6 @@ void PartMergerWriter::prepare() bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Block cur_block; - Block header; if (MutationHelpers::checkOperationIsNotCanceled(*ctx->merges_blocker, ctx->mutate_entry) && ctx->mutating_executor->pull(cur_block)) { if (ctx->minmax_idx) @@ -1315,7 +1314,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() ProfileEventTimeIncrement watch(ProfileEvents::MutateTaskProjectionsCalculationMicroseconds); Block block_to_squash = projection.calculate(cur_block, ctx->context); - header = block_to_squash.cloneWithoutColumns(); + projection_squashes[i].header = block_to_squash; Chunk planned_chunk = projection_squashes[i].add({block_to_squash.getColumns(), block_to_squash.rows()}); if (planned_chunk.hasChunkInfo()) @@ -1348,7 +1347,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() { Chunk projection_chunk = DB::Squashing::squash(std::move(planned_chunk)); - auto result = header.cloneWithColumns(projection_chunk.getColumns()); + auto result = projection_squash_plan.header.cloneWithColumns(projection_chunk.getColumns()); auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, result, projection, ctx->new_data_part.get(), ++block_num); temp_part.finalize(); From 07c628f746b5b007cb552e5c877e4600b63aa0a5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 12 Jun 2024 06:17:48 +0200 Subject: [PATCH 126/254] Fix bad error message --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 4c8f1240cf55..f3a54092a75b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -737,10 +737,10 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks { /// Don't scare people with broken part error if (!isRetryableException(std::current_exception())) - LOG_ERROR(storage.log, "Part {} is broken and need manual correction", getDataPartStorage().getFullPath()); + LOG_ERROR(storage.log, "Part {} is broken and needs manual correction", getDataPartStorage().getFullPath()); // There could be conditions that data part to be loaded is broken, but some of meta infos are already written - // into meta data before exception, need to clean them all. + // into metadata before exception, need to clean them all. metadata_manager->deleteAll(/*include_projection*/ true); metadata_manager->assertAllDeleted(/*include_projection*/ true); throw; From 089f5bfecdf26d18a5e3dda3e3f8f2c4a86deac5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 12 Jun 2024 15:09:38 +0200 Subject: [PATCH 127/254] Update StorageMerge.cpp --- src/Storages/StorageMerge.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 4f8ecf6a8137..cae819fa9823 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1198,6 +1198,8 @@ ReadFromMerge::ChildPlan ReadFromMerge::createPlanForTable( if (allow_experimental_analyzer) { + /// Converting query to AST because types might be different in the source table. + /// Need to resolve types again. InterpreterSelectQueryAnalyzer interpreter(modified_query_info.query_tree->toAST(), modified_context, SelectQueryOptions(processed_stage)); From 6a670645b2213cecee1e17e851463591dff3556f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 12 Jun 2024 15:48:33 +0200 Subject: [PATCH 128/254] Update 02156_storage_merge_prewhere.reference --- tests/queries/0_stateless/02156_storage_merge_prewhere.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference index e36d548e0b80..876cee60baa0 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere.reference @@ -4,7 +4,6 @@ Prewhere info Prewhere filter Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) - Filter column: and(equals(k, 3), notEmpty(v)) (removed) Prewhere info Prewhere filter Prewhere filter column: and(notEmpty(v), equals(k, 3)) (removed) From ae42f8635cbf4d18c10272a5c3e9ff74df60eb0e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 12 Jun 2024 16:53:32 +0000 Subject: [PATCH 129/254] Remove flags drom ActionsDAG (part 2). --- src/Interpreters/ActionsDAG.cpp | 51 +++++----- src/Interpreters/ActionsDAG.h | 5 +- src/Interpreters/ActionsVisitor.cpp | 9 +- src/Interpreters/ActionsVisitor.h | 4 + src/Interpreters/ExpressionActions.cpp | 9 +- src/Interpreters/ExpressionActions.h | 4 +- src/Interpreters/ExpressionAnalyzer.cpp | 39 ++++---- src/Interpreters/ExpressionAnalyzer.h | 6 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 + src/Interpreters/InterpreterSelectQuery.cpp | 93 +++++++++---------- src/Interpreters/InterpreterSelectQuery.h | 12 +-- src/Interpreters/MutationsInterpreter.cpp | 20 ++-- src/Interpreters/WindowDescription.h | 4 +- src/Planner/ActionsChain.cpp | 20 ++-- src/Planner/ActionsChain.h | 8 +- src/Planner/CollectTableExpressionData.cpp | 4 +- src/Planner/Planner.cpp | 34 +++++-- src/Planner/PlannerActionsVisitor.cpp | 11 ++- src/Planner/PlannerActionsVisitor.h | 2 +- src/Planner/PlannerExpressionAnalysis.cpp | 80 +++++++++------- src/Planner/PlannerExpressionAnalysis.h | 14 +-- src/Planner/PlannerJoinTree.cpp | 13 +-- src/Planner/PlannerJoins.cpp | 4 +- src/Planner/Utils.cpp | 8 +- src/Planner/Utils.h | 2 +- .../Optimizations/optimizeReadInOrder.cpp | 2 - .../optimizeUseAggregateProjection.cpp | 2 +- .../QueryPlan/ReadFromMergeTree.cpp | 7 +- .../Transforms/AddingDefaultsTransform.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/StorageMaterializedView.cpp | 1 - src/Storages/StorageMerge.cpp | 4 +- 32 files changed, 264 insertions(+), 214 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 271f8f7474e2..da53f197fd82 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -786,9 +786,6 @@ Block ActionsDAG::updateHeader(const Block & header) const for (auto & col : result_columns) res.insert(std::move(col)); - if (isInputProjected()) - return res; - res.reserve(header.columns() - pos_to_remove.size()); for (size_t i = 0; i < header.columns(); i++) { @@ -1152,6 +1149,33 @@ void ActionsDAG::project(const NamesWithAliases & projection) removeUnusedActions(); } +void ActionsDAG::appendInputsForUnusedColumns(const Block & sample_block) +{ + std::unordered_map> names_map; + size_t num_columns = sample_block.columns(); + for (size_t pos = 0; pos < num_columns; ++pos) + names_map[sample_block.getByPosition(pos).name].push_back(pos); + + for (const auto * input : inputs) + { + auto & positions = names_map[input->result_name]; + if (positions.empty()) + throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, + "Not found column {} in block {}", input->result_name, sample_block.dumpStructure()); + + positions.pop_front(); + } + + for (const auto & [_, positions] : names_map) + { + for (auto pos : positions) + { + const auto & col = sample_block.getByPosition(pos); + addInput(col.name, col.type); + } + } +} + bool ActionsDAG::tryRestoreColumn(const std::string & column_name) { for (const auto * output_node : outputs) @@ -1225,8 +1249,6 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name) ActionsDAGPtr ActionsDAG::clone() const { auto actions = std::make_shared(); - actions->project_input = project_input; - actions->projected_output = projected_output; std::unordered_map copy_map; @@ -1320,9 +1342,6 @@ std::string ActionsDAG::dumpDAG() const out << ' ' << map[node]; out << '\n'; - out << "Project input: " << project_input << '\n'; - out << "Projected output: " << projected_output << '\n'; - return out.str(); } @@ -1581,10 +1600,6 @@ void ActionsDAG::mergeInplace(ActionsDAG && second) auto it = first_result.find(input_node->result_name); if (it == first_result.end() || it->second.empty()) { - if (first.project_input) - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "Cannot find column {} in ActionsDAG result", input_node->result_name); - first.inputs.push_back(input_node); } else @@ -1620,13 +1635,6 @@ void ActionsDAG::mergeInplace(ActionsDAG && second) } } - /// Update output nodes. - if (second.project_input) - { - first.outputs.swap(second.outputs); - first.project_input = true; - } - else { /// Add not removed result from first actions. for (const auto * output_node : first.outputs) @@ -1642,8 +1650,6 @@ void ActionsDAG::mergeInplace(ActionsDAG && second) } first.nodes.splice(first.nodes.end(), std::move(second.nodes)); - - first.projected_output = second.projected_output; } void ActionsDAG::mergeNodes(ActionsDAG && second, NodeRawConstPtrs * out_outputs) @@ -2039,7 +2045,6 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & } auto res = split(split_nodes); - res.second->project_input = project_input; return res; } @@ -2083,7 +2088,6 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsBySortingDescription(const NameS dumpDAG()); auto res = split(split_nodes); - res.second->project_input = project_input; return res; } @@ -2155,7 +2159,6 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsForFilter(const std::string & co std::unordered_set split_nodes = {node}; auto res = split(split_nodes); - res.second->project_input = project_input; return res; } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index ae2cb4129d55..d251f66a1296 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -166,9 +166,12 @@ class ActionsDAG /// Call addAlias several times. void addAliases(const NamesWithAliases & aliases); - /// Add alias actions and remove unused columns from outputs. Also specify result columns order in outputs. + /// Add alias actions. Also specify result columns order in outputs. void project(const NamesWithAliases & projection); + /// Add input for every column from sample_block which is not mapped to existing input. + void appendInputsForUnusedColumns(const Block & sample_block); + /// If column is not in outputs, try to find it in nodes and insert back into outputs. bool tryRestoreColumn(const std::string & column_name); diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index b41c4509bd36..b292d02ccaa2 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -392,6 +392,9 @@ Block createBlockForSet( } +ScopeStack::Level::Level() = default; +ScopeStack::Level::~Level() = default; +ScopeStack::Level::Level(Level &&) = default; FutureSetPtr makeExplicitSet( const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets) @@ -529,7 +532,9 @@ std::vector ActionsMatcher::Data::getAllColumnNames() const ScopeStack::ScopeStack(ActionsDAG actions_dag, ContextPtr context_) : WithContext(context_) { - auto & level = stack.emplace_back(ScopeStack::Level{std::move(actions_dag), {}, {}}); + ScopeStack::Level tmp; + tmp.actions_dag = std::move(actions_dag); + auto & level = stack.emplace_back(std::move(tmp)); level.index = std::make_unique(level.actions_dag.getOutputs()); for (const auto & node : level.actions_dag.getOutputs()) @@ -1268,7 +1273,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & lambda_dag.removeUnusedActions(Names(1, result_name)); auto lambda_actions = std::make_shared( - std::make_shared(lambda_dag), + std::make_shared(std::move(lambda_dag)), ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes)); DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index a0064637939b..e13bd82be069 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -47,6 +47,10 @@ struct ScopeStack : WithContext ActionsDAG actions_dag; IndexPtr index; NameSet inputs; + + ~Level(); + Level(); + Level(Level &&); }; using Levels = std::vector; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 3ba448a0da0f..d25d5b532263 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -49,8 +49,9 @@ namespace ErrorCodes static std::unordered_set processShortCircuitFunctions(const ActionsDAG & actions_dag, ShortCircuitFunctionEvaluation short_circuit_function_evaluation); -ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_) - : settings(settings_) +ExpressionActions::ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_, bool project_inputs_) + : project_inputs(project_inputs_) + , settings(settings_) { actions_dag = actions_dag_->clone(); @@ -757,6 +758,10 @@ void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run, } } + if (project_inputs) + { + block.clear(); + } if (allow_duplicates_in_input) { /// This case is the same as when the input is projected diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index b8ac10fd80e4..7c6af41c04db 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -79,11 +79,13 @@ class ExpressionActions ColumnNumbers result_positions; Block sample_block; + bool project_inputs = false; + ExpressionActionsSettings settings; public: ExpressionActions() = delete; - explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_ = {}); + explicit ExpressionActions(ActionsDAGPtr actions_dag_, const ExpressionActionsSettings & settings_ = {}, bool project_inputs_ = false); ExpressionActions(const ExpressionActions &) = default; ExpressionActions & operator=(const ExpressionActions &) = default; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 530bace3b1e0..c53cdd0d2ed7 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -658,8 +658,8 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, with_alias->getColumnName(), 1 /* direction */, 1 /* nulls_direction */)); - ActionsDAG actions_dag(aggregated_columns); - getRootActions(column_ast, false, actions_dag); + auto actions_dag = std::make_shared(aggregated_columns); + getRootActions(column_ast, false, *actions_dag); desc.partition_by_actions.push_back(std::move(actions_dag)); } } @@ -679,8 +679,8 @@ void ExpressionAnalyzer::makeWindowDescriptionFromAST(const Context & context_, order_by_element.direction, order_by_element.nulls_direction)); - ActionsDAG actions_dag(aggregated_columns); - getRootActions(column_ast, false, actions_dag); + auto actions_dag = std::make_shared(aggregated_columns); + getRootActions(column_ast, false, *actions_dag); desc.order_by_actions.push_back(std::move(actions_dag)); } } @@ -1068,6 +1068,7 @@ static std::unique_ptr buildJoinedPlan( rename_dag->getOutputs()[pos] = &alias; } } + rename_dag->appendInputsForUnusedColumns(joined_plan->getCurrentDataStream().header); auto rename_step = std::make_unique(joined_plan->getCurrentDataStream(), std::move(rename_dag)); rename_step->setStepDescription("Rename joined columns"); joined_plan->addStep(std::move(rename_step)); @@ -1219,7 +1220,7 @@ ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendPrewhere( } { - ActionsDAG actions; + auto actions = std::make_shared(); auto required_columns = prewhere_actions->actions.getRequiredColumns(); NameSet prewhere_input_names; @@ -1265,7 +1266,7 @@ ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendPrewhere( } } - actions = ActionsDAG(required_columns); + actions->actions = ActionsDAG(required_columns); } else { @@ -1280,7 +1281,7 @@ ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendPrewhere( } } - actions = ActionsDAG(columns); + actions->actions = ActionsDAG(columns); } chain.steps.emplace_back( @@ -1351,8 +1352,8 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain { for (auto & child : asts) { - ActionsDAG actions_dag(columns_after_join); - getRootActions(child, only_types, actions_dag); + auto actions_dag = std::make_shared(columns_after_join); + getRootActions(child, only_types, *actions_dag); group_by_elements_actions.emplace_back( std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(getContext(), CompileExpressions::yes))); } @@ -1730,9 +1731,8 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const step.addRequiredOutput(expr->getColumnName()); } -ActionsAndFlagsPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool project_result) +ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_unused_result) { - ActionsAndFlagsPtr res; ActionsDAG actions_dag(aggregated_columns); NamesWithAliases result_columns; Names result_names; @@ -1759,16 +1759,15 @@ ActionsAndFlagsPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool proj if (add_aliases) { - if (project_result) + if (remove_unused_result) { actions_dag.project(result_columns); - res->project_input = res->projected_output = true; } else actions_dag.addAliases(result_columns); } - if (!(add_aliases && project_result)) + if (!(add_aliases && remove_unused_result)) { NameSet name_set(result_names.begin(), result_names.end()); /// We will not delete the original columns. @@ -1784,14 +1783,13 @@ ActionsAndFlagsPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool proj actions_dag.removeUnusedActions(name_set); } - res->actions = std::move(actions_dag); - return res; + return std::make_unique(std::move(actions_dag)); } -ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result, CompileExpressions compile_expressions) +ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remove_unused_result, CompileExpressions compile_expressions) { return std::make_shared( - getActionsDAG(add_aliases, project_result), ExpressionActionsSettings::fromContext(getContext(), compile_expressions)); + getActionsDAG(add_aliases, remove_unused_result), ExpressionActionsSettings::fromContext(getContext(), compile_expressions), remove_unused_result); } ActionsDAGPtr ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs) @@ -1925,7 +1923,8 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( { /// Prewhere is always the first one. prewhere_step_num = 0; - prewhere_info = std::make_shared(actions, query.prewhere()->getColumnName()); + auto dag = std::make_shared(std::move(actions->actions)); + prewhere_info = std::make_shared(std::move(dag), query.prewhere()->getColumnName()); if (allowEarlyConstantFolding(*prewhere_info->prewhere_actions, settings)) { @@ -1967,7 +1966,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (sanitizeBlock(before_where_sample)) { ExpressionActions( - std::make_shared(before_where->actions.clone()), + before_where->actions.clone(), ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample); auto & column_elem diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 43608cab4121..6fa50bb70c9a 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -115,10 +115,10 @@ class ExpressionAnalyzer : protected ExpressionAnalyzerData, private boost::nonc /// If `ast` is not a SELECT query, just gets all the actions to evaluate the expression. /// If add_aliases, only the calculated values in the desired order and add aliases. - /// If also project_result, than only aliases remain in the output block. + /// If also remove_unused_result, than only aliases remain in the output block. /// Otherwise, only temporary columns will be deleted from the block. - ActionsAndFlagsPtr getActionsDAG(bool add_aliases, bool project_result); - ExpressionActionsPtr getActions(bool add_aliases, bool project_result = true, CompileExpressions compile_expressions = CompileExpressions::no); + ActionsDAGPtr getActionsDAG(bool add_aliases, bool remove_unused_result = true); + ExpressionActionsPtr getActions(bool add_aliases, bool remove_unused_result = true, CompileExpressions compile_expressions = CompileExpressions::no); /// Get actions to evaluate a constant expression. The function adds constants and applies functions that depend only on constants. /// Does not execute subqueries. diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 927bafe4bfbc..91bfa863bd9b 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -386,6 +386,8 @@ Chain InterpreterInsertQuery::buildPreSinkChain( auto adding_missing_defaults_actions = std::make_shared(adding_missing_defaults_dag); + std::cerr << adding_missing_defaults_actions->dumpActions() << std::endl; + /// Actually we don't know structure of input blocks from query/table, /// because some clients break insertion protocol (columns != header) out.addSource(std::make_shared(query_sample_block, adding_missing_defaults_actions)); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index b72399df2c1a..fac3c0637b53 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -124,6 +124,7 @@ namespace ErrorCodes extern const int UNKNOWN_IDENTIFIER; extern const int BAD_ARGUMENTS; extern const int SUPPORT_IS_DISABLED; + extern const int NOT_FOUND_COLUMN_IN_BLOCK; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. @@ -175,11 +176,10 @@ FilterDAGInfoPtr generateFilterActions( /// Using separate expression analyzer to prevent any possible alias injection auto syntax_result = TreeRewriter(context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, storage_snapshot)); SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot, {}, false, {}, prepared_sets); - filter_info->actions = analyzer.simpleSelectActions(); + filter_info->actions = std::make_unique(std::move(analyzer.simpleSelectActions()->actions)); filter_info->column_name = expr_list->children.at(0)->getColumnName(); filter_info->actions->removeUnusedActions(NameSet{filter_info->column_name}); - filter_info->actions->projectInput(false); for (const auto * node : filter_info->actions->getInputs()) filter_info->actions->getOutputs().push_back(node); @@ -1078,15 +1078,15 @@ Block InterpreterSelectQuery::getSampleBlockImpl() // with this code. See // https://github.com/ClickHouse/ClickHouse/issues/19857 for details. if (analysis_result.before_window) - return analysis_result.before_window->getResultColumns(); + return analysis_result.before_window->actions.getResultColumns(); // NOTE: should not handle before_limit_by specially since // WithMergeableState does not process LIMIT BY - return analysis_result.before_order_by->getResultColumns(); + return analysis_result.before_order_by->actions.getResultColumns(); } - Block header = analysis_result.before_aggregation->getResultColumns(); + Block header = analysis_result.before_aggregation->actions.getResultColumns(); Block res; @@ -1124,18 +1124,18 @@ Block InterpreterSelectQuery::getSampleBlockImpl() // It's different from selected_columns, see the comment above for // WithMergeableState stage. if (analysis_result.before_window) - return analysis_result.before_window->getResultColumns(); + return analysis_result.before_window->actions.getResultColumns(); // In case of query on remote shards executed up to // WithMergeableStateAfterAggregation*, they can process LIMIT BY, // since the initiator will not apply LIMIT BY again. if (analysis_result.before_limit_by) - return analysis_result.before_limit_by->getResultColumns(); + return analysis_result.before_limit_by->actions.getResultColumns(); - return analysis_result.before_order_by->getResultColumns(); + return analysis_result.before_order_by->actions.getResultColumns(); } - return analysis_result.final_projection->getResultColumns(); + return analysis_result.final_projection->actions.getResultColumns(); } @@ -1636,12 +1636,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(query_plan.getCurrentDataStream(), expressions.before_array_join); - before_array_join_step->setStepDescription("Before ARRAY JOIN"); - query_plan.addStep(std::move(before_array_join_step)); - } + executeExpression(query_plan, expressions.before_array_join, "Before ARRAY JOIN"); if (expressions.array_join) { @@ -1653,23 +1648,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

( - query_plan.getCurrentDataStream(), - expressions.before_join); - before_join_step->setStepDescription("Before JOIN"); - query_plan.addStep(std::move(before_join_step)); - } + executeExpression(query_plan, expressions.before_join, "Before JOIN"); /// Optional step to convert key columns to common supertype. if (expressions.converting_join_columns) - { - QueryPlanStepPtr convert_join_step = std::make_unique( - query_plan.getCurrentDataStream(), - expressions.converting_join_columns); - convert_join_step->setStepDescription("Convert JOIN columns"); - query_plan.addStep(std::move(convert_join_step)); - } + executeExpression(query_plan, expressions.converting_join_columns, "Convert JOIN columns"); if (expressions.hasJoin()) { @@ -2113,7 +2096,6 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis { /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. analysis.prewhere_info = std::make_shared(analysis.filter_info->actions, analysis.filter_info->column_name); - analysis.prewhere_info->prewhere_actions->projectInput(false); analysis.prewhere_info->remove_prewhere_column = analysis.filter_info->do_remove_column; analysis.prewhere_info->need_filter = true; analysis.filter_info = nullptr; @@ -2124,7 +2106,6 @@ void InterpreterSelectQuery::applyFiltersToPrewhereInAnalysis(ExpressionAnalysis /// Add row level security actions to prewhere. analysis.prewhere_info->row_level_filter = analysis.filter_info->actions; analysis.prewhere_info->row_level_column_name = analysis.filter_info->column_name; - analysis.prewhere_info->row_level_filter->projectInput(false); analysis.filter_info = nullptr; } } @@ -2333,7 +2314,7 @@ std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle } if (analysis_result.hasWhere()) { - filter_nodes.push_back(&analysis_result.before_where->findInOutputs(analysis_result.where_column_name)); + filter_nodes.push_back(&analysis_result.before_where->actions.findInOutputs(analysis_result.where_column_name)); } auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes); @@ -2442,7 +2423,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc auto column = ColumnAggregateFunction::create(func); column->insertFrom(place); - Block header = analysis_result.before_aggregation->getResultColumns(); + Block header = analysis_result.before_aggregation->actions.getResultColumns(); size_t arguments_size = desc.argument_names.size(); DataTypes argument_types(arguments_size); for (size_t j = 0; j < arguments_size; ++j) @@ -2604,7 +2585,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc /// Possible filters: row-security, additional filter, replica filter (before array join), where (after array join) query_info.has_filters_and_no_array_join_before_filter = row_policy_filter || additional_filter_info || parallel_replicas_custom_filter_info - || (analysis_result.hasWhere() && !analysis_result.before_where->hasArrayJoin() && !analysis_result.array_join); + || (analysis_result.hasWhere() && !analysis_result.before_where->actions.hasArrayJoin() && !analysis_result.array_join); storage->read(query_plan, required_columns, storage_snapshot, query_info, context, processing_stage, max_block_size, max_streams); if (context->hasQueryContext() && !options.is_internal) @@ -2646,10 +2627,14 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc } } -void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter) +void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, bool remove_filter) { + auto dag = expression->actions.clone(); + if (expression->project_input) + dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + auto where_step = std::make_unique( - query_plan.getCurrentDataStream(), expression, getSelectQuery().where()->getColumnName(), remove_filter); + query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().where()->getColumnName(), remove_filter); where_step->setStepDescription("WHERE"); query_plan.addStep(std::move(where_step)); @@ -2723,11 +2708,9 @@ static GroupingSetsParamsList getAggregatorGroupingSetsParams(const SelectQueryE return result; } -void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) +void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) { - auto expression_before_aggregation = std::make_unique(query_plan.getCurrentDataStream(), expression); - expression_before_aggregation->setStepDescription("Before GROUP BY"); - query_plan.addStep(std::move(expression_before_aggregation)); + executeExpression(query_plan, expression, "Before GROUP BY"); AggregateDescriptions aggregates = query_analyzer->aggregates(); const Settings & settings = context->getSettingsRef(); @@ -2818,10 +2801,14 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool } -void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter) +void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, bool remove_filter) { + auto dag = expression->actions.clone(); + if (expression->project_input) + dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + auto having_step - = std::make_unique(query_plan.getCurrentDataStream(), expression, getSelectQuery().having()->getColumnName(), remove_filter); + = std::make_unique(query_plan.getCurrentDataStream(), std::move(dag), getSelectQuery().having()->getColumnName(), remove_filter); having_step->setStepDescription("HAVING"); query_plan.addStep(std::move(having_step)); @@ -2829,15 +2816,19 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions void InterpreterSelectQuery::executeTotalsAndHaving( - QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool remove_filter, bool overflow_row, bool final) + QueryPlan & query_plan, bool has_having, const ActionsAndFlagsPtr & expression, bool remove_filter, bool overflow_row, bool final) { + auto dag = expression->actions.clone(); + if (expression->project_input) + dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + const Settings & settings = context->getSettingsRef(); auto totals_having_step = std::make_unique( query_plan.getCurrentDataStream(), query_analyzer->aggregates(), overflow_row, - expression, + std::move(dag), has_having ? getSelectQuery().having()->getColumnName() : "", remove_filter, settings.totals_mode, @@ -2870,12 +2861,16 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific query_plan.addStep(std::move(step)); } -void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description) +void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, const std::string & description) { if (!expression) return; - auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), expression); + auto dag = expression->actions.clone(); + if (expression->project_input) + dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + + auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(dag)); expression_step->setStepDescription(description); query_plan.addStep(std::move(expression_step)); @@ -3045,11 +3040,9 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const st } -void InterpreterSelectQuery::executeProjection(QueryPlan & query_plan, const ActionsDAGPtr & expression) +void InterpreterSelectQuery::executeProjection(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression) { - auto projection_step = std::make_unique(query_plan.getCurrentDataStream(), expression); - projection_step->setStepDescription("Projection"); - query_plan.addStep(std::move(projection_step)); + executeExpression(query_plan, expression, "Projection"); } diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index e89a1e5febf8..c4012180b0f2 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -174,13 +174,13 @@ class InterpreterSelectQuery : public IInterpreterUnionOrSelectQuery /// Different stages of query execution. void executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan); - void executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter); + void executeWhere(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, bool remove_filter); void executeAggregation( - QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); + QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); void executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final, bool has_grouping_sets); - void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsDAGPtr & expression, bool remove_filter, bool overflow_row, bool final); - void executeHaving(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter); - static void executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description); + void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsAndFlagsPtr & expression, bool remove_filter, bool overflow_row, bool final); + void executeHaving(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, bool remove_filter); + static void executeExpression(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, const std::string & description); /// FIXME should go through ActionsDAG to behave as a proper function void executeWindow(QueryPlan & query_plan); void executeOrder(QueryPlan & query_plan, InputOrderInfoPtr sorting_info); @@ -191,7 +191,7 @@ class InterpreterSelectQuery : public IInterpreterUnionOrSelectQuery void executeLimitBy(QueryPlan & query_plan); void executeLimit(QueryPlan & query_plan); void executeOffset(QueryPlan & query_plan); - static void executeProjection(QueryPlan & query_plan, const ActionsDAGPtr & expression); + static void executeProjection(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression); void executeDistinct(QueryPlan & query_plan, bool before_order, Names columns, bool pre_distinct); void executeExtremes(QueryPlan & query_plan); void executeSubqueriesInSetsAndJoins(QueryPlan & query_plan); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index ba33b70b59c9..309ab2691a90 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1137,9 +1137,9 @@ void MutationsInterpreter::prepareMutationStages(std::vector & prepared_s for (const auto & kv : stage.column_to_updated) { auto column_name = kv.second->getColumnName(); - const auto & dag_node = actions->findInOutputs(column_name); - const auto & alias = actions->addAlias(dag_node, kv.first); - actions->addOrReplaceInOutputs(alias); + const auto & dag_node = actions->actions.findInOutputs(column_name); + const auto & alias = actions->actions.addAlias(dag_node, kv.first); + actions->actions.addOrReplaceInOutputs(alias); } } @@ -1202,7 +1202,7 @@ void MutationsInterpreter::Source::read( { ActionsDAG::NodeRawConstPtrs nodes(num_filters); for (size_t i = 0; i < num_filters; ++i) - nodes[i] = &steps[i]->actions()->findInOutputs(names[i]); + nodes[i] = &steps[i]->actions()->actions.findInOutputs(names[i]); filter = ActionsDAG::buildFilterActionsDAG(nodes); } @@ -1273,18 +1273,24 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v for (size_t i = 0; i < stage.expressions_chain.steps.size(); ++i) { const auto & step = stage.expressions_chain.steps[i]; - if (step->actions()->hasArrayJoin()) + if (step->actions()->actions.hasArrayJoin()) throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "arrayJoin is not allowed in mutations"); if (i < stage.filter_column_names.size()) { + auto dag = step->actions()->actions.clone(); + if (step->actions()->project_input) + dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute DELETEs. - plan.addStep(std::make_unique(plan.getCurrentDataStream(), step->actions(), stage.filter_column_names[i], false)); + plan.addStep(std::make_unique(plan.getCurrentDataStream(), dag, stage.filter_column_names[i], false)); } else { + auto dag = step->actions()->actions.clone(); + if (step->actions()->project_input) + dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute UPDATE or final projection. - plan.addStep(std::make_unique(plan.getCurrentDataStream(), step->actions())); + plan.addStep(std::make_unique(plan.getCurrentDataStream(), dag)); } } diff --git a/src/Interpreters/WindowDescription.h b/src/Interpreters/WindowDescription.h index 15004189f77f..c26e4517c9a3 100644 --- a/src/Interpreters/WindowDescription.h +++ b/src/Interpreters/WindowDescription.h @@ -93,8 +93,8 @@ struct WindowDescription // then by ORDER BY. This field holds this combined sort order. SortDescription full_sort_description; - std::vector partition_by_actions; - std::vector order_by_actions; + std::vector partition_by_actions; + std::vector order_by_actions; WindowFrame frame; diff --git a/src/Planner/ActionsChain.cpp b/src/Planner/ActionsChain.cpp index c5438b5d2d45..85d50429bab6 100644 --- a/src/Planner/ActionsChain.cpp +++ b/src/Planner/ActionsChain.cpp @@ -11,7 +11,7 @@ namespace DB { -ActionsChainStep::ActionsChainStep(ActionsDAGPtr actions_, +ActionsChainStep::ActionsChainStep(ActionsAndFlagsPtr actions_, bool use_actions_nodes_as_output_columns_, ColumnsWithTypeAndName additional_output_columns_) : actions(std::move(actions_)) @@ -28,12 +28,12 @@ void ActionsChainStep::finalizeInputAndOutputColumns(const NameSet & child_input auto child_input_columns_copy = child_input_columns; std::unordered_set output_nodes_names; - output_nodes_names.reserve(actions->getOutputs().size()); + output_nodes_names.reserve(actions->actions.getOutputs().size()); - for (auto & output_node : actions->getOutputs()) + for (auto & output_node : actions->actions.getOutputs()) output_nodes_names.insert(output_node->result_name); - for (const auto & node : actions->getNodes()) + for (const auto & node : actions->actions.getNodes()) { auto it = child_input_columns_copy.find(node.result_name); if (it == child_input_columns_copy.end()) @@ -45,20 +45,20 @@ void ActionsChainStep::finalizeInputAndOutputColumns(const NameSet & child_input if (output_nodes_names.contains(node.result_name)) continue; - actions->getOutputs().push_back(&node); + actions->actions.getOutputs().push_back(&node); output_nodes_names.insert(node.result_name); } - actions->removeUnusedActions(); + actions->actions.removeUnusedActions(); /// TODO: Analyzer fix ActionsDAG input and constant nodes with same name - actions->projectInput(); + actions->project_input = true; initialize(); } void ActionsChainStep::dump(WriteBuffer & buffer) const { buffer << "DAG" << '\n'; - buffer << actions->dumpDAG(); + buffer << actions->actions.dumpDAG(); if (!available_output_columns.empty()) { @@ -84,7 +84,7 @@ String ActionsChainStep::dump() const void ActionsChainStep::initialize() { - auto required_columns_names = actions->getRequiredColumnsNames(); + auto required_columns_names = actions->actions.getRequiredColumnsNames(); input_columns_names = NameSet(required_columns_names.begin(), required_columns_names.end()); available_output_columns.clear(); @@ -93,7 +93,7 @@ void ActionsChainStep::initialize() { std::unordered_set available_output_columns_names; - for (const auto & node : actions->getNodes()) + for (const auto & node : actions->actions.getNodes()) { if (available_output_columns_names.contains(node.result_name)) continue; diff --git a/src/Planner/ActionsChain.h b/src/Planner/ActionsChain.h index 4907fdbad87a..ab5823b99d75 100644 --- a/src/Planner/ActionsChain.h +++ b/src/Planner/ActionsChain.h @@ -48,18 +48,18 @@ class ActionsChainStep * If use_actions_nodes_as_output_columns = true output columns are initialized using actions dag nodes. * If additional output columns are specified they are added to output columns. */ - explicit ActionsChainStep(ActionsDAGPtr actions_, + explicit ActionsChainStep(ActionsAndFlagsPtr actions_, bool use_actions_nodes_as_output_columns = true, ColumnsWithTypeAndName additional_output_columns_ = {}); /// Get actions - ActionsDAGPtr & getActions() + ActionsAndFlagsPtr & getActions() { return actions; } /// Get actions - const ActionsDAGPtr & getActions() const + const ActionsAndFlagsPtr & getActions() const { return actions; } @@ -98,7 +98,7 @@ class ActionsChainStep private: void initialize(); - ActionsDAGPtr actions; + ActionsAndFlagsPtr actions; bool use_actions_nodes_as_output_columns = true; diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 27b5909c13b0..d5e39a9f1234 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -90,7 +90,7 @@ class CollectSourceColumnsVisitor : public InDepthQueryTreeVisitor(); PlannerActionsVisitor actions_visitor(planner_context, false); - auto outputs = actions_visitor.visit(alias_column_actions_dag, column_node->getExpression()); + auto outputs = actions_visitor.visit(*alias_column_actions_dag, column_node->getExpression()); if (outputs.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected single output in actions dag for alias column {}. Actual {}", column_node->dumpTree(), outputs.size()); @@ -340,7 +340,7 @@ void collectTableExpressionData(QueryTreeNodePtr & query_node, PlannerContextPtr QueryTreeNodePtr query_tree_node = query_node_typed.getPrewhere(); PlannerActionsVisitor visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); - auto expression_nodes = visitor.visit(prewhere_actions_dag, query_tree_node); + auto expression_nodes = visitor.visit(*prewhere_actions_dag, query_tree_node); if (expression_nodes.size() != 1) throw Exception(ErrorCodes::ILLEGAL_PREWHERE, "Invalid PREWHERE. Expected single boolean expression. In query {}", diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index b40e23a9553a..b117a0cd34bd 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -99,6 +99,7 @@ namespace ErrorCodes extern const int TOO_DEEP_SUBQUERIES; extern const int NOT_IMPLEMENTED; extern const int SUPPORT_IS_DISABLED; + extern const int NOT_FOUND_COLUMN_IN_BLOCK; } namespace @@ -329,12 +330,16 @@ class QueryAnalysisResult }; void addExpressionStep(QueryPlan & query_plan, - const ActionsDAGPtr & expression_actions, + const ActionsAndFlagsPtr & expression_actions, const std::string & step_description, std::vector & result_actions_to_execute) { - result_actions_to_execute.push_back(expression_actions); - auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), expression_actions); + auto actions = expression_actions->actions.clone(); + if (expression_actions->project_input) + actions->appendInputsForUnusedColumns( query_plan.getCurrentDataStream().header); + + result_actions_to_execute.push_back(actions); + auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), actions); expression_step->setStepDescription(step_description); query_plan.addStep(std::move(expression_step)); } @@ -344,9 +349,13 @@ void addFilterStep(QueryPlan & query_plan, const std::string & step_description, std::vector & result_actions_to_execute) { - result_actions_to_execute.push_back(filter_analysis_result.filter_actions); + auto actions = filter_analysis_result.filter_actions->actions.clone(); + if (filter_analysis_result.filter_actions->project_input) + actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + + result_actions_to_execute.push_back(actions); auto where_step = std::make_unique(query_plan.getCurrentDataStream(), - filter_analysis_result.filter_actions, + actions, filter_analysis_result.filter_column_name, filter_analysis_result.remove_filter_column); where_step->setStepDescription(step_description); @@ -545,14 +554,21 @@ void addTotalsHavingStep(QueryPlan & query_plan, const auto & having_analysis_result = expression_analysis_result.getHaving(); bool need_finalize = !query_node.isGroupByWithRollup() && !query_node.isGroupByWithCube(); + ActionsDAGPtr actions; if (having_analysis_result.filter_actions) - result_actions_to_execute.push_back(having_analysis_result.filter_actions); + { + actions = having_analysis_result.filter_actions->actions.clone(); + if (having_analysis_result.filter_actions->project_input) + actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + + result_actions_to_execute.push_back(actions); + } auto totals_having_step = std::make_unique( query_plan.getCurrentDataStream(), aggregation_analysis_result.aggregate_descriptions, query_analysis_result.aggregate_overflow_row, - having_analysis_result.filter_actions, + actions, having_analysis_result.filter_column_name, having_analysis_result.remove_filter_column, settings.totals_mode, @@ -728,12 +744,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, auto & interpolate_node_typed = interpolate_node->as(); PlannerActionsVisitor planner_actions_visitor(planner_context); - auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(interpolate_actions_dag, + auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag, interpolate_node_typed.getExpression()); if (expression_to_interpolate_expression_nodes.size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression to interpolate expected to have single action node"); - auto interpolate_expression_nodes = planner_actions_visitor.visit(interpolate_actions_dag, + auto interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression()); if (interpolate_expression_nodes.size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Interpolate expression expected to have single action node"); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 837307ba2ca1..f5e71acee387 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -886,7 +886,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi for (const auto & argument : function_node.getArguments()) { - auto index_hint_argument_expression_dag_nodes = actions_visitor.visit(index_hint_actions_dag, argument); + auto index_hint_argument_expression_dag_nodes = actions_visitor.visit(*index_hint_actions_dag, argument); for (auto & expression_dag_node : index_hint_argument_expression_dag_nodes) { @@ -1013,10 +1013,13 @@ PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_c , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) {} -ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAGPtr actions_dag, QueryTreeNodePtr expression_node) +ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAG & actions_dag, QueryTreeNodePtr expression_node) { - PlannerActionsVisitorImpl actions_visitor_impl(actions_dag, planner_context, use_column_identifier_as_action_node_name); - return actions_visitor_impl.visit(expression_node); + auto ptr = std::make_shared(std::move(actions_dag)); + PlannerActionsVisitorImpl actions_visitor_impl(ptr, planner_context, use_column_identifier_as_action_node_name); + auto res = actions_visitor_impl.visit(expression_node); + actions_dag = std::move(*ptr); + return res; } String calculateActionNodeName(const QueryTreeNodePtr & node, diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 8506c3091718..6bb320473270 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -37,7 +37,7 @@ class PlannerActionsVisitor * Necessary actions are not added in actions dag output. * Returns query tree expression node actions dag nodes. */ - ActionsDAG::NodeRawConstPtrs visit(ActionsDAGPtr actions_dag, QueryTreeNodePtr expression_node); + ActionsDAG::NodeRawConstPtrs visit(ActionsDAG & actions_dag, QueryTreeNodePtr expression_node); private: const PlannerContextPtr planner_context; diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index f0a2845c3e8d..b2223d9c51d6 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -44,8 +44,9 @@ FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_no { FilterAnalysisResult result; - result.filter_actions = buildActionsDAGFromExpressionNode(filter_expression_node, input_columns, planner_context); - result.filter_column_name = result.filter_actions->getOutputs().at(0)->result_name; + result.filter_actions = std::make_shared(); + result.filter_actions->actions = buildActionsDAGFromExpressionNode(filter_expression_node, input_columns, planner_context); + result.filter_column_name = result.filter_actions->actions.getOutputs().at(0)->result_name; actions_chain.addStep(std::make_unique(result.filter_actions)); return result; @@ -111,8 +112,9 @@ std::optional analyzeAggregation(const QueryTreeNodeP Names aggregation_keys; - ActionsDAGPtr before_aggregation_actions = std::make_shared(input_columns); - before_aggregation_actions->getOutputs().clear(); + ActionsAndFlagsPtr before_aggregation_actions = std::make_shared(); + before_aggregation_actions->actions = ActionsDAG(input_columns); + before_aggregation_actions->actions.getOutputs().clear(); std::unordered_set before_aggregation_actions_output_node_names; @@ -147,7 +149,7 @@ std::optional analyzeAggregation(const QueryTreeNodeP if (constant_key && !aggregates_descriptions.empty() && (!check_constants_for_group_by_key || canRemoveConstantFromGroupByKey(*constant_key))) continue; - auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, grouping_set_key_node); + auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions->actions, grouping_set_key_node); aggregation_keys.reserve(expression_dag_nodes.size()); for (auto & expression_dag_node : expression_dag_nodes) @@ -160,7 +162,7 @@ std::optional analyzeAggregation(const QueryTreeNodeP auto column_after_aggregation = group_by_use_nulls && expression_dag_node->column != nullptr ? makeNullableSafe(expression_dag_node->column) : expression_dag_node->column; available_columns_after_aggregation.emplace_back(std::move(column_after_aggregation), expression_type_after_aggregation, expression_dag_node->result_name); aggregation_keys.push_back(expression_dag_node->result_name); - before_aggregation_actions->getOutputs().push_back(expression_dag_node); + before_aggregation_actions->actions.getOutputs().push_back(expression_dag_node); before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name); } } @@ -199,7 +201,7 @@ std::optional analyzeAggregation(const QueryTreeNodeP if (constant_key && !aggregates_descriptions.empty() && (!check_constants_for_group_by_key || canRemoveConstantFromGroupByKey(*constant_key))) continue; - auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, group_by_key_node); + auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions->actions, group_by_key_node); aggregation_keys.reserve(expression_dag_nodes.size()); for (auto & expression_dag_node : expression_dag_nodes) @@ -211,7 +213,7 @@ std::optional analyzeAggregation(const QueryTreeNodeP auto column_after_aggregation = group_by_use_nulls && expression_dag_node->column != nullptr ? makeNullableSafe(expression_dag_node->column) : expression_dag_node->column; available_columns_after_aggregation.emplace_back(std::move(column_after_aggregation), expression_type_after_aggregation, expression_dag_node->result_name); aggregation_keys.push_back(expression_dag_node->result_name); - before_aggregation_actions->getOutputs().push_back(expression_dag_node); + before_aggregation_actions->actions.getOutputs().push_back(expression_dag_node); before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name); } } @@ -225,13 +227,13 @@ std::optional analyzeAggregation(const QueryTreeNodeP auto & aggregate_function_node_typed = aggregate_function_node->as(); for (const auto & aggregate_function_node_argument : aggregate_function_node_typed.getArguments().getNodes()) { - auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, aggregate_function_node_argument); + auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions->actions, aggregate_function_node_argument); for (auto & expression_dag_node : expression_dag_nodes) { if (before_aggregation_actions_output_node_names.contains(expression_dag_node->result_name)) continue; - before_aggregation_actions->getOutputs().push_back(expression_dag_node); + before_aggregation_actions->actions.getOutputs().push_back(expression_dag_node); before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name); } } @@ -278,8 +280,9 @@ std::optional analyzeWindow(const QueryTreeNodePtr & query PlannerActionsVisitor actions_visitor(planner_context); - ActionsDAGPtr before_window_actions = std::make_shared(input_columns); - before_window_actions->getOutputs().clear(); + ActionsAndFlagsPtr before_window_actions = std::make_shared(); + before_window_actions->actions = ActionsDAG(input_columns); + before_window_actions->actions.getOutputs().clear(); std::unordered_set before_window_actions_output_node_names; @@ -288,25 +291,25 @@ std::optional analyzeWindow(const QueryTreeNodePtr & query auto & window_function_node_typed = window_function_node->as(); auto & window_node = window_function_node_typed.getWindowNode()->as(); - auto expression_dag_nodes = actions_visitor.visit(before_window_actions, window_function_node_typed.getArgumentsNode()); + auto expression_dag_nodes = actions_visitor.visit(before_window_actions->actions, window_function_node_typed.getArgumentsNode()); for (auto & expression_dag_node : expression_dag_nodes) { if (before_window_actions_output_node_names.contains(expression_dag_node->result_name)) continue; - before_window_actions->getOutputs().push_back(expression_dag_node); + before_window_actions->actions.getOutputs().push_back(expression_dag_node); before_window_actions_output_node_names.insert(expression_dag_node->result_name); } - expression_dag_nodes = actions_visitor.visit(before_window_actions, window_node.getPartitionByNode()); + expression_dag_nodes = actions_visitor.visit(before_window_actions->actions, window_node.getPartitionByNode()); for (auto & expression_dag_node : expression_dag_nodes) { if (before_window_actions_output_node_names.contains(expression_dag_node->result_name)) continue; - before_window_actions->getOutputs().push_back(expression_dag_node); + before_window_actions->actions.getOutputs().push_back(expression_dag_node); before_window_actions_output_node_names.insert(expression_dag_node->result_name); } @@ -317,14 +320,14 @@ std::optional analyzeWindow(const QueryTreeNodePtr & query for (auto & sort_node : order_by_node_list.getNodes()) { auto & sort_node_typed = sort_node->as(); - expression_dag_nodes = actions_visitor.visit(before_window_actions, sort_node_typed.getExpression()); + expression_dag_nodes = actions_visitor.visit(before_window_actions->actions, sort_node_typed.getExpression()); for (auto & expression_dag_node : expression_dag_nodes) { if (before_window_actions_output_node_names.contains(expression_dag_node->result_name)) continue; - before_window_actions->getOutputs().push_back(expression_dag_node); + before_window_actions->actions.getOutputs().push_back(expression_dag_node); before_window_actions_output_node_names.insert(expression_dag_node->result_name); } } @@ -357,7 +360,8 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { - auto projection_actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), input_columns, planner_context); + auto projection_actions = std::make_shared(); + projection_actions->actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), input_columns, planner_context); auto projection_columns = query_node.getProjectionColumns(); size_t projection_columns_size = projection_columns.size(); @@ -366,7 +370,7 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, NamesWithAliases projection_column_names_with_display_aliases; projection_column_names_with_display_aliases.reserve(projection_columns_size); - auto & projection_actions_outputs = projection_actions->getOutputs(); + auto & projection_actions_outputs = projection_actions->actions.getOutputs(); size_t projection_outputs_size = projection_actions_outputs.size(); if (projection_columns_size != projection_outputs_size) @@ -404,8 +408,9 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { - ActionsDAGPtr before_sort_actions = std::make_shared(input_columns); - auto & before_sort_actions_outputs = before_sort_actions->getOutputs(); + auto before_sort_actions = std::make_shared(); + before_sort_actions->actions = ActionsDAG(input_columns); + auto & before_sort_actions_outputs = before_sort_actions->actions.getOutputs(); before_sort_actions_outputs.clear(); PlannerActionsVisitor actions_visitor(planner_context); @@ -419,7 +424,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, for (const auto & sort_node : order_by_node_list.getNodes()) { auto & sort_node_typed = sort_node->as(); - auto expression_dag_nodes = actions_visitor.visit(before_sort_actions, sort_node_typed.getExpression()); + auto expression_dag_nodes = actions_visitor.visit(before_sort_actions->actions, sort_node_typed.getExpression()); has_with_fill |= sort_node_typed.withFill(); for (auto & action_dag_node : expression_dag_nodes) @@ -435,7 +440,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, if (has_with_fill) { for (auto & output_node : before_sort_actions_outputs) - output_node = &before_sort_actions->materializeNode(*output_node); + output_node = &before_sort_actions->actions.materializeNode(*output_node); } /// We add only INPUT columns necessary for INTERPOLATE expression in before ORDER BY actions DAG @@ -444,7 +449,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, auto & interpolate_list_node = query_node.getInterpolate()->as(); PlannerActionsVisitor interpolate_actions_visitor(planner_context); - auto interpolate_actions_dag = std::make_shared(); + ActionsDAG interpolate_actions_dag; for (auto & interpolate_node : interpolate_list_node.getNodes()) { @@ -453,10 +458,10 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, } std::unordered_map before_sort_actions_inputs_name_to_node; - for (const auto & node : before_sort_actions->getInputs()) + for (const auto & node : before_sort_actions->actions.getInputs()) before_sort_actions_inputs_name_to_node.emplace(node->result_name, node); - for (const auto & node : interpolate_actions_dag->getNodes()) + for (const auto & node : interpolate_actions_dag.getNodes()) { if (before_sort_actions_dag_output_node_names.contains(node.result_name) || node.type != ActionsDAG::ActionType::INPUT) @@ -466,7 +471,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, if (input_node_it == before_sort_actions_inputs_name_to_node.end()) { auto input_column = ColumnWithTypeAndName{node.column, node.result_type, node.result_name}; - const auto * input_node = &before_sort_actions->addInput(std::move(input_column)); + const auto * input_node = &before_sort_actions->actions.addInput(std::move(input_column)); auto [it, _] = before_sort_actions_inputs_name_to_node.emplace(node.result_name, input_node); input_node_it = it; } @@ -491,22 +496,23 @@ LimitByAnalysisResult analyzeLimitBy(const QueryNode & query_node, const NameSet & required_output_nodes_names, ActionsChain & actions_chain) { - auto before_limit_by_actions = buildActionsDAGFromExpressionNode(query_node.getLimitByNode(), input_columns, planner_context); + auto before_limit_by_actions = std::make_shared(); + before_limit_by_actions->actions = buildActionsDAGFromExpressionNode(query_node.getLimitByNode(), input_columns, planner_context); NameSet limit_by_column_names_set; Names limit_by_column_names; - limit_by_column_names.reserve(before_limit_by_actions->getOutputs().size()); - for (auto & output_node : before_limit_by_actions->getOutputs()) + limit_by_column_names.reserve(before_limit_by_actions->actions.getOutputs().size()); + for (auto & output_node : before_limit_by_actions->actions.getOutputs()) { limit_by_column_names_set.insert(output_node->result_name); limit_by_column_names.push_back(output_node->result_name); } - for (const auto & node : before_limit_by_actions->getNodes()) + for (const auto & node : before_limit_by_actions->actions.getNodes()) { if (required_output_nodes_names.contains(node.result_name) && !limit_by_column_names_set.contains(node.result_name)) - before_limit_by_actions->getOutputs().push_back(&node); + before_limit_by_actions->actions.getOutputs().push_back(&node); } auto actions_step_before_limit_by = std::make_unique(before_limit_by_actions); @@ -591,7 +597,7 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo if (sort_analysis_result_optional.has_value() && planner_query_processing_info.isFirstStage() && planner_query_processing_info.getToStage() != QueryProcessingStage::Complete) { const auto & before_order_by_actions = sort_analysis_result_optional->before_order_by_actions; - for (const auto & output_node : before_order_by_actions->getOutputs()) + for (const auto & output_node : before_order_by_actions->actions.getOutputs()) required_output_nodes_names.insert(output_node->result_name); } @@ -647,8 +653,10 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo } } - auto project_names_actions = std::make_shared(project_names_input); - project_names_actions->project(projection_analysis_result.projection_column_names_with_display_aliases); + auto project_names_actions = std::make_shared(); + project_names_actions->actions = ActionsDAG(project_names_input); + project_names_actions->actions.project(projection_analysis_result.projection_column_names_with_display_aliases); + project_names_actions->project_input = true; actions_chain.addStep(std::make_unique(project_names_actions)); actions_chain.finalize(); diff --git a/src/Planner/PlannerExpressionAnalysis.h b/src/Planner/PlannerExpressionAnalysis.h index 0773272e49a9..3a9ed903bbc4 100644 --- a/src/Planner/PlannerExpressionAnalysis.h +++ b/src/Planner/PlannerExpressionAnalysis.h @@ -17,22 +17,22 @@ namespace DB struct ProjectionAnalysisResult { - ActionsDAGPtr projection_actions; + ActionsAndFlagsPtr projection_actions; Names projection_column_names; NamesWithAliases projection_column_names_with_display_aliases; - ActionsDAGPtr project_names_actions; + ActionsAndFlagsPtr project_names_actions; }; struct FilterAnalysisResult { - ActionsDAGPtr filter_actions; + ActionsAndFlagsPtr filter_actions; std::string filter_column_name; bool remove_filter_column = false; }; struct AggregationAnalysisResult { - ActionsDAGPtr before_aggregation_actions; + ActionsAndFlagsPtr before_aggregation_actions; Names aggregation_keys; AggregateDescriptions aggregate_descriptions; GroupingSetsParamsList grouping_sets_parameters_list; @@ -41,19 +41,19 @@ struct AggregationAnalysisResult struct WindowAnalysisResult { - ActionsDAGPtr before_window_actions; + ActionsAndFlagsPtr before_window_actions; std::vector window_descriptions; }; struct SortAnalysisResult { - ActionsDAGPtr before_order_by_actions; + ActionsAndFlagsPtr before_order_by_actions; bool has_with_fill = false; }; struct LimitByAnalysisResult { - ActionsDAGPtr before_limit_by_actions; + ActionsAndFlagsPtr before_limit_by_actions; Names limit_by_column_names; }; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index efc449402b9a..18c3744f864a 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -79,6 +79,7 @@ namespace ErrorCodes extern const int TOO_MANY_COLUMNS; extern const int UNSUPPORTED_METHOD; extern const int BAD_ARGUMENTS; + extern const int NOT_FOUND_COLUMN_IN_BLOCK; } namespace @@ -1072,7 +1073,7 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP } } - cast_actions_dag->projectInput(); + cast_actions_dag->appendInputsForUnusedColumns( plan_to_add_cast.getCurrentDataStream().header); auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable"); plan_to_add_cast.addStep(std::move(cast_join_columns_step)); @@ -1118,12 +1119,12 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ join_table_expression, planner_context); - join_clauses_and_actions.left_join_expressions_actions->projectInput(); + join_clauses_and_actions.left_join_expressions_actions->appendInputsForUnusedColumns(left_plan.getCurrentDataStream().header); auto left_join_expressions_actions_step = std::make_unique(left_plan.getCurrentDataStream(), join_clauses_and_actions.left_join_expressions_actions); left_join_expressions_actions_step->setStepDescription("JOIN actions"); left_plan.addStep(std::move(left_join_expressions_actions_step)); - join_clauses_and_actions.right_join_expressions_actions->projectInput(); + join_clauses_and_actions.right_join_expressions_actions->appendInputsForUnusedColumns(right_plan.getCurrentDataStream().header); auto right_join_expressions_actions_step = std::make_unique(right_plan.getCurrentDataStream(), join_clauses_and_actions.right_join_expressions_actions); right_join_expressions_actions_step->setStepDescription("JOIN actions"); right_plan.addStep(std::move(right_join_expressions_actions_step)); @@ -1175,7 +1176,7 @@ JoinTreeQueryPlan buildQueryPlanForJoinNode(const QueryTreeNodePtr & join_table_ output_node = &cast_actions_dag->addCast(*output_node, cast_type, output_node->result_name); } - cast_actions_dag->projectInput(); + cast_actions_dag->appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header); auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); cast_join_columns_step->setStepDescription("Cast JOIN USING columns"); @@ -1570,7 +1571,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ array_join_column_names.insert(array_join_column_identifier); auto & array_join_expression_column = array_join_expression->as(); - auto expression_dag_index_nodes = actions_visitor.visit(array_join_action_dag, array_join_expression_column.getExpressionOrThrow()); + auto expression_dag_index_nodes = actions_visitor.visit(*array_join_action_dag, array_join_expression_column.getExpressionOrThrow()); for (auto & expression_dag_index_node : expression_dag_index_nodes) { @@ -1580,7 +1581,7 @@ JoinTreeQueryPlan buildQueryPlanForArrayJoinNode(const QueryTreeNodePtr & array_ } } - array_join_action_dag->projectInput(); + array_join_action_dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); join_tree_query_plan.actions_dags.push_back(array_join_action_dag); diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index c410b04f2097..84efdd213367 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -183,7 +183,7 @@ const ActionsDAG::Node * appendExpression( const JoinNode & join_node) { PlannerActionsVisitor join_expression_visitor(planner_context); - auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(dag, expression); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*dag, expression); if (join_expression_dag_node_raw_pointers.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "JOIN {} ON clause contains multiple expressions", @@ -603,7 +603,7 @@ JoinClausesAndActions buildJoinClausesAndActions( { auto mixed_join_expressions_actions = std::make_shared(mixed_table_expression_columns); PlannerActionsVisitor join_expression_visitor(planner_context); - auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(mixed_join_expressions_actions, join_expression); + auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(*mixed_join_expressions_actions, join_expression); if (join_expression_dag_node_raw_pointers.size() != 1) throw Exception( ErrorCodes::LOGICAL_ERROR, "JOIN {} ON clause contains multiple expressions", join_node.formatASTForErrorMessage()); diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 4a74bf413d3d..18a6d297838f 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -213,14 +213,14 @@ StorageLimits buildStorageLimits(const Context & context, const SelectQueryOptio return {limits, leaf_limits}; } -ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & expression_node, +ActionsDAG buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & expression_node, const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context) { - ActionsDAGPtr action_dag = std::make_shared(input_columns); + ActionsDAG action_dag(input_columns); PlannerActionsVisitor actions_visitor(planner_context); auto expression_dag_index_nodes = actions_visitor.visit(action_dag, expression_node); - action_dag->getOutputs() = std::move(expression_dag_index_nodes); + action_dag.getOutputs() = std::move(expression_dag_index_nodes); return action_dag; } @@ -443,7 +443,7 @@ FilterDAGInfo buildFilterInfo(QueryTreeNodePtr filter_query_tree, auto filter_actions_dag = std::make_shared(); PlannerActionsVisitor actions_visitor(planner_context, false /*use_column_identifier_as_action_node_name*/); - auto expression_nodes = actions_visitor.visit(filter_actions_dag, filter_query_tree); + auto expression_nodes = actions_visitor.visit(*filter_actions_dag, filter_query_tree); if (expression_nodes.size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Filter actions must return single output node. Actual {}", diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 4706f552c9dd..3172847f053c 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -47,7 +47,7 @@ StorageLimits buildStorageLimits(const Context & context, const SelectQueryOptio * Inputs are not used for actions dag outputs. * Only root query tree expression node is used as actions dag output. */ -ActionsDAGPtr buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & expression_node, +ActionsDAG buildActionsDAGFromExpressionNode(const QueryTreeNodePtr & expression_node, const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context); diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index c175cd516ac7..537555afa2a7 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -176,8 +176,6 @@ static void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expressi dag->mergeInplace(std::move(*expression->clone())); else dag = expression->clone(); - - dag->projectInput(false); } /// This function builds a common DAG which is a merge of DAGs from Filter and Expression steps chain. diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 4017670ad14d..7eca8ed74f92 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -77,7 +77,7 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( AggregateProjectionInfo info; info.context = interpreter.getContext(); - info.before_aggregation = analysis_result.before_aggregation; + info.before_aggregation = analysis_result.before_aggregation->actions.clone(); info.keys = query_analyzer->aggregationKeys().getNames(); info.aggregates = query_analyzer->aggregates(); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 887a95da60da..750bb5600b3a 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -850,8 +850,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ static ActionsDAGPtr createProjection(const Block & header) { auto projection = std::make_shared(header.getNamesAndTypesList()); - projection->removeUnusedActions(header.getNames()); - projection->projectInput(); + // projection->removeUnusedActions(header.getNames()); return projection; } @@ -2010,6 +2009,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons Block cur_header = pipe.getHeader(); + bool project_inputs = result_projection != nullptr; auto append_actions = [&result_projection](ActionsDAGPtr actions) { if (!result_projection) @@ -2035,6 +2035,9 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result_projection) { + if (project_inputs) + result_projection->appendInputsForUnusedColumns(pipe.getHeader()); + auto projection_actions = std::make_shared(result_projection); pipe.addSimpleTransform([&](const Block & header) { diff --git a/src/Processors/Transforms/AddingDefaultsTransform.cpp b/src/Processors/Transforms/AddingDefaultsTransform.cpp index e6c2bcec2c85..7945b3999c11 100644 --- a/src/Processors/Transforms/AddingDefaultsTransform.cpp +++ b/src/Processors/Transforms/AddingDefaultsTransform.cpp @@ -178,7 +178,7 @@ void AddingDefaultsTransform::transform(Chunk & chunk) auto dag = evaluateMissingDefaults(evaluate_block, header.getNamesAndTypesList(), columns, context, false); if (dag) { - auto actions = std::make_shared(std::move(dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes)); + auto actions = std::make_shared(std::move(dag), ExpressionActionsSettings::fromContext(context, CompileExpressions::yes), true); actions->execute(evaluate_block); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index cd706dab9aea..ea1d9cd8131d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7050,7 +7050,7 @@ ActionDAGNodes MergeTreeData::getFiltersForPrimaryKeyAnalysis(const InterpreterS filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name)); if (before_where) - filter_nodes.nodes.push_back(&before_where->findInOutputs(where_column_name)); + filter_nodes.nodes.push_back(&before_where->actions.findInOutputs(where_column_name)); return filter_nodes; } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 735f51e1f320..9625c13197cc 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -273,7 +273,6 @@ void StorageMaterializedView::read( * They may be added in case of distributed query with JOIN. * In that case underlying table returns joined columns as well. */ - converting_actions->projectInput(false); auto converting_step = std::make_unique(query_plan.getCurrentDataStream(), converting_actions); converting_step->setStepDescription("Convert target table structure to MaterializedView structure"); query_plan.addStep(std::move(converting_step)); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 4c678a1228b6..bb27715ddd10 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -964,7 +964,7 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo } PlannerActionsVisitor actions_visitor(modified_query_info.planner_context, false /*use_column_identifier_as_action_node_name*/); - actions_visitor.visit(filter_actions_dag, column_node); + actions_visitor.visit(*filter_actions_dag, column_node); } column_names_as_aliases = filter_actions_dag->getRequiredColumnsNames(); if (column_names_as_aliases.empty()) @@ -1513,7 +1513,7 @@ void ReadFromMerge::convertAndFilterSourceStream( query_analysis_pass.run(query_tree, local_context); PlannerActionsVisitor actions_visitor(modified_query_info.planner_context, false /*use_column_identifier_as_action_node_name*/); - const auto & nodes = actions_visitor.visit(actions_dag, query_tree); + const auto & nodes = actions_visitor.visit(*actions_dag, query_tree); if (nodes.size() != 1) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected to have 1 output but got {}", nodes.size()); From cb9ea78810e1ccb894fae0831c299a7e5958f3af Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jun 2024 08:13:51 +0000 Subject: [PATCH 130/254] Fix THERE_IS_NO_COLUMN error in case move to PREWHERE applied to storage merge inside another table function (cherry picked from commit ac22904ff2b960b46b85b8197cbf814f26855049) --- ...3165_storage_merge_view_prewhere.reference | 7 ++++ .../03165_storage_merge_view_prewhere.sql | 41 +++++++++++++++++++ 2 files changed, 48 insertions(+) create mode 100644 tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference create mode 100644 tests/queries/0_stateless/03165_storage_merge_view_prewhere.sql diff --git a/tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference b/tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference new file mode 100644 index 000000000000..3ee56295b2e6 --- /dev/null +++ b/tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference @@ -0,0 +1,7 @@ +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever +a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever diff --git a/tests/queries/0_stateless/03165_storage_merge_view_prewhere.sql b/tests/queries/0_stateless/03165_storage_merge_view_prewhere.sql new file mode 100644 index 000000000000..97651d1b0fdb --- /dev/null +++ b/tests/queries/0_stateless/03165_storage_merge_view_prewhere.sql @@ -0,0 +1,41 @@ +-- Tags: distributed + +DROP TABLE IF EXISTS ids; +DROP TABLE IF EXISTS data; +DROP TABLE IF EXISTS data2; + +CREATE TABLE ids (id UUID, whatever String) Engine=MergeTree ORDER BY tuple(); +INSERT INTO ids VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', 'whatever'); + +CREATE TABLE data (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); +INSERT INTO data VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-01', 'CREATED'); + +CREATE TABLE data2 (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); +INSERT INTO data2 VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-02', 'CREATED'); + +SELECT + id, + whatever +FROM ids AS l +INNER JOIN merge(currentDatabase(), 'data*') AS s ON l.id = s.id +WHERE (status IN ['CREATED', 'CREATING']) +ORDER BY event_time DESC +; + +SELECT + id, + whatever +FROM ids AS l +INNER JOIN clusterAllReplicas(test_cluster_two_shards, merge(currentDatabase(), 'data*')) AS s ON l.id = s.id +WHERE (status IN ['CREATED', 'CREATING']) +ORDER BY event_time DESC +; + +SELECT + id, + whatever +FROM ids AS l +INNER JOIN view(SELECT * FROM merge(currentDatabase(), 'data*')) AS s ON l.id = s.id +WHERE (status IN ['CREATED', 'CREATING']) +ORDER BY event_time DESC +; From d8b9b00624fd7698de20eb63e5c9ecbb8632a7ae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 13 Jun 2024 14:03:41 +0000 Subject: [PATCH 131/254] Revert "Fix THERE_IS_NO_COLUMN error in case move to PREWHERE applied to storage merge inside another table function" This reverts commit cb9ea78810e1ccb894fae0831c299a7e5958f3af. --- ...3165_storage_merge_view_prewhere.reference | 7 ---- .../03165_storage_merge_view_prewhere.sql | 41 ------------------- 2 files changed, 48 deletions(-) delete mode 100644 tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference delete mode 100644 tests/queries/0_stateless/03165_storage_merge_view_prewhere.sql diff --git a/tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference b/tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference deleted file mode 100644 index 3ee56295b2e6..000000000000 --- a/tests/queries/0_stateless/03165_storage_merge_view_prewhere.reference +++ /dev/null @@ -1,7 +0,0 @@ -a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever -a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever -a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever -a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever -a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever -a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever -a1451105-722e-4fe7-bfaa-65ad2ae249c2 whatever diff --git a/tests/queries/0_stateless/03165_storage_merge_view_prewhere.sql b/tests/queries/0_stateless/03165_storage_merge_view_prewhere.sql deleted file mode 100644 index 97651d1b0fdb..000000000000 --- a/tests/queries/0_stateless/03165_storage_merge_view_prewhere.sql +++ /dev/null @@ -1,41 +0,0 @@ --- Tags: distributed - -DROP TABLE IF EXISTS ids; -DROP TABLE IF EXISTS data; -DROP TABLE IF EXISTS data2; - -CREATE TABLE ids (id UUID, whatever String) Engine=MergeTree ORDER BY tuple(); -INSERT INTO ids VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', 'whatever'); - -CREATE TABLE data (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); -INSERT INTO data VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-01', 'CREATED'); - -CREATE TABLE data2 (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); -INSERT INTO data2 VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-02', 'CREATED'); - -SELECT - id, - whatever -FROM ids AS l -INNER JOIN merge(currentDatabase(), 'data*') AS s ON l.id = s.id -WHERE (status IN ['CREATED', 'CREATING']) -ORDER BY event_time DESC -; - -SELECT - id, - whatever -FROM ids AS l -INNER JOIN clusterAllReplicas(test_cluster_two_shards, merge(currentDatabase(), 'data*')) AS s ON l.id = s.id -WHERE (status IN ['CREATED', 'CREATING']) -ORDER BY event_time DESC -; - -SELECT - id, - whatever -FROM ids AS l -INNER JOIN view(SELECT * FROM merge(currentDatabase(), 'data*')) AS s ON l.id = s.id -WHERE (status IN ['CREATED', 'CREATING']) -ORDER BY event_time DESC -; From c79433e6f4864169762c8548c5bab071fd70bf69 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Thu, 13 Jun 2024 17:18:34 +0300 Subject: [PATCH 132/254] fix ci --- src/Storages/MergeTree/KeyCondition.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 443c39ec88bc..55cdc9c4aff8 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -755,7 +755,7 @@ void KeyCondition::getAllSpaceFillingCurves() { if (action.node->type == ActionsDAG::ActionType::FUNCTION && action.node->children.size() >= 2 - && space_filling_curve_name_to_type.count(action.node->function_base->getName()) > 0) + && space_filling_curve_name_to_type.contains(action.node->function_base->getName())) { SpaceFillingCurveDescription curve; curve.function_name = action.node->function_base->getName(); From 2997509dbff9c49f4e99f05a5d32f9c802aa51a8 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Thu, 13 Jun 2024 17:44:30 +0300 Subject: [PATCH 133/254] docs upd + tests --- docs/en/operations/settings/settings.md | 14 ++++---- .../03171_indexing_by_hilbert_curve.reference | 9 +++++ .../03171_indexing_by_hilbert_curve.sql | 35 +++++++++++++++++++ 3 files changed, 51 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/03171_indexing_by_hilbert_curve.reference create mode 100644 tests/queries/0_stateless/03171_indexing_by_hilbert_curve.sql diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 79d0ca4f1512..76c59a7be505 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1592,19 +1592,19 @@ Default value: `default`. ## parallel_replicas_custom_key_range_lower {#parallel_replicas_custom_key_range_lower} -Allows the filter type `range` to split the work evenly between replicas based on the custom range `[parallel_replicas_custom_key_range_lower, INT_MAX]`. +Allows the filter type `range` to split the work evenly between replicas based on the custom range `[parallel_replicas_custom_key_range_lower, INT_MAX]`. -When used in conjuction with [parallel_replicas_custom_key_range_upper](#parallel_replicas_custom_key_range_upper), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. +When used in conjuction with [parallel_replicas_custom_key_range_upper](#parallel_replicas_custom_key_range_upper), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. -Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing. +Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing. ## parallel_replicas_custom_key_range_upper {#parallel_replicas_custom_key_range_upper} Allows the filter type `range` to split the work evenly between replicas based on the custom range `[0, parallel_replicas_custom_key_range_upper]`. A value of 0 disables the upper bound, setting it the max value of the custom key expression. -When used in conjuction with [parallel_replicas_custom_key_range_lower](#parallel_replicas_custom_key_range_lower), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. +When used in conjuction with [parallel_replicas_custom_key_range_lower](#parallel_replicas_custom_key_range_lower), it lets the filter evenly split the work over replicas for the range `[parallel_replicas_custom_key_range_lower, parallel_replicas_custom_key_range_upper]`. -Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing. +Note: This setting will not cause any additional data to be filtered during query processing, rather it changes the points at which the range filter breaks up the range `[0, INT_MAX]` for parallel processing. ## allow_experimental_parallel_reading_from_replicas @@ -3188,7 +3188,7 @@ Default value: `0`. ## lightweight_deletes_sync {#lightweight_deletes_sync} -The same as 'mutation_sync', but controls only execution of lightweight deletes. +The same as 'mutation_sync', but controls only execution of lightweight deletes. Possible values: @@ -5150,7 +5150,7 @@ Allows using statistic to optimize the order of [prewhere conditions](../../sql- ## analyze_index_with_space_filling_curves -If a table has a space-filling curve in its index, e.g. `ORDER BY mortonEncode(x, y)`, and the query has conditions on its arguments, e.g. `x >= 10 AND x <= 20 AND y >= 20 AND y <= 30`, use the space-filling curve for index analysis. +If a table has a space-filling curve in its index, e.g. `ORDER BY mortonEncode(x, y)` or `ORDER BY hilbertEncode(x, y)`, and the query has conditions on its arguments, e.g. `x >= 10 AND x <= 20 AND y >= 20 AND y <= 30`, use the space-filling curve for index analysis. ## query_plan_enable_optimizations {#query_plan_enable_optimizations} diff --git a/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.reference b/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.reference new file mode 100644 index 000000000000..a223737eae65 --- /dev/null +++ b/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.reference @@ -0,0 +1,9 @@ +121 +121 +32 +21 +10 +32 +22 +11 +1 \ No newline at end of file diff --git a/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.sql b/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.sql new file mode 100644 index 000000000000..207f54587c9d --- /dev/null +++ b/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.sql @@ -0,0 +1,35 @@ +DROP TABLE IF EXISTS test_hilbert_encode_hilbert_encode; + +CREATE TABLE test_hilbert_encode (x UInt32, y UInt32) ENGINE = MergeTree ORDER BY hilbertEncode(x, y) SETTINGS index_granularity = 8192, index_granularity_bytes = '1Mi'; +INSERT INTO test_hilbert_encode SELECT number DIV 1024, number % 1024 FROM numbers(1048576); + +SET max_rows_to_read = 8192, force_primary_key = 1, analyze_index_with_space_filling_curves = 1; +SELECT count() FROM test_hilbert_encode WHERE x >= 10 AND x <= 20 AND y >= 20 AND y <= 30; + +SET max_rows_to_read = 8192, force_primary_key = 1, analyze_index_with_space_filling_curves = 0; +SELECT count() FROM test_hilbert_encode WHERE x >= 10 AND x <= 20 AND y >= 20 AND y <= 30; -- { serverError 277 } + +DROP TABLE test_hilbert_encode; + +-- The same, but with more precise index + +CREATE TABLE test_hilbert_encode (x UInt32, y UInt32) ENGINE = MergeTree ORDER BY hilbertEncode(x, y) SETTINGS index_granularity = 1; +SET max_rows_to_read = 0; +INSERT INTO test_hilbert_encode SELECT number DIV 32, number % 32 FROM numbers(1024); + +SET max_rows_to_read = 200, force_primary_key = 1, analyze_index_with_space_filling_curves = 1; +SELECT count() FROM test_hilbert_encode WHERE x >= 10 AND x <= 20 AND y >= 20 AND y <= 30; + +-- Various other conditions + +SELECT count() FROM test_hilbert_encode WHERE x = 10 SETTINGS max_rows_to_read = 64; +SELECT count() FROM test_hilbert_encode WHERE x = 10 AND y > 10 SETTINGS max_rows_to_read = 42; +SELECT count() FROM test_hilbert_encode WHERE x = 10 AND y < 10 SETTINGS max_rows_to_read = 20; + +SELECT count() FROM test_hilbert_encode WHERE y = 10 SETTINGS max_rows_to_read = 48; +SELECT count() FROM test_hilbert_encode WHERE x >= 10 AND y = 10 SETTINGS max_rows_to_read = 33; +SELECT count() FROM test_hilbert_encode WHERE y = 10 AND x <= 10 SETTINGS max_rows_to_read = 17; + +SELECT count() FROM test_hilbert_encode PREWHERE x >= 10 WHERE x < 11 AND y = 10 SETTINGS max_rows_to_read = 3; + +DROP TABLE test_hilbert_encode; \ No newline at end of file From c5159bc3aa409448f6663718607909c34b63fb71 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 13 Jun 2024 14:52:34 +0000 Subject: [PATCH 134/254] fix setNeeded() in PlanSquashing --- src/Processors/Transforms/PlanSquashingTransform.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index 5ae605cc51a1..8598a29278aa 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -64,12 +64,12 @@ IProcessor::Status PlanSquashingTransform::prepareConsume() for (auto & input : inputs) { if (!input.isFinished()) - all_finished = false; - else { + all_finished = false; input.setNeeded(); - continue; } + else + continue; if (input.hasData()) { From ff6d1d09d60126922a8786f2bab10d28fa923db6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 13 Jun 2024 14:23:45 +0200 Subject: [PATCH 135/254] Fix possible crash for hedged requests Previously, it was possible for hedged requests to continue choosing replica even after the query had been cancelled (RemoteQueryExecutor::tryCancel()), and not only this does not make sense, but could also lead to a crash, due to use-after-free of current_thread (ThreadStatus), since fiber had been created on a different thread (thread for query pipeline), but will be destroyed from another thread (that calls QueryPipeline dtor), and the query pipeline's thread could be already destroyed by that time (especially under threads pressure). v0: IConnection::cancelAsync() v2: remove it, since the query is sent in a deferred manner for hedged requests, so that said that modifying HedgedConnections::sendCancel() should be enough Signed-off-by: Azat Khuzhin --- src/Client/HedgedConnections.cpp | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index fb4d9a6bdcc3..8c993f906e0f 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -255,6 +255,17 @@ void HedgedConnections::sendCancel() if (!sent_query || cancelled) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot cancel. Either no query sent or already cancelled."); + /// All hedged connections should be stopped, since otherwise before the + /// HedgedConnectionsFactory will be destroyed (that will happen from + /// QueryPipeline dtor) they could still do some work. + /// And not only this does not make sense, but it also could lead to + /// use-after-free of the current_thread, since the thread from which they + /// had been created differs from the thread where the dtor of + /// QueryPipeline will be called and the initial thread could be already + /// destroyed (especially when the system is under pressure). + if (hedged_connections_factory.hasEventsInProcess()) + hedged_connections_factory.stopChoosingReplicas(); + cancelled = true; for (auto & offset_status : offset_states) From f82c173c209e2cc7b2e8aa3e9ad134e86c4c02cf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 13 Jun 2024 16:23:35 +0000 Subject: [PATCH 136/254] Fixing some tests. --- src/Interpreters/ActionsDAG.cpp | 31 +++++++++++++++++-------- src/Interpreters/ExpressionAnalyzer.cpp | 23 +++++++++++------- 2 files changed, 36 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index da53f197fd82..92d8e3ace9ea 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1149,6 +1149,21 @@ void ActionsDAG::project(const NamesWithAliases & projection) removeUnusedActions(); } +static void appendInputsFromNamesMap( + ActionsDAG & dag, + const ColumnsWithTypeAndName & source_columns, + const std::unordered_map> & names_map) +{ + for (const auto & [_, positions] : names_map) + { + for (auto pos : positions) + { + const auto & col = source_columns[pos]; + dag.addInput(col.name, col.type); + } + } +} + void ActionsDAG::appendInputsForUnusedColumns(const Block & sample_block) { std::unordered_map> names_map; @@ -1166,14 +1181,7 @@ void ActionsDAG::appendInputsForUnusedColumns(const Block & sample_block) positions.pop_front(); } - for (const auto & [_, positions] : names_map) - { - for (auto pos : positions) - { - const auto & col = sample_block.getByPosition(pos); - addInput(col.name, col.type); - } - } + appendInputsFromNamesMap(*this, sample_block.getColumnsWithTypeAndName(), names_map); } bool ActionsDAG::tryRestoreColumn(const std::string & column_name) @@ -1426,7 +1434,7 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( FunctionOverloadResolverPtr func_builder_materialize = std::make_unique(std::make_shared()); - std::map> inputs; + std::unordered_map> inputs; if (mode == MatchColumnsMode::Name) { size_t input_nodes_size = actions_dag->inputs.size(); @@ -1542,7 +1550,10 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( } actions_dag->outputs.swap(projection); - actions_dag->removeUnusedActions(); + actions_dag->removeUnusedActions(false); + + if (mode == MatchColumnsMode::Name) + appendInputsFromNamesMap(*actions_dag, source, inputs); return actions_dag; } diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index c53cdd0d2ed7..b4fa852916ee 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1789,7 +1789,7 @@ ActionsDAGPtr ExpressionAnalyzer::getActionsDAG(bool add_aliases, bool remove_un ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool remove_unused_result, CompileExpressions compile_expressions) { return std::make_shared( - getActionsDAG(add_aliases, remove_unused_result), ExpressionActionsSettings::fromContext(getContext(), compile_expressions), remove_unused_result); + getActionsDAG(add_aliases, remove_unused_result), ExpressionActionsSettings::fromContext(getContext(), compile_expressions), add_aliases && remove_unused_result); } ActionsDAGPtr ExpressionAnalyzer::getConstActionsDAG(const ColumnsWithTypeAndName & constant_inputs) @@ -1850,14 +1850,16 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( ssize_t where_step_num = -1; ssize_t having_step_num = -1; + ActionsAndFlagsPtr prewhere_dag_and_flags; + auto finalize_chain = [&](ExpressionActionsChain & chain) -> ColumnsWithTypeAndName { if (prewhere_step_num >= 0) { ExpressionActionsChain::Step & step = *chain.steps.at(prewhere_step_num); - auto required_columns_ = prewhere_info->prewhere_actions->getRequiredColumnsNames(); - NameSet required_source_columns(required_columns_.begin(), required_columns_.end()); + auto prewhere_required_columns = prewhere_dag_and_flags->actions.getRequiredColumnsNames(); + NameSet required_source_columns(prewhere_required_columns.begin(), prewhere_required_columns.end()); /// Add required columns to required output in order not to remove them after prewhere execution. /// TODO: add sampling and final execution to common chain. for (const auto & column : additional_required_columns_after_prewhere) @@ -1869,6 +1871,12 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( chain.finalize(); + if (prewhere_dag_and_flags) + { + auto dag = std::make_shared(std::move(prewhere_dag_and_flags->actions)); + prewhere_info = std::make_shared(std::move(dag), query.prewhere()->getColumnName()); + } + finalize(chain, prewhere_step_num, where_step_num, having_step_num, query); auto res = chain.getLastStep().getResultColumns(); @@ -1919,20 +1927,19 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( filter_info->do_remove_column = true; } - if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage)) + if (prewhere_dag_and_flags = query_analyzer.appendPrewhere(chain, !first_stage); prewhere_dag_and_flags) { /// Prewhere is always the first one. prewhere_step_num = 0; - auto dag = std::make_shared(std::move(actions->actions)); - prewhere_info = std::make_shared(std::move(dag), query.prewhere()->getColumnName()); - if (allowEarlyConstantFolding(*prewhere_info->prewhere_actions, settings)) + if (allowEarlyConstantFolding(prewhere_dag_and_flags->actions, settings)) { Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { + auto dag = prewhere_dag_and_flags->actions.clone(); ExpressionActions( - prewhere_info->prewhere_actions, + dag, ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); auto & column_elem = before_prewhere_sample.getByName(query.prewhere()->getColumnName()); /// If the filter column is a constant, record it. From fb32a99578b57cf185f6e868879aaf2ff218419d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 13 Jun 2024 19:13:13 +0200 Subject: [PATCH 137/254] Initialize global trace collector for Poco::ThreadPool --- base/poco/Foundation/CMakeLists.txt | 1 + .../poco/Foundation/include/Poco/ThreadPool.h | 20 ++++- base/poco/Foundation/src/ThreadPool.cpp | 75 ++++++++++++------- programs/server/Server.cpp | 18 +++-- src/Server/HTTPHandler.cpp | 1 - src/Server/InterserverIOHTTPHandler.cpp | 2 - src/Server/KeeperTCPHandler.cpp | 1 - src/Server/MySQLHandler.cpp | 2 - src/Server/PostgreSQLHandler.cpp | 2 - src/Server/TCPHandler.cpp | 1 - src/Server/TCPHandler.h | 1 - 11 files changed, 81 insertions(+), 43 deletions(-) diff --git a/base/poco/Foundation/CMakeLists.txt b/base/poco/Foundation/CMakeLists.txt index dfb41a33fb1a..324a0170bdde 100644 --- a/base/poco/Foundation/CMakeLists.txt +++ b/base/poco/Foundation/CMakeLists.txt @@ -213,6 +213,7 @@ target_compile_definitions (_poco_foundation ) target_include_directories (_poco_foundation SYSTEM PUBLIC "include") +target_link_libraries (_poco_foundation PRIVATE clickhouse_common_io) target_link_libraries (_poco_foundation PRIVATE diff --git a/base/poco/Foundation/include/Poco/ThreadPool.h b/base/poco/Foundation/include/Poco/ThreadPool.h index b9506cc5b7f0..e2187bfeb66d 100644 --- a/base/poco/Foundation/include/Poco/ThreadPool.h +++ b/base/poco/Foundation/include/Poco/ThreadPool.h @@ -48,7 +48,13 @@ class Foundation_API ThreadPool /// from the pool. { public: - ThreadPool(int minCapacity = 2, int maxCapacity = 16, int idleTime = 60, int stackSize = POCO_THREAD_STACK_SIZE); + explicit ThreadPool( + int minCapacity = 2, + int maxCapacity = 16, + int idleTime = 60, + int stackSize = POCO_THREAD_STACK_SIZE, + size_t global_profiler_real_time_period_ns_ = 0, + size_t global_profiler_cpu_time_period_ns_ = 0); /// Creates a thread pool with minCapacity threads. /// If required, up to maxCapacity threads are created /// a NoThreadAvailableException exception is thrown. @@ -56,8 +62,14 @@ class Foundation_API ThreadPool /// and more than minCapacity threads are running, the thread /// is killed. Threads are created with given stack size. - ThreadPool( - const std::string & name, int minCapacity = 2, int maxCapacity = 16, int idleTime = 60, int stackSize = POCO_THREAD_STACK_SIZE); + explicit ThreadPool( + const std::string & name, + int minCapacity = 2, + int maxCapacity = 16, + int idleTime = 60, + int stackSize = POCO_THREAD_STACK_SIZE, + size_t global_profiler_real_time_period_ns_ = 0, + size_t global_profiler_cpu_time_period_ns_ = 0); /// Creates a thread pool with the given name and minCapacity threads. /// If required, up to maxCapacity threads are created /// a NoThreadAvailableException exception is thrown. @@ -171,6 +183,8 @@ class Foundation_API ThreadPool int _serial; int _age; int _stackSize; + size_t _globalProfilerRealTimePeriodNs; + size_t _globalProfilerCPUTimePeriodNs; ThreadVec _threads; mutable FastMutex _mutex; }; diff --git a/base/poco/Foundation/src/ThreadPool.cpp b/base/poco/Foundation/src/ThreadPool.cpp index 6335ee82b477..f57c81e4128d 100644 --- a/base/poco/Foundation/src/ThreadPool.cpp +++ b/base/poco/Foundation/src/ThreadPool.cpp @@ -20,6 +20,7 @@ #include "Poco/ErrorHandler.h" #include #include +#include namespace Poco { @@ -28,7 +29,11 @@ namespace Poco { class PooledThread: public Runnable { public: - PooledThread(const std::string& name, int stackSize = POCO_THREAD_STACK_SIZE); + explicit PooledThread( + const std::string& name, + int stackSize = POCO_THREAD_STACK_SIZE, + size_t globalProfilerRealTimePeriodNs_ = 0, + size_t globalProfilerCPUTimePeriodNs_ = 0); ~PooledThread(); void start(); @@ -51,16 +56,24 @@ class PooledThread: public Runnable Event _targetCompleted; Event _started; FastMutex _mutex; + size_t _globalProfilerRealTimePeriodNs; + size_t _globalProfilerCPUTimePeriodNs; }; -PooledThread::PooledThread(const std::string& name, int stackSize): - _idle(true), - _idleTime(0), - _pTarget(0), - _name(name), +PooledThread::PooledThread( + const std::string& name, + int stackSize, + size_t globalProfilerRealTimePeriodNs_, + size_t globalProfilerCPUTimePeriodNs_) : + _idle(true), + _idleTime(0), + _pTarget(0), + _name(name), _thread(name), - _targetCompleted(false) + _targetCompleted(false), + _globalProfilerRealTimePeriodNs(globalProfilerRealTimePeriodNs_), + _globalProfilerCPUTimePeriodNs(globalProfilerCPUTimePeriodNs_) { poco_assert_dbg (stackSize >= 0); _thread.setStackSize(stackSize); @@ -83,7 +96,7 @@ void PooledThread::start() void PooledThread::start(Thread::Priority priority, Runnable& target) { FastMutex::ScopedLock lock(_mutex); - + poco_assert (_pTarget == 0); _pTarget = ⌖ @@ -109,7 +122,7 @@ void PooledThread::start(Thread::Priority priority, Runnable& target, const std: } _thread.setName(fullName); _thread.setPriority(priority); - + poco_assert (_pTarget == 0); _pTarget = ⌖ @@ -145,7 +158,7 @@ void PooledThread::join() void PooledThread::activate() { FastMutex::ScopedLock lock(_mutex); - + poco_assert (_idle); _idle = false; _targetCompleted.reset(); @@ -155,7 +168,7 @@ void PooledThread::activate() void PooledThread::release() { const long JOIN_TIMEOUT = 10000; - + _mutex.lock(); _pTarget = 0; _mutex.unlock(); @@ -174,6 +187,10 @@ void PooledThread::release() void PooledThread::run() { + DB::ThreadStatus thread_status; + if (unlikely(_globalProfilerRealTimePeriodNs != 0 || _globalProfilerCPUTimePeriodNs != 0)) + thread_status.initGlobalProfiler(_globalProfilerRealTimePeriodNs, _globalProfilerCPUTimePeriodNs); + _started.set(); for (;;) { @@ -220,13 +237,17 @@ void PooledThread::run() ThreadPool::ThreadPool(int minCapacity, int maxCapacity, int idleTime, - int stackSize): - _minCapacity(minCapacity), - _maxCapacity(maxCapacity), + int stackSize, + size_t globalProfilerRealTimePeriodNs_, + size_t globalProfilerCPUTimePeriodNs_) : + _minCapacity(minCapacity), + _maxCapacity(maxCapacity), _idleTime(idleTime), _serial(0), _age(0), - _stackSize(stackSize) + _stackSize(stackSize), + _globalProfilerRealTimePeriodNs(globalProfilerRealTimePeriodNs_), + _globalProfilerCPUTimePeriodNs(globalProfilerCPUTimePeriodNs_) { poco_assert (minCapacity >= 1 && maxCapacity >= minCapacity && idleTime > 0); @@ -243,14 +264,18 @@ ThreadPool::ThreadPool(const std::string& name, int minCapacity, int maxCapacity, int idleTime, - int stackSize): + int stackSize, + size_t globalProfilerRealTimePeriodNs_, + size_t globalProfilerCPUTimePeriodNs_) : _name(name), - _minCapacity(minCapacity), - _maxCapacity(maxCapacity), + _minCapacity(minCapacity), + _maxCapacity(maxCapacity), _idleTime(idleTime), _serial(0), _age(0), - _stackSize(stackSize) + _stackSize(stackSize), + _globalProfilerRealTimePeriodNs(globalProfilerRealTimePeriodNs_), + _globalProfilerCPUTimePeriodNs(globalProfilerCPUTimePeriodNs_) { poco_assert (minCapacity >= 1 && maxCapacity >= minCapacity && idleTime > 0); @@ -393,15 +418,15 @@ void ThreadPool::housekeep() ThreadVec activeThreads; idleThreads.reserve(_threads.size()); activeThreads.reserve(_threads.size()); - + for (ThreadVec::iterator it = _threads.begin(); it != _threads.end(); ++it) { if ((*it)->idle()) { if ((*it)->idleTime() < _idleTime) idleThreads.push_back(*it); - else - expiredThreads.push_back(*it); + else + expiredThreads.push_back(*it); } else activeThreads.push_back(*it); } @@ -463,7 +488,7 @@ PooledThread* ThreadPool::createThread() { std::ostringstream name; name << _name << "[#" << ++_serial << "]"; - return new PooledThread(name.str(), _stackSize); + return new PooledThread(name.str(), _stackSize, _globalProfilerRealTimePeriodNs, _globalProfilerCPUTimePeriodNs); } @@ -481,7 +506,7 @@ class ThreadPoolSingletonHolder ThreadPool* pool() { FastMutex::ScopedLock lock(_mutex); - + if (!_pPool) { _pPool = new ThreadPool("default"); @@ -490,7 +515,7 @@ class ThreadPoolSingletonHolder } return _pPool; } - + private: ThreadPool* _pPool; FastMutex _mutex; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c3a5be706b40..8a5f6173c49b 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -721,11 +722,6 @@ try CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); - Poco::ThreadPool server_pool(3, server_settings.max_connections); - std::mutex servers_lock; - std::vector servers; - std::vector servers_to_start_before_tables; - /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases, ... */ @@ -823,6 +819,18 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } + Poco::ThreadPool server_pool( + /* minCapacity */3, + /* maxCapacity */server_settings.max_connections, + /* idleTime */60, + /* stackSize */POCO_THREAD_STACK_SIZE, + server_settings.global_profiler_real_time_period_ns, + server_settings.global_profiler_cpu_time_period_ns); + + std::mutex servers_lock; + std::vector servers; + std::vector servers_to_start_before_tables; + /// Wait for all threads to avoid possible use-after-free (for example logging objects can be already destroyed). SCOPE_EXIT({ Stopwatch watch; diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 02d0959ff502..d6afa571e712 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1060,7 +1060,6 @@ void HTTPHandler::formatExceptionForClient(int exception_code, HTTPServerRequest void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) { setThreadName("HTTPHandler"); - ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::HTTP, request.isSecure()); SCOPE_EXIT({ session.reset(); }); diff --git a/src/Server/InterserverIOHTTPHandler.cpp b/src/Server/InterserverIOHTTPHandler.cpp index 0d79aaa227b3..45c28babe3a5 100644 --- a/src/Server/InterserverIOHTTPHandler.cpp +++ b/src/Server/InterserverIOHTTPHandler.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include @@ -81,7 +80,6 @@ void InterserverIOHTTPHandler::processQuery(HTTPServerRequest & request, HTTPSer void InterserverIOHTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event) { setThreadName("IntersrvHandler"); - ThreadStatus thread_status; /// In order to work keep-alive. if (request.getVersion() == HTTPServerRequest::HTTP_1_1) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 6709cd298e5b..4612e2e9fa8a 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -309,7 +309,6 @@ Poco::Timespan KeeperTCPHandler::receiveHandshake(int32_t handshake_length, bool void KeeperTCPHandler::runImpl() { setThreadName("KeeperHandler"); - ThreadStatus thread_status; socket().setReceiveTimeout(receive_timeout); socket().setSendTimeout(send_timeout); diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 6456f6d24ffd..9471509ad4ba 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -24,7 +24,6 @@ #include #include #include -#include #include #include #include @@ -199,7 +198,6 @@ MySQLHandler::~MySQLHandler() = default; void MySQLHandler::run() { setThreadName("MySQLHandler"); - ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::MYSQL); SCOPE_EXIT({ session.reset(); }); diff --git a/src/Server/PostgreSQLHandler.cpp b/src/Server/PostgreSQLHandler.cpp index 473d681ddb21..8ba8421e6f05 100644 --- a/src/Server/PostgreSQLHandler.cpp +++ b/src/Server/PostgreSQLHandler.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include #include @@ -59,7 +58,6 @@ void PostgreSQLHandler::changeIO(Poco::Net::StreamSocket & socket) void PostgreSQLHandler::run() { setThreadName("PostgresHandler"); - ThreadStatus thread_status; session = std::make_unique(server.context(), ClientInfo::Interface::POSTGRESQL); SCOPE_EXIT({ session.reset(); }); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index e3a820340adb..b60339e9fd8b 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -246,7 +246,6 @@ TCPHandler::~TCPHandler() void TCPHandler::runImpl() { setThreadName("TCPHandler"); - ThreadStatus thread_status; extractConnectionSettingsFromContext(server.context()); diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 191617f19050..75e36836b634 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include From 3f7286dc51bcffe499f1da45fdfe75ae4504aa3c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 13 Jun 2024 18:08:00 +0000 Subject: [PATCH 138/254] Fixing build. --- src/Planner/PlannerExpressionAnalysis.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 2e597a7c4669..738b8c1e9d54 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -48,7 +48,7 @@ std::optional analyzeFilter(const QueryTreeNodePtr & filte result.filter_actions = std::make_shared(); result.filter_actions->actions = buildActionsDAGFromExpressionNode(filter_expression_node, input_columns, planner_context); - const auto * output = result.filter_actions->getOutputs().at(0); + const auto * output = result.filter_actions->actions.getOutputs().at(0); if (output->column && ConstantFilterDescription(*output->column).always_true) return {}; From c349df44af8fcdf0e39fc85dab346f463832f5ee Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 13 Jun 2024 18:14:13 +0000 Subject: [PATCH 139/254] Fixing some tests. --- src/Interpreters/ActionsVisitor.cpp | 5 ++--- src/Interpreters/ActionsVisitor.h | 3 ++- src/Interpreters/ExpressionActions.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 -- 4 files changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index b292d02ccaa2..89de76d28153 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -532,9 +532,8 @@ std::vector ActionsMatcher::Data::getAllColumnNames() const ScopeStack::ScopeStack(ActionsDAG actions_dag, ContextPtr context_) : WithContext(context_) { - ScopeStack::Level tmp; - tmp.actions_dag = std::move(actions_dag); - auto & level = stack.emplace_back(std::move(tmp)); + auto & level = stack.emplace_back(); + level.actions_dag = std::move(actions_dag); level.index = std::make_unique(level.actions_dag.getOutputs()); for (const auto & node : level.actions_dag.getOutputs()) diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index e13bd82be069..7e3b7fcb6bbe 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -53,7 +54,7 @@ struct ScopeStack : WithContext Level(Level &&); }; - using Levels = std::vector; + using Levels = std::deque; Levels stack; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index d25d5b532263..e19c4b0cd20a 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -762,7 +762,7 @@ void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run, { block.clear(); } - if (allow_duplicates_in_input) + else if (allow_duplicates_in_input) { /// This case is the same as when the input is projected /// since we do not need any input columns. diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 91bfa863bd9b..927bafe4bfbc 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -386,8 +386,6 @@ Chain InterpreterInsertQuery::buildPreSinkChain( auto adding_missing_defaults_actions = std::make_shared(adding_missing_defaults_dag); - std::cerr << adding_missing_defaults_actions->dumpActions() << std::endl; - /// Actually we don't know structure of input blocks from query/table, /// because some clients break insertion protocol (columns != header) out.addSource(std::make_shared(query_sample_block, adding_missing_defaults_actions)); From fcc16b9f5229dde2207cf3789fe800a03548b682 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Fri, 14 Jun 2024 02:08:17 +0300 Subject: [PATCH 140/254] improve code + correct tests --- src/Common/HilbertUtils.h | 29 ++++++++++++++----- src/Storages/MergeTree/KeyCondition.cpp | 1 + src/Storages/MergeTree/KeyCondition.h | 2 -- .../03171_indexing_by_hilbert_curve.sql | 12 ++++---- 4 files changed, 29 insertions(+), 15 deletions(-) diff --git a/src/Common/HilbertUtils.h b/src/Common/HilbertUtils.h index 9a6ebe9fe81b..a6437019bd34 100644 --- a/src/Common/HilbertUtils.h +++ b/src/Common/HilbertUtils.h @@ -117,12 +117,27 @@ inline std::array, 2> createRangeFromCorners(UInt64 x1 UInt64 dist_x = x1 > x2 ? x1 - x2 : x2 - x1; UInt64 dist_y = y1 > y2 ? y1 - y2 : y2 - y1; UInt64 range_size = std::max(dist_x, dist_y); - UInt64 x_min = std::min(x1, x2); - UInt64 y_min = std::min(y1, y2); - return { - std::pair{x_min, x_min + range_size}, - std::pair{y_min, y_min + range_size} - }; + bool containsMinimumVertice = x1 % (range_size + 1) == 0; + if (containsMinimumVertice) + { + UInt64 x_min = std::min(x1, x2); + UInt64 y_min = std::min(y1, y2); + return { + std::pair{x_min, x_min + range_size}, + std::pair{y_min, y_min + range_size} + }; + } + else + { + UInt64 x_max = std::max(x1, x2); + UInt64 y_max = std::max(y1, y2); + chassert(x_max >= range_size); + chassert(y_max >= range_size); + return { + std::pair{x_max - range_size, x_max}, + std::pair{y_max - range_size, y_max} + }; + } } /** Unpack an interval of Hilbert curve to hyperrectangles covered by it across N dimensions. @@ -130,7 +145,7 @@ inline std::array, 2> createRangeFromCorners(UInt64 x1 template void hilbertIntervalToHyperrectangles2D(UInt64 first, UInt64 last, F && callback) { - const auto equal_bits_count = getLeadingZeroBits(last - first); + const auto equal_bits_count = getLeadingZeroBits(last | first); const auto even_equal_bits_count = equal_bits_count - equal_bits_count % 2; segmentBinaryPartition(first, last, 64 - even_equal_bits_count, [&](HilbertDetails::Segment range) { diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 55cdc9c4aff8..7e4b1db4c89a 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 7f92206dd241..6e5956706aa5 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -2,8 +2,6 @@ #include -#include - #include #include #include diff --git a/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.sql b/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.sql index 207f54587c9d..590f979efb2c 100644 --- a/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.sql +++ b/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.sql @@ -22,14 +22,14 @@ SELECT count() FROM test_hilbert_encode WHERE x >= 10 AND x <= 20 AND y >= 20 AN -- Various other conditions -SELECT count() FROM test_hilbert_encode WHERE x = 10 SETTINGS max_rows_to_read = 64; -SELECT count() FROM test_hilbert_encode WHERE x = 10 AND y > 10 SETTINGS max_rows_to_read = 42; -SELECT count() FROM test_hilbert_encode WHERE x = 10 AND y < 10 SETTINGS max_rows_to_read = 20; +SELECT count() FROM test_hilbert_encode WHERE x = 10 SETTINGS max_rows_to_read = 49; +SELECT count() FROM test_hilbert_encode WHERE x = 10 AND y > 10 SETTINGS max_rows_to_read = 33; +SELECT count() FROM test_hilbert_encode WHERE x = 10 AND y < 10 SETTINGS max_rows_to_read = 15; -SELECT count() FROM test_hilbert_encode WHERE y = 10 SETTINGS max_rows_to_read = 48; -SELECT count() FROM test_hilbert_encode WHERE x >= 10 AND y = 10 SETTINGS max_rows_to_read = 33; +SELECT count() FROM test_hilbert_encode WHERE y = 10 SETTINGS max_rows_to_read = 50; +SELECT count() FROM test_hilbert_encode WHERE x >= 10 AND y = 10 SETTINGS max_rows_to_read = 35; SELECT count() FROM test_hilbert_encode WHERE y = 10 AND x <= 10 SETTINGS max_rows_to_read = 17; -SELECT count() FROM test_hilbert_encode PREWHERE x >= 10 WHERE x < 11 AND y = 10 SETTINGS max_rows_to_read = 3; +SELECT count() FROM test_hilbert_encode PREWHERE x >= 10 WHERE x < 11 AND y = 10 SETTINGS max_rows_to_read = 2; DROP TABLE test_hilbert_encode; \ No newline at end of file From bd0da139afe4e7b8397e831694f71c5f13b42bd1 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Thu, 30 May 2024 16:51:36 +0800 Subject: [PATCH 141/254] Add 2 server_setting to limit numebr of table/db max_table_num_to_throw max_database_num_to_throw --- .../settings.md | 16 ++ src/Common/ErrorCodes.cpp | 2 + src/Core/ServerSettings.h | 2 + src/Databases/DatabaseLazy.cpp | 10 +- src/Interpreters/InterpreterCreateQuery.cpp | 29 ++++ ...abase_and_table_count_limitation_reference | 20 +++ ...database_and_table_count_limitation_result | 20 +++ ...56_database_and_table_count_limitation_sql | 21 +++ ..._and_table_count_limitation_test.reference | 1 + ...atabase_and_table_count_limitation_test.sh | 154 ++++++++++++++++++ 10 files changed, 273 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/31156_database_and_table_count_limitation_reference create mode 100644 tests/queries/0_stateless/31156_database_and_table_count_limitation_result create mode 100644 tests/queries/0_stateless/31156_database_and_table_count_limitation_sql create mode 100644 tests/queries/0_stateless/31156_database_and_table_count_limitation_test.reference create mode 100755 tests/queries/0_stateless/31156_database_and_table_count_limitation_test.sh diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index f50dae0f1a23..6c4a9fa39d61 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -591,6 +591,22 @@ Default value: 100000 400 ``` +## max\_table\_num\_to\_throw {#max-table-num-to-throw} +If number of tables is greater than this value, server will throw an exception. 0 means no limitation. View, remote tables, dictionary, system tables are not counted. Only count table in Atomic/Ordinary/Replicated/Lazy database engine.Default value: 0 + +**Example** +```xml +400 +``` + +## max\_database\_num\_to\_throw {#max-table-num-to-throw} +If number of _database is greater than this value, server will throw an exception. 0 means no limitation. +Default value: 0 + +**Example** +```xml +400 +``` ## max_temporary_data_on_disk_size diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 29993ed33e41..b1b8e2367a4b 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -602,6 +602,8 @@ M(721, DEPRECATED_FUNCTION) \ M(722, ASYNC_LOAD_WAIT_FAILED) \ M(723, PARQUET_EXCEPTION) \ + M(724, TOO_MANY_TABLES) \ + M(725, TOO_MANY_DATABASES) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 309becdd78ff..8261db6ce8e4 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -102,6 +102,8 @@ namespace DB M(UInt64, max_dictionary_num_to_warn, 1000lu, "If the number of dictionaries is greater than this value, the server will create a warning that will displayed to user.", 0) \ M(UInt64, max_database_num_to_warn, 1000lu, "If the number of databases is greater than this value, the server will create a warning that will displayed to user.", 0) \ M(UInt64, max_part_num_to_warn, 100000lu, "If the number of parts is greater than this value, the server will create a warning that will displayed to user.", 0) \ + M(UInt64, max_table_num_to_throw, 0lu, "If number of tables is greater than this value, server will throw an exception. 0 means no limitation. View, remote tables, dictionary, system tables are not counted. Only count table in Atomic/Ordinary/Replicated/Lazy database engine.", 0) \ + M(UInt64, max_database_num_to_throw, 0lu, "If number of databases is greater than this value, server will throw an exception. 0 means no limitation.", 0) \ M(UInt64, concurrent_threads_soft_limit_num, 0, "Sets how many concurrent thread can be allocated before applying CPU pressure. Zero means unlimited.", 0) \ M(UInt64, concurrent_threads_soft_limit_ratio_to_cores, 0, "Same as concurrent_threads_soft_limit_num, but with ratio to cores.", 0) \ \ diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index e72834eddbe1..feb7113e6f9b 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -186,7 +186,10 @@ void DatabaseLazy::attachTable(ContextPtr /* context_ */, const String & table_n throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists.", backQuote(database_name), backQuote(table_name)); it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), current_time, table_name); - CurrentMetrics::add(CurrentMetrics::AttachedTable, 1); + + if (table->isView() == false && table->isRemote() == false + && table->isDictionary() == false && table->isSystemStorage() == false) + CurrentMetrics::add(CurrentMetrics::AttachedTable, 1); } StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & table_name) @@ -202,7 +205,10 @@ StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & ta if (it->second.expiration_iterator != cache_expiration_queue.end()) cache_expiration_queue.erase(it->second.expiration_iterator); tables_cache.erase(it); - CurrentMetrics::sub(CurrentMetrics::AttachedTable, 1); + + if (res->isView() == false && res->isRemote() == false + && res->isDictionary() == false && res->isSystemStorage() == false) + CurrentMetrics::sub(CurrentMetrics::AttachedTable, 1); } return res; } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 66936dc25d72..b0351a004a49 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -88,6 +88,11 @@ #include #include +namespace CurrentMetrics +{ + extern const Metric AttachedTable; +} + namespace DB { @@ -113,6 +118,8 @@ namespace ErrorCodes extern const int UNKNOWN_STORAGE; extern const int SYNTAX_ERROR; extern const int SUPPORT_IS_DISABLED; + extern const int TOO_MANY_TABLES; + extern const int TOO_MANY_DATABASES; } namespace fs = std::filesystem; @@ -138,6 +145,18 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) throw Exception(ErrorCodes::DATABASE_ALREADY_EXISTS, "Database {} already exists.", database_name); } + if (auto max_db = getContext()->getGlobalContext()->getServerSettings().max_database_num_to_throw; max_db > 0) + { + size_t db_count = DatabaseCatalog::instance().getDatabases().size(); + // there's an invisiable system database _temporary_and_external_tables, so we need to subtract 1 + if (db_count > 0) + db_count--; + if (db_count >= max_db) + throw Exception(ErrorCodes::TOO_MANY_DATABASES, + "Too many databases, max: {}, now: {}. " + "See setting max_database_num_to_throw.", max_db, db_count); + } + /// Will write file with database metadata, if needed. String database_name_escaped = escapeForFileName(database_name); fs::path metadata_path = fs::weakly_canonical(getContext()->getPath()); @@ -1544,6 +1563,16 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, } } + if (UInt64 max_table = getContext()->getGlobalContext()->getServerSettings().max_table_num_to_throw; max_table > 0) + { + UInt64 table_count = CurrentMetrics::get(CurrentMetrics::AttachedTable); + if (table_count >= max_table) + throw Exception(ErrorCodes::TOO_MANY_TABLES, + "Too many tables in the system. Current is {}, limit is {}. " + "See setting 'max_table_num_to_throw'.", + table_count, max_table); + } + database->createTable(getContext(), create.getTable(), res, query_ptr); /// Move table data to the proper place. Wo do not move data earlier to avoid situations diff --git a/tests/queries/0_stateless/31156_database_and_table_count_limitation_reference b/tests/queries/0_stateless/31156_database_and_table_count_limitation_reference new file mode 100644 index 000000000000..e0d61993b118 --- /dev/null +++ b/tests/queries/0_stateless/31156_database_and_table_count_limitation_reference @@ -0,0 +1,20 @@ +INFORMATION_SCHEMA +db1 +db2 +db3 +db4 +db5 +db6 +default +information_schema +system +table1 +table10 +table2 +table3 +table4 +table5 +table6 +table7 +table8 +table9 diff --git a/tests/queries/0_stateless/31156_database_and_table_count_limitation_result b/tests/queries/0_stateless/31156_database_and_table_count_limitation_result new file mode 100644 index 000000000000..e0d61993b118 --- /dev/null +++ b/tests/queries/0_stateless/31156_database_and_table_count_limitation_result @@ -0,0 +1,20 @@ +INFORMATION_SCHEMA +db1 +db2 +db3 +db4 +db5 +db6 +default +information_schema +system +table1 +table10 +table2 +table3 +table4 +table5 +table6 +table7 +table8 +table9 diff --git a/tests/queries/0_stateless/31156_database_and_table_count_limitation_sql b/tests/queries/0_stateless/31156_database_and_table_count_limitation_sql new file mode 100644 index 000000000000..127153956f98 --- /dev/null +++ b/tests/queries/0_stateless/31156_database_and_table_count_limitation_sql @@ -0,0 +1,21 @@ +create database db1; +create database db2; +create database db3; +create database db4; +create database db5; +create database db6; +create database db7; +create database db8; +select name from system.databases; +create table db1.table1 (x UInt64) engine = Memory; +create table db1.table2 (x UInt64) engine = Memory; +create table db1.table3 (x UInt64) engine = Memory; +create table db1.table4 (x UInt64) engine = Memory; +create table db1.table5 (x UInt64) engine = Memory; +create table db1.table6 (x UInt64) engine = Memory; +create table db1.table7 (x UInt64) engine = Memory; +create table db1.table8 (x UInt64) engine = Memory; +create table db1.table9 (x UInt64) engine = Memory; +create table db1.table10 (x UInt64) engine = Memory; +create table db1.table11 (x UInt64) engine = Memory; +select name from system.tables where database != 'system' and database != 'information_schema' and database != 'INFORMATION_SCHEMA'; \ No newline at end of file diff --git a/tests/queries/0_stateless/31156_database_and_table_count_limitation_test.reference b/tests/queries/0_stateless/31156_database_and_table_count_limitation_test.reference new file mode 100644 index 000000000000..386f3d2fd86e --- /dev/null +++ b/tests/queries/0_stateless/31156_database_and_table_count_limitation_test.reference @@ -0,0 +1 @@ +Test passed diff --git a/tests/queries/0_stateless/31156_database_and_table_count_limitation_test.sh b/tests/queries/0_stateless/31156_database_and_table_count_limitation_test.sh new file mode 100755 index 000000000000..365fad5e99d9 --- /dev/null +++ b/tests/queries/0_stateless/31156_database_and_table_count_limitation_test.sh @@ -0,0 +1,154 @@ +#!/usr/bin/env bash + +#Tags: shared, no-parallel + +# Get the current script directory +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +BASE="$CUR_DIR/$(basename "${BASH_SOURCE[0]}" .sh)" + +# Load shell_config.sh +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# remove --database=$CLICKHOUSE_DATABASE from $CLICKHOUSE_CLIENT +CLICKHOUSE_CLIENT=$(echo $CLICKHOUSE_CLIENT | sed "s/--database=$CLICKHOUSE_DATABASE//") + +# Create a temporary directory +TEMP_DIR=$(mktemp -d "$CUR_DIR/$(basename "$BASE").XXXXXX") + +# Generate config.xml +CONFIG_FILE="$TEMP_DIR/config.xml" +cat > "$CONFIG_FILE" < + + information + $TEMP_DIR/clickhouse-server.log + $TEMP_DIR/clickhouse-server.err.log + + 10 + 10 + + + + + users.xml + + + + +EOL + +echo " + + + + + + + + ::/0 + + default + default + + + + + + " > $TEMP_DIR/users.xml + +# Function to start the server +function start_server() { + local server_opts=( + "--config-file=$CONFIG_FILE" + "--" + "--tcp_port=0" + "--shutdown_wait_unfinished=0" + "--listen_host=127.1" + "--path=$TEMP_DIR" + ) + CLICKHOUSE_WATCHDOG_ENABLE=0 $CLICKHOUSE_SERVER_BINARY "${server_opts[@]}" > /dev/null 2>&1 & + local pid=$! + + echo "$pid" +} + +# Function to get the server port +function get_server_port() { + local pid=$1 && shift + local port='' + while [[ -z $port ]]; do + port=$(lsof -n -a -P -i tcp -s tcp:LISTEN -p "$pid" 2>/dev/null | awk -F'[ :]' '/LISTEN/ { print $(NF-1) }') + sleep 0.5 + done + echo "$port" +} + +# Function to wait for the server port to be available +function wait_server_port() { + local port=$1 && shift + local i=0 retries=30 + while ! $CLICKHOUSE_CLIENT --host 127.1 --port "$port" --format Null -q 'select 1' 2>/dev/null && [[ $i -lt $retries ]]; do + sleep 0.5 + ((i++)) + done + if ! $CLICKHOUSE_CLIENT --host 127.1 --port "$port" --format Null -q 'select 1'; then + echo "Cannot wait until server will start accepting connections on port $port" >&2 + exit 1 + fi +} + +# Function to stop the server +function stop_server() { + if [[ -n "$SERVER_PID" ]]; then + kill -9 "$SERVER_PID" + fi +} + +# Function to clean up +function cleanup() { + stop_server + rm -rf "$TEMP_DIR" +} + +trap cleanup EXIT + +# Start the server and get the port +SERVER_PID=$(start_server) +PORT=$(get_server_port "$SERVER_PID") + +# Wait for the server to start +wait_server_port "$PORT" + +# check result +sql_file="$CUR_DIR/31156_database_and_table_count_limitation_sql" +result_file="$CUR_DIR/31156_database_and_table_count_limitation_result" +reference_file="$CUR_DIR/31156_database_and_table_count_limitation_reference" + +$CLICKHOUSE_CLIENT --host 127.1 --port "$PORT" --multiquery --ignore-error --queries-file=$sql_file 2>/dev/null > "$result_file" + +# Compare the result with the reference, if not same, print the diff +if ! diff -u "$reference_file" "$result_file"; then + echo "Test failed" + exit 1 +fi + +# check errors in error log +log_file="$TEMP_DIR/clickhouse-server.err.log" +database_error=$(grep -c " executeQuery: Code: 725. DB::Exception: Too many databases" $log_file) +table_error=$(grep -c " executeQuery: Code: 724. DB::Exception: Too many tables" $log_file) +#database_error should be 2 +if [ $database_error -ne 2 ]; then + echo "database_error should be 2, but now is $database_error. Tried to create 8 db, 6 should be created and 2 should fail" + echo "Limitation is 10 databases, 4 exist by default: default, system, information_schema, INFORMATION_SCHEMA" + exit 1 +fi + +#table_error should be 1 +if [ $table_error -ne 1 ]; then + echo "table_error should be 1, but now -s $table_error. Tried to create 11 tables, 10 should be created and 1 should fail" + echo "Limitation is 10 tables" + exit 1 +fi + +echo "Test passed" From 25e927ad38c040d00a7d31a00551c4adcc0609e5 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 4 Jun 2024 14:32:45 +0800 Subject: [PATCH 142/254] do not count systemStorage in AttachedTable metric --- src/Databases/DatabaseLazy.cpp | 8 ++------ src/Databases/DatabasesCommon.cpp | 8 ++++++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index feb7113e6f9b..233db07cd68e 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -187,9 +187,7 @@ void DatabaseLazy::attachTable(ContextPtr /* context_ */, const String & table_n it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), current_time, table_name); - if (table->isView() == false && table->isRemote() == false - && table->isDictionary() == false && table->isSystemStorage() == false) - CurrentMetrics::add(CurrentMetrics::AttachedTable, 1); + CurrentMetrics::add(CurrentMetrics::AttachedTable, 1); } StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & table_name) @@ -206,9 +204,7 @@ StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & ta cache_expiration_queue.erase(it->second.expiration_iterator); tables_cache.erase(it); - if (res->isView() == false && res->isRemote() == false - && res->isDictionary() == false && res->isSystemStorage() == false) - CurrentMetrics::sub(CurrentMetrics::AttachedTable, 1); + CurrentMetrics::sub(CurrentMetrics::AttachedTable, 1); } return res; } diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index fd38a31da5ce..8c509a1dd53f 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -260,7 +260,9 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n res = it->second; tables.erase(it); res->is_detached = true; - CurrentMetrics::sub(getAttachedCounterForStorage(res), 1); + + if (res->isSystemStorage() == false) + CurrentMetrics::sub(getAttachedCounterForStorage(res), 1); auto table_id = res->getStorageID(); if (table_id.hasUUID()) @@ -301,7 +303,9 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c /// It is important to reset is_detached here since in case of RENAME in /// non-Atomic database the is_detached is set to true before RENAME. table->is_detached = false; - CurrentMetrics::add(getAttachedCounterForStorage(table), 1); + + if (res->isSystemStorage() == false) + CurrentMetrics::add(getAttachedCounterForStorage(table), 1); } void DatabaseWithOwnTablesBase::shutdown() From 0cdcc0a704a83f65f577cd3fceb9fae0b50ec7b7 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 4 Jun 2024 15:11:32 +0800 Subject: [PATCH 143/254] modify DatabaseCommon.cpp compile errors; rename testcases, remove test result file --- src/Databases/DatabasesCommon.cpp | 2 +- ...base_and_table_count_limitation_reference} | 0 ...5_database_and_table_count_limitation_sql} | 0 ...and_table_count_limitation_test.reference} | 0 ...tabase_and_table_count_limitation_test.sh} | 6 +++--- ...database_and_table_count_limitation_result | 20 ------------------- 6 files changed, 4 insertions(+), 24 deletions(-) rename tests/queries/0_stateless/{31156_database_and_table_count_limitation_reference => 03165_database_and_table_count_limitation_reference} (100%) rename tests/queries/0_stateless/{31156_database_and_table_count_limitation_sql => 03165_database_and_table_count_limitation_sql} (100%) rename tests/queries/0_stateless/{31156_database_and_table_count_limitation_test.reference => 03165_database_and_table_count_limitation_test.reference} (100%) rename tests/queries/0_stateless/{31156_database_and_table_count_limitation_test.sh => 03165_database_and_table_count_limitation_test.sh} (95%) delete mode 100644 tests/queries/0_stateless/31156_database_and_table_count_limitation_result diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 8c509a1dd53f..c590486d10e2 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -304,7 +304,7 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c /// non-Atomic database the is_detached is set to true before RENAME. table->is_detached = false; - if (res->isSystemStorage() == false) + if (table->isSystemStorage() == false) CurrentMetrics::add(getAttachedCounterForStorage(table), 1); } diff --git a/tests/queries/0_stateless/31156_database_and_table_count_limitation_reference b/tests/queries/0_stateless/03165_database_and_table_count_limitation_reference similarity index 100% rename from tests/queries/0_stateless/31156_database_and_table_count_limitation_reference rename to tests/queries/0_stateless/03165_database_and_table_count_limitation_reference diff --git a/tests/queries/0_stateless/31156_database_and_table_count_limitation_sql b/tests/queries/0_stateless/03165_database_and_table_count_limitation_sql similarity index 100% rename from tests/queries/0_stateless/31156_database_and_table_count_limitation_sql rename to tests/queries/0_stateless/03165_database_and_table_count_limitation_sql diff --git a/tests/queries/0_stateless/31156_database_and_table_count_limitation_test.reference b/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.reference similarity index 100% rename from tests/queries/0_stateless/31156_database_and_table_count_limitation_test.reference rename to tests/queries/0_stateless/03165_database_and_table_count_limitation_test.reference diff --git a/tests/queries/0_stateless/31156_database_and_table_count_limitation_test.sh b/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh similarity index 95% rename from tests/queries/0_stateless/31156_database_and_table_count_limitation_test.sh rename to tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh index 365fad5e99d9..f665c7ce4bf7 100755 --- a/tests/queries/0_stateless/31156_database_and_table_count_limitation_test.sh +++ b/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh @@ -121,9 +121,9 @@ PORT=$(get_server_port "$SERVER_PID") wait_server_port "$PORT" # check result -sql_file="$CUR_DIR/31156_database_and_table_count_limitation_sql" -result_file="$CUR_DIR/31156_database_and_table_count_limitation_result" -reference_file="$CUR_DIR/31156_database_and_table_count_limitation_reference" +sql_file="$CUR_DIR/03165_database_and_table_count_limitation_sql" +result_file="$CUR_DIR/03165_database_and_table_count_limitation_result" +reference_file="$CUR_DIR/03165_database_and_table_count_limitation_reference" $CLICKHOUSE_CLIENT --host 127.1 --port "$PORT" --multiquery --ignore-error --queries-file=$sql_file 2>/dev/null > "$result_file" diff --git a/tests/queries/0_stateless/31156_database_and_table_count_limitation_result b/tests/queries/0_stateless/31156_database_and_table_count_limitation_result deleted file mode 100644 index e0d61993b118..000000000000 --- a/tests/queries/0_stateless/31156_database_and_table_count_limitation_result +++ /dev/null @@ -1,20 +0,0 @@ -INFORMATION_SCHEMA -db1 -db2 -db3 -db4 -db5 -db6 -default -information_schema -system -table1 -table10 -table2 -table3 -table4 -table5 -table6 -table7 -table8 -table9 From 06239e115663062508c46cc32590416e5943515d Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 4 Jun 2024 16:42:14 +0800 Subject: [PATCH 144/254] fix typo --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index b0351a004a49..30d9ddb93f50 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -148,7 +148,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) if (auto max_db = getContext()->getGlobalContext()->getServerSettings().max_database_num_to_throw; max_db > 0) { size_t db_count = DatabaseCatalog::instance().getDatabases().size(); - // there's an invisiable system database _temporary_and_external_tables, so we need to subtract 1 + // there's an invisible system database _temporary_and_external_tables, so we need to subtract 1 if (db_count > 0) db_count--; if (db_count >= max_db) From 446cc0542bdc46af03205740daa004800ee343b9 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Wed, 5 Jun 2024 17:17:29 +0800 Subject: [PATCH 145/254] add a space to trigger CI after fixed a CI bug --- .../03165_database_and_table_count_limitation_test.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh b/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh index f665c7ce4bf7..440173393d07 100755 --- a/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh +++ b/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh @@ -144,7 +144,7 @@ if [ $database_error -ne 2 ]; then exit 1 fi -#table_error should be 1 +# table_error should be 1 if [ $table_error -ne 1 ]; then echo "table_error should be 1, but now -s $table_error. Tried to create 11 tables, 10 should be created and 1 should fail" echo "Limitation is 10 tables" From a2fd206a4dd729321dc318e4edb921bc379dbe13 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Mon, 10 Jun 2024 16:45:38 +0800 Subject: [PATCH 146/254] adjust database count checking DatabaseCatalog::TEMPORARY_DATABASE explicitly, change 'yandex' into 'clickhouse' in config.xml --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- .../03165_database_and_table_count_limitation_test.sh | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 30d9ddb93f50..1904dd58d9f8 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -149,7 +149,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) { size_t db_count = DatabaseCatalog::instance().getDatabases().size(); // there's an invisible system database _temporary_and_external_tables, so we need to subtract 1 - if (db_count > 0) + if (db_count > 0 && DatabaseCatalog::instance().isDatabaseExist(DatabaseCatalog::TEMPORARY_DATABASE)) db_count--; if (db_count >= max_db) throw Exception(ErrorCodes::TOO_MANY_DATABASES, diff --git a/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh b/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh index 440173393d07..c0b0639dec42 100755 --- a/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh +++ b/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh @@ -19,7 +19,7 @@ TEMP_DIR=$(mktemp -d "$CUR_DIR/$(basename "$BASE").XXXXXX") # Generate config.xml CONFIG_FILE="$TEMP_DIR/config.xml" cat > "$CONFIG_FILE" < + information $TEMP_DIR/clickhouse-server.log @@ -35,7 +35,7 @@ cat > "$CONFIG_FILE" < - + EOL echo " From 5ff65a1bf546403883aecd824e9ceb2302eea040 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 10 Jun 2024 20:46:13 +0000 Subject: [PATCH 147/254] write integration tests --- .../test_table_db_num_limit/config/config.xml | 5 + .../test_table_db_num_limit/test.py | 34 ++++ ...abase_and_table_count_limitation_reference | 20 --- ...65_database_and_table_count_limitation_sql | 21 --- ..._and_table_count_limitation_test.reference | 1 - ...atabase_and_table_count_limitation_test.sh | 154 ------------------ 6 files changed, 39 insertions(+), 196 deletions(-) create mode 100644 tests/integration/test_table_db_num_limit/config/config.xml create mode 100644 tests/integration/test_table_db_num_limit/test.py delete mode 100644 tests/queries/0_stateless/03165_database_and_table_count_limitation_reference delete mode 100644 tests/queries/0_stateless/03165_database_and_table_count_limitation_sql delete mode 100644 tests/queries/0_stateless/03165_database_and_table_count_limitation_test.reference delete mode 100755 tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh diff --git a/tests/integration/test_table_db_num_limit/config/config.xml b/tests/integration/test_table_db_num_limit/config/config.xml new file mode 100644 index 000000000000..9a573b158feb --- /dev/null +++ b/tests/integration/test_table_db_num_limit/config/config.xml @@ -0,0 +1,5 @@ + + 10 + 10 + + diff --git a/tests/integration/test_table_db_num_limit/test.py b/tests/integration/test_table_db_num_limit/test.py new file mode 100644 index 000000000000..d5bd22287995 --- /dev/null +++ b/tests/integration/test_table_db_num_limit/test.py @@ -0,0 +1,34 @@ +import pytest +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance( + "node1", main_configs=["config/config.xml"], with_zookeeper=True +) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def test_table_db_limit(started_cluster): + for i in range(6): + node1.query("create database db{}".format(i)) + + with pytest.raises(QueryRuntimeException) as exp_info: + node1.query("create database db_exp".format(i)) + + assert "TOO_MANY_DATABASES" in str(exp_info) + for i in range(10): + node1.query("create table t{} (a Int32) Engine = Log".format(i)) + + with pytest.raises(QueryRuntimeException) as exp_info: + node1.query("create table default.tx (a Int32) Engine = Log") + assert "TOO_MANY_TABLES" in str(exp_info) diff --git a/tests/queries/0_stateless/03165_database_and_table_count_limitation_reference b/tests/queries/0_stateless/03165_database_and_table_count_limitation_reference deleted file mode 100644 index e0d61993b118..000000000000 --- a/tests/queries/0_stateless/03165_database_and_table_count_limitation_reference +++ /dev/null @@ -1,20 +0,0 @@ -INFORMATION_SCHEMA -db1 -db2 -db3 -db4 -db5 -db6 -default -information_schema -system -table1 -table10 -table2 -table3 -table4 -table5 -table6 -table7 -table8 -table9 diff --git a/tests/queries/0_stateless/03165_database_and_table_count_limitation_sql b/tests/queries/0_stateless/03165_database_and_table_count_limitation_sql deleted file mode 100644 index 127153956f98..000000000000 --- a/tests/queries/0_stateless/03165_database_and_table_count_limitation_sql +++ /dev/null @@ -1,21 +0,0 @@ -create database db1; -create database db2; -create database db3; -create database db4; -create database db5; -create database db6; -create database db7; -create database db8; -select name from system.databases; -create table db1.table1 (x UInt64) engine = Memory; -create table db1.table2 (x UInt64) engine = Memory; -create table db1.table3 (x UInt64) engine = Memory; -create table db1.table4 (x UInt64) engine = Memory; -create table db1.table5 (x UInt64) engine = Memory; -create table db1.table6 (x UInt64) engine = Memory; -create table db1.table7 (x UInt64) engine = Memory; -create table db1.table8 (x UInt64) engine = Memory; -create table db1.table9 (x UInt64) engine = Memory; -create table db1.table10 (x UInt64) engine = Memory; -create table db1.table11 (x UInt64) engine = Memory; -select name from system.tables where database != 'system' and database != 'information_schema' and database != 'INFORMATION_SCHEMA'; \ No newline at end of file diff --git a/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.reference b/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.reference deleted file mode 100644 index 386f3d2fd86e..000000000000 --- a/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.reference +++ /dev/null @@ -1 +0,0 @@ -Test passed diff --git a/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh b/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh deleted file mode 100755 index c0b0639dec42..000000000000 --- a/tests/queries/0_stateless/03165_database_and_table_count_limitation_test.sh +++ /dev/null @@ -1,154 +0,0 @@ -#!/usr/bin/env bash - -#Tags: shared, no-parallel - -# Get the current script directory -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -BASE="$CUR_DIR/$(basename "${BASH_SOURCE[0]}" .sh)" - -# Load shell_config.sh -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -# remove --database=$CLICKHOUSE_DATABASE from $CLICKHOUSE_CLIENT -CLICKHOUSE_CLIENT=$(echo $CLICKHOUSE_CLIENT | sed "s/--database=$CLICKHOUSE_DATABASE//") - -# Create a temporary directory -TEMP_DIR=$(mktemp -d "$CUR_DIR/$(basename "$BASE").XXXXXX") - -# Generate config.xml -CONFIG_FILE="$TEMP_DIR/config.xml" -cat > "$CONFIG_FILE" < - - information - $TEMP_DIR/clickhouse-server.log - $TEMP_DIR/clickhouse-server.err.log - - 10 - 10 - - - - - users.xml - - - - -EOL - -echo " - - - - - - - - ::/0 - - default - default - - - - - - " > $TEMP_DIR/users.xml - -# Function to start the server -function start_server() { - local server_opts=( - "--config-file=$CONFIG_FILE" - "--" - "--tcp_port=0" - "--shutdown_wait_unfinished=0" - "--listen_host=127.1" - "--path=$TEMP_DIR" - ) - CLICKHOUSE_WATCHDOG_ENABLE=0 $CLICKHOUSE_SERVER_BINARY "${server_opts[@]}" > /dev/null 2>&1 & - local pid=$! - - echo "$pid" -} - -# Function to get the server port -function get_server_port() { - local pid=$1 && shift - local port='' - while [[ -z $port ]]; do - port=$(lsof -n -a -P -i tcp -s tcp:LISTEN -p "$pid" 2>/dev/null | awk -F'[ :]' '/LISTEN/ { print $(NF-1) }') - sleep 0.5 - done - echo "$port" -} - -# Function to wait for the server port to be available -function wait_server_port() { - local port=$1 && shift - local i=0 retries=30 - while ! $CLICKHOUSE_CLIENT --host 127.1 --port "$port" --format Null -q 'select 1' 2>/dev/null && [[ $i -lt $retries ]]; do - sleep 0.5 - ((i++)) - done - if ! $CLICKHOUSE_CLIENT --host 127.1 --port "$port" --format Null -q 'select 1'; then - echo "Cannot wait until server will start accepting connections on port $port" >&2 - exit 1 - fi -} - -# Function to stop the server -function stop_server() { - if [[ -n "$SERVER_PID" ]]; then - kill -9 "$SERVER_PID" - fi -} - -# Function to clean up -function cleanup() { - stop_server - rm -rf "$TEMP_DIR" -} - -trap cleanup EXIT - -# Start the server and get the port -SERVER_PID=$(start_server) -PORT=$(get_server_port "$SERVER_PID") - -# Wait for the server to start -wait_server_port "$PORT" - -# check result -sql_file="$CUR_DIR/03165_database_and_table_count_limitation_sql" -result_file="$CUR_DIR/03165_database_and_table_count_limitation_result" -reference_file="$CUR_DIR/03165_database_and_table_count_limitation_reference" - -$CLICKHOUSE_CLIENT --host 127.1 --port "$PORT" --multiquery --ignore-error --queries-file=$sql_file 2>/dev/null > "$result_file" - -# Compare the result with the reference, if not same, print the diff -if ! diff -u "$reference_file" "$result_file"; then - echo "Test failed" - exit 1 -fi - -# check errors in error log -log_file="$TEMP_DIR/clickhouse-server.err.log" -database_error=$(grep -c " executeQuery: Code: 725. DB::Exception: Too many databases" $log_file) -table_error=$(grep -c " executeQuery: Code: 724. DB::Exception: Too many tables" $log_file) -#database_error should be 2 -if [ $database_error -ne 2 ]; then - echo "database_error should be 2, but now is $database_error. Tried to create 8 db, 6 should be created and 2 should fail" - echo "Limitation is 10 databases, 4 exist by default: default, system, information_schema, INFORMATION_SCHEMA" - exit 1 -fi - -# table_error should be 1 -if [ $table_error -ne 1 ]; then - echo "table_error should be 1, but now -s $table_error. Tried to create 11 tables, 10 should be created and 1 should fail" - echo "Limitation is 10 tables" - exit 1 -fi - -echo "Test passed" From f073a456dc02446bc71a2b892ab10eb258418459 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 10 Jun 2024 21:02:34 +0000 Subject: [PATCH 148/254] fix style --- tests/integration/test_table_db_num_limit/__init__.py | 0 tests/integration/test_table_db_num_limit/test.py | 2 ++ 2 files changed, 2 insertions(+) create mode 100644 tests/integration/test_table_db_num_limit/__init__.py diff --git a/tests/integration/test_table_db_num_limit/__init__.py b/tests/integration/test_table_db_num_limit/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_table_db_num_limit/test.py b/tests/integration/test_table_db_num_limit/test.py index d5bd22287995..71bb494e04e8 100644 --- a/tests/integration/test_table_db_num_limit/test.py +++ b/tests/integration/test_table_db_num_limit/test.py @@ -8,6 +8,7 @@ "node1", main_configs=["config/config.xml"], with_zookeeper=True ) + @pytest.fixture(scope="module") def started_cluster(): try: @@ -18,6 +19,7 @@ def started_cluster(): finally: cluster.shutdown() + def test_table_db_limit(started_cluster): for i in range(6): node1.query("create database db{}".format(i)) From 76d6bc37eb7b3b90be0bb7b6a509f6e3e32655db Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Wed, 12 Jun 2024 21:51:32 +0800 Subject: [PATCH 149/254] adjust code based on review comment --- src/Interpreters/DatabaseCatalog.h | 1 + src/Interpreters/InterpreterCreateQuery.cpp | 38 +++++++++++++------ .../test_table_db_num_limit/test.py | 2 +- 3 files changed, 28 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 37125d9900c1..ac9c01e5dd70 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -129,6 +129,7 @@ class DatabaseCatalog : boost::noncopyable, WithMutableContext static constexpr const char * SYSTEM_DATABASE = "system"; static constexpr const char * INFORMATION_SCHEMA = "information_schema"; static constexpr const char * INFORMATION_SCHEMA_UPPERCASE = "INFORMATION_SCHEMA"; + static constexpr const char * DEFAULT_DATABASE = "default"; /// Returns true if a passed name is one of the predefined databases' names. static bool isPredefinedDatabase(std::string_view database_name); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 1904dd58d9f8..697e8136ee1e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -145,16 +145,29 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) throw Exception(ErrorCodes::DATABASE_ALREADY_EXISTS, "Database {} already exists.", database_name); } - if (auto max_db = getContext()->getGlobalContext()->getServerSettings().max_database_num_to_throw; max_db > 0) + auto db_num_limit = getContext()->getGlobalContext()->getServerSettings().max_database_num_to_throw; + if (db_num_limit > 0) { size_t db_count = DatabaseCatalog::instance().getDatabases().size(); - // there's an invisible system database _temporary_and_external_tables, so we need to subtract 1 - if (db_count > 0 && DatabaseCatalog::instance().isDatabaseExist(DatabaseCatalog::TEMPORARY_DATABASE)) - db_count--; - if (db_count >= max_db) + std::vector system_databases = { + DatabaseCatalog::TEMPORARY_DATABASE, + DatabaseCatalog::SYSTEM_DATABASE, + DatabaseCatalog::INFORMATION_SCHEMA, + DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE, + DatabaseCatalog::DEFAULT_DATABASE + }; + + for (const auto & system_database : system_databases) + { + if (db_count > 0 && DatabaseCatalog::instance().isDatabaseExist(system_database)) + db_count--; + } + + if (db_count >= db_num_limit) throw Exception(ErrorCodes::TOO_MANY_DATABASES, - "Too many databases, max: {}, now: {}. " - "See setting max_database_num_to_throw.", max_db, db_count); + "Too many databases in the Clickhouse. " + "The limit (setting 'max_database_num_to_throw') is set to {}, currnt number of databases is {}", + db_num_limit, db_count); } /// Will write file with database metadata, if needed. @@ -1563,14 +1576,15 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, } } - if (UInt64 max_table = getContext()->getGlobalContext()->getServerSettings().max_table_num_to_throw; max_table > 0) + UInt64 table_num_limit = getContext()->getGlobalContext()->getServerSettings().max_table_num_to_throw; + if (table_num_limit > 0) { UInt64 table_count = CurrentMetrics::get(CurrentMetrics::AttachedTable); - if (table_count >= max_table) + if (table_count >= table_num_limit) throw Exception(ErrorCodes::TOO_MANY_TABLES, - "Too many tables in the system. Current is {}, limit is {}. " - "See setting 'max_table_num_to_throw'.", - table_count, max_table); + "Too many tables in the Clickhouse. " + "The limit (setting 'max_table_num_to_throw') is set to {}, currnt number of tables is {}", + table_num_limit, table_count); } database->createTable(getContext(), create.getTable(), res, query_ptr); diff --git a/tests/integration/test_table_db_num_limit/test.py b/tests/integration/test_table_db_num_limit/test.py index 71bb494e04e8..bde9a956d700 100644 --- a/tests/integration/test_table_db_num_limit/test.py +++ b/tests/integration/test_table_db_num_limit/test.py @@ -21,7 +21,7 @@ def started_cluster(): def test_table_db_limit(started_cluster): - for i in range(6): + for i in range(10): node1.query("create database db{}".format(i)) with pytest.raises(QueryRuntimeException) as exp_info: From 555e166ed6d2cb214e42e9502982c07c2baa91b2 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Thu, 13 Jun 2024 08:38:19 +0800 Subject: [PATCH 150/254] fix type --- src/Interpreters/InterpreterCreateQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 697e8136ee1e..c33f97dc80db 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -166,7 +166,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) if (db_count >= db_num_limit) throw Exception(ErrorCodes::TOO_MANY_DATABASES, "Too many databases in the Clickhouse. " - "The limit (setting 'max_database_num_to_throw') is set to {}, currnt number of databases is {}", + "The limit (setting 'max_database_num_to_throw') is set to {}, current number of databases is {}", db_num_limit, db_count); } @@ -1583,7 +1583,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, if (table_count >= table_num_limit) throw Exception(ErrorCodes::TOO_MANY_TABLES, "Too many tables in the Clickhouse. " - "The limit (setting 'max_table_num_to_throw') is set to {}, currnt number of tables is {}", + "The limit (setting 'max_table_num_to_throw') is set to {}, current number of tables is {}", table_num_limit, table_count); } From d44d5254056c5266235c0b94b0622d298e57e917 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Thu, 13 Jun 2024 22:38:47 +0800 Subject: [PATCH 151/254] trigger CI From 08b481af3d8d59088216df5bc182871ef73284ff Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Fri, 14 Jun 2024 09:44:08 +0800 Subject: [PATCH 152/254] trigger CI From 1944e0ae5f4346dba2ed2aebdad2951855985708 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Fri, 14 Jun 2024 10:44:14 +0300 Subject: [PATCH 153/254] empty lines --- .../0_stateless/03171_indexing_by_hilbert_curve.reference | 2 +- tests/queries/0_stateless/03171_indexing_by_hilbert_curve.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.reference b/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.reference index a223737eae65..6e8a5df9145f 100644 --- a/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.reference +++ b/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.reference @@ -6,4 +6,4 @@ 32 22 11 -1 \ No newline at end of file +1 diff --git a/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.sql b/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.sql index 590f979efb2c..2d566e52c941 100644 --- a/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.sql +++ b/tests/queries/0_stateless/03171_indexing_by_hilbert_curve.sql @@ -32,4 +32,4 @@ SELECT count() FROM test_hilbert_encode WHERE y = 10 AND x <= 10 SETTINGS max_ro SELECT count() FROM test_hilbert_encode PREWHERE x >= 10 WHERE x < 11 AND y = 10 SETTINGS max_rows_to_read = 2; -DROP TABLE test_hilbert_encode; \ No newline at end of file +DROP TABLE test_hilbert_encode; From 840df04f9064874d207fc73b71ee8db829989425 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 14 Jun 2024 10:08:50 +0200 Subject: [PATCH 154/254] Restart CI From 57e84c0344119d1628eed1a5e6f048cae0505b5c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 Jun 2024 09:40:11 +0000 Subject: [PATCH 155/254] Fixing style. --- src/Interpreters/InterpreterSelectQuery.cpp | 1 - src/Planner/Planner.cpp | 3 +-- src/Planner/PlannerJoinTree.cpp | 3 +-- 3 files changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 51d602b0bc86..68967f331f04 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -124,7 +124,6 @@ namespace ErrorCodes extern const int UNKNOWN_IDENTIFIER; extern const int BAD_ARGUMENTS; extern const int SUPPORT_IS_DISABLED; - extern const int NOT_FOUND_COLUMN_IN_BLOCK; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 8e8183c07626..5496a7e42574 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -99,7 +99,6 @@ namespace ErrorCodes extern const int TOO_DEEP_SUBQUERIES; extern const int NOT_IMPLEMENTED; extern const int SUPPORT_IS_DISABLED; - extern const int NOT_FOUND_COLUMN_IN_BLOCK; } namespace @@ -336,7 +335,7 @@ void addExpressionStep(QueryPlan & query_plan, { auto actions = expression_actions->actions.clone(); if (expression_actions->project_input) - actions->appendInputsForUnusedColumns( query_plan.getCurrentDataStream().header); + actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); result_actions_to_execute.push_back(actions); auto expression_step = std::make_unique(query_plan.getCurrentDataStream(), actions); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index fc3cd16f8ede..43b223172e66 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -81,7 +81,6 @@ namespace ErrorCodes extern const int TOO_MANY_COLUMNS; extern const int UNSUPPORTED_METHOD; extern const int BAD_ARGUMENTS; - extern const int NOT_FOUND_COLUMN_IN_BLOCK; } namespace @@ -1133,7 +1132,7 @@ void joinCastPlanColumnsToNullable(QueryPlan & plan_to_add_cast, PlannerContextP } } - cast_actions_dag->appendInputsForUnusedColumns( plan_to_add_cast.getCurrentDataStream().header); + cast_actions_dag->appendInputsForUnusedColumns(plan_to_add_cast.getCurrentDataStream().header); auto cast_join_columns_step = std::make_unique(plan_to_add_cast.getCurrentDataStream(), std::move(cast_actions_dag)); cast_join_columns_step->setStepDescription("Cast JOIN columns to Nullable"); plan_to_add_cast.addStep(std::move(cast_join_columns_step)); From 6e3199ebea73a8b2768f487171fbe0263d4ce0a0 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 10 Jun 2024 11:18:03 +0200 Subject: [PATCH 156/254] CI: ci_config refactoring --- .github/PULL_REQUEST_TEMPLATE.md | 3 +- .github/workflows/master.yml | 15 +- .github/workflows/pull_request.yml | 19 +- tests/ci/artifacts_helper.py | 4 +- tests/ci/ast_fuzzer_check.py | 5 +- tests/ci/bugfix_validate_check.py | 15 +- tests/ci/build_check.py | 9 +- tests/ci/build_download_helper.py | 10 +- tests/ci/build_report_check.py | 39 +- tests/ci/ci.py | 72 +- tests/ci/ci_cache.py | 71 +- tests/ci/ci_config.py | 2029 +++++------------ tests/ci/ci_definitions.py | 754 ++++++ tests/ci/ci_settings.py | 30 +- tests/ci/commit_status_helper.py | 34 +- tests/ci/digest_helper.py | 11 +- tests/ci/download_binary.py | 5 +- tests/ci/env_helper.py | 3 +- tests/ci/finish_check.py | 12 +- tests/ci/integration_tests_runner.py | 6 +- tests/ci/jepsen_check.py | 4 +- tests/ci/performance_comparison_check.py | 4 +- tests/ci/pr_info.py | 12 +- tests/ci/report.py | 9 +- tests/ci/run_check.py | 4 +- tests/ci/s3_helper.py | 14 +- tests/ci/sqlancer_check.py | 5 +- tests/ci/sqltest.py | 5 +- tests/ci/ssh.py | 6 +- tests/ci/style_check.py | 4 +- tests/ci/sync_pr.py | 4 +- tests/ci/test_ci_cache.py | 40 +- tests/ci/test_ci_config.py | 314 ++- tests/ci/test_ci_options.py | 36 +- utils/check-style/check_py.sh | 10 +- .../check-style/process_style_check_result.py | 2 +- 36 files changed, 1852 insertions(+), 1767 deletions(-) create mode 100644 tests/ci/ci_definitions.py diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 73b2155482ee..74a6f95dbb34 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -48,8 +48,7 @@ At a minimum, the following information should be added (but add more as needed) - [ ] Allow: Stateful tests - [ ] Allow: Integration Tests - [ ] Allow: Performance tests -- [ ] Allow: Normal Builds -- [ ] Allow: Special Builds +- [ ] Allow: All Builds - [ ] Allow: All NOT Required Checks - [ ] Allow: batch 1, 2 for multi-batch jobs - [ ] Allow: batch 3, 4, 5, 6 for multi-batch jobs diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index f5c78a6b6a1d..837dbba61743 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -115,8 +115,8 @@ jobs: data: ${{ needs.RunConfig.outputs.data }} ################################# Reports ################################# - # Reports should be run even if Builds_1/2 failed - put them separately in wf (not in Tests_1/2) - Builds_1_Report: + # Reports should run even if Builds_1/2 fail - run them separately, not in Tests_1/2/3 + Builds_Report: # run report check for failed builds to indicate the CI error if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} needs: [RunConfig, Builds_1] @@ -125,15 +125,6 @@ jobs: test_name: ClickHouse build check runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} - Builds_2_Report: - # run report check for failed builds to indicate the CI error - if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse special build check') }} - needs: [RunConfig, Builds_2] - uses: ./.github/workflows/reusable_test.yml - with: - test_name: ClickHouse special build check - runner_type: style-checker-aarch64 - data: ${{ needs.RunConfig.outputs.data }} MarkReleaseReady: if: ${{ !failure() && !cancelled() }} @@ -165,7 +156,7 @@ jobs: FinishCheck: if: ${{ !cancelled() }} - needs: [RunConfig, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2, Tests_3] + needs: [RunConfig, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 66ca3381a404..422bcf551ae4 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -143,29 +143,20 @@ jobs: data: ${{ needs.RunConfig.outputs.data }} ################################# Reports ################################# - # Reports should by run even if Builds_1/2 fail, so put them separately in wf (not in Tests_1/2) - Builds_1_Report: + # Reports should run even if Builds_1/2 fail - run them separately (not in Tests_1/2/3) + Builds_Report: # run report check for failed builds to indicate the CI error - if: ${{ !cancelled() && needs.StyleCheck.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} + if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} needs: [RunConfig, StyleCheck, Builds_1] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickHouse build check runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} - Builds_2_Report: - # run report check for failed builds to indicate the CI error - if: ${{ !cancelled() && needs.StyleCheck.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse special build check') }} - needs: [RunConfig, StyleCheck, Builds_2] - uses: ./.github/workflows/reusable_test.yml - with: - test_name: ClickHouse special build check - runner_type: style-checker-aarch64 - data: ${{ needs.RunConfig.outputs.data }} CheckReadyForMerge: if: ${{ !cancelled() && needs.StyleCheck.result == 'success' }} - needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2] + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code @@ -181,7 +172,7 @@ jobs: # FinishCheck: if: ${{ !cancelled() }} - needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2, Tests_3] + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2, Tests_3] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code diff --git a/tests/ci/artifacts_helper.py b/tests/ci/artifacts_helper.py index 37abf0bdefb5..503ba2e1ec47 100644 --- a/tests/ci/artifacts_helper.py +++ b/tests/ci/artifacts_helper.py @@ -15,7 +15,7 @@ from build_download_helper import download_build_with_progress from commit_status_helper import post_commit_status from compress_files import SUFFIX, compress_fast, decompress_fast -from env_helper import CI, RUNNER_TEMP, S3_BUILDS_BUCKET +from env_helper import IS_CI, RUNNER_TEMP, S3_BUILDS_BUCKET from git_helper import SHA_REGEXP from report import FOOTER_HTML_TEMPLATE, HEAD_HTML_TEMPLATE, SUCCESS from s3_helper import S3Helper @@ -131,7 +131,7 @@ def post_commit_status(commit: Commit, url: str) -> None: post_commit_status(commit, SUCCESS, url, "Artifacts for workflow", "Artifacts") def _regenerate_index(self) -> None: - if CI: + if IS_CI: files = self._get_s3_objects() else: files = self._get_local_s3_objects() diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index b88a9476a6d9..8bc0f51dfc73 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -6,7 +6,7 @@ import sys from pathlib import Path -from build_download_helper import get_build_name_for_check, read_build_urls +from build_download_helper import read_build_urls from clickhouse_helper import CiLogsCredentials from docker_images_helper import DockerImage, get_docker_image, pull_image from env_helper import REPORT_PATH, TEMP_PATH @@ -14,6 +14,7 @@ from report import FAIL, FAILURE, OK, SUCCESS, JobReport, TestResult from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_config import CI IMAGE_NAME = "clickhouse/fuzzer" @@ -64,7 +65,7 @@ def main(): docker_image = pull_image(get_docker_image(IMAGE_NAME)) - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) urls = read_build_urls(build_name, reports_path) if not urls: raise ValueError("No build URLs found") diff --git a/tests/ci/bugfix_validate_check.py b/tests/ci/bugfix_validate_check.py index d41fdaf05ffe..71b185729380 100644 --- a/tests/ci/bugfix_validate_check.py +++ b/tests/ci/bugfix_validate_check.py @@ -7,7 +7,7 @@ from pathlib import Path from typing import List, Sequence, Tuple -from ci_config import JobNames +from ci_config import CI from ci_utils import normalize_string from env_helper import TEMP_PATH from functional_test_check import NO_CHANGES_MSG @@ -92,16 +92,19 @@ def main(): logging.basicConfig(level=logging.INFO) # args = parse_args() stopwatch = Stopwatch() - jobs_to_validate = [JobNames.STATELESS_TEST_RELEASE, JobNames.INTEGRATION_TEST] + jobs_to_validate = [ + CI.JobNames.STATELESS_TEST_RELEASE, + CI.JobNames.INTEGRATION_TEST, + ] functional_job_report_file = Path(TEMP_PATH) / "functional_test_job_report.json" integration_job_report_file = Path(TEMP_PATH) / "integration_test_job_report.json" jobs_report_files = { - JobNames.STATELESS_TEST_RELEASE: functional_job_report_file, - JobNames.INTEGRATION_TEST: integration_job_report_file, + CI.JobNames.STATELESS_TEST_RELEASE: functional_job_report_file, + CI.JobNames.INTEGRATION_TEST: integration_job_report_file, } jobs_scripts = { - JobNames.STATELESS_TEST_RELEASE: "functional_test_check.py", - JobNames.INTEGRATION_TEST: "integration_test_check.py", + CI.JobNames.STATELESS_TEST_RELEASE: "functional_test_check.py", + CI.JobNames.INTEGRATION_TEST: "integration_test_check.py", } for test_job in jobs_to_validate: diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 260b77b0ee50..39f34ed9ccfb 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -9,7 +9,7 @@ from typing import Tuple import docker_images_helper -from ci_config import CI_CONFIG, BuildConfig +from ci_config import CI from env_helper import REPO_COPY, S3_BUILDS_BUCKET, TEMP_PATH from git_helper import Git from lambda_shared_package.lambda_shared.pr import Labels @@ -27,7 +27,7 @@ BUILD_LOG_NAME = "build_log.log" -def _can_export_binaries(build_config: BuildConfig) -> bool: +def _can_export_binaries(build_config: CI.BuildConfig) -> bool: if build_config.package_type != "deb": return False if build_config.sanitizer != "": @@ -38,7 +38,7 @@ def _can_export_binaries(build_config: BuildConfig) -> bool: def get_packager_cmd( - build_config: BuildConfig, + build_config: CI.BuildConfig, packager_path: Path, output_path: Path, build_version: str, @@ -147,7 +147,8 @@ def main(): stopwatch = Stopwatch() build_name = args.build_name - build_config = CI_CONFIG.build_config[build_name] + build_config = CI.JOB_CONFIGS[build_name].build_config + assert build_config temp_path = Path(TEMP_PATH) temp_path.mkdir(parents=True, exist_ok=True) diff --git a/tests/ci/build_download_helper.py b/tests/ci/build_download_helper.py index a641f9f4544d..8482abb26e0a 100644 --- a/tests/ci/build_download_helper.py +++ b/tests/ci/build_download_helper.py @@ -10,7 +10,7 @@ import requests -from ci_config import CI_CONFIG +from ci_config import CI try: # A work around for scripts using this downloading module without required deps @@ -122,10 +122,6 @@ def set_auth_header(): raise APIException(f"Unable to request data from GH API: {url}") from exc -def get_build_name_for_check(check_name: str) -> str: - return CI_CONFIG.test_configs[check_name].required_build - - def read_build_urls(build_name: str, reports_path: Union[Path, str]) -> List[str]: for root, _, files in os.walk(reports_path): for file in files: @@ -210,7 +206,7 @@ def download_builds_filter( result_path: Path, filter_fn: Callable[[str], bool] = lambda _: True, ) -> None: - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) urls = read_build_urls(build_name, reports_path) logger.info("The build report for %s contains the next URLs: %s", build_name, urls) @@ -247,7 +243,7 @@ def download_clickhouse_binary( def get_clickhouse_binary_url( check_name: str, reports_path: Union[Path, str] ) -> Optional[str]: - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) urls = read_build_urls(build_name, reports_path) logger.info("The build report for %s contains the next URLs: %s", build_name, urls) for url in urls: diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 1d734fbb3f8f..5f24fd8f2242 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -1,4 +1,5 @@ #!/usr/bin/env python3 +import argparse import json import logging import os @@ -6,7 +7,6 @@ from pathlib import Path from typing import List -from ci_config import CI_CONFIG, Build from env_helper import ( GITHUB_JOB_URL, GITHUB_REPOSITORY, @@ -14,7 +14,7 @@ REPORT_PATH, TEMP_PATH, CI_CONFIG_PATH, - CI, + IS_CI, ) from pr_info import PRInfo from report import ( @@ -27,6 +27,7 @@ get_worst_status, ) from stopwatch import Stopwatch +from ci_config import CI # Old way to read the neads_data NEEDS_DATA_PATH = os.getenv("NEEDS_DATA_PATH", "") @@ -46,16 +47,13 @@ def main(): "\n ".join(p.as_posix() for p in reports_path.rglob("*.json")), ) - build_check_name = sys.argv[1] + build_check_name = CI.JobNames.BUILD_CHECK pr_info = PRInfo() - builds_for_check = CI_CONFIG.get_builds_for_report( - build_check_name, - release=pr_info.is_release, - backport=pr_info.head_ref.startswith("backport/"), - ) - if CI: + args = parse_args() + + if (CI_CONFIG_PATH or IS_CI) and not args.reports: # In CI only specific builds might be manually selected, or some wf does not build all builds. # Filtering @builds_for_check to verify only builds that are present in the current CI workflow with open(CI_CONFIG_PATH, encoding="utf-8") as jfd: @@ -64,8 +62,12 @@ def main(): ci_config["jobs_data"]["jobs_to_skip"] + ci_config["jobs_data"]["jobs_to_do"] ) - builds_for_check = [job for job in builds_for_check if job in all_ci_jobs] - print(f"NOTE: following build reports will be accounted: [{builds_for_check}]") + builds_for_check = [job for job in CI.BuildNames if job in all_ci_jobs] + print(f"NOTE: following build reports will be checked: [{builds_for_check}]") + else: + builds_for_check = parse_args().reports + for job in builds_for_check: + assert job in CI.BuildNames, "Builds must be known build job names" required_builds = len(builds_for_check) missing_builds = 0 @@ -77,8 +79,8 @@ def main(): build_name, pr_info.number, pr_info.head_ref ) if not build_result: - if build_name == Build.FUZZERS: - logging.info("Build [%s] is missing - skip", Build.FUZZERS) + if build_name == CI.BuildNames.FUZZERS: + logging.info("Build [%s] is missing - skip", CI.BuildNames.FUZZERS) continue logging.warning("Build results for %s is missing", build_name) build_result = BuildResult.missing_result("missing") @@ -158,5 +160,16 @@ def main(): sys.exit(1) +def parse_args(): + parser = argparse.ArgumentParser("Generates overall build report") + + parser.add_argument( + "--reports", + nargs="+", + help="List of build reports to check", + ) + return parser.parse_args() + + if __name__ == "__main__": main() diff --git a/tests/ci/ci.py b/tests/ci/ci.py index bb23de142df0..cdf1be96b681 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -13,14 +13,7 @@ import docker_images_helper import upload_result_helper from build_check import get_release_or_pr -from ci_config import ( - CI_CONFIG, - Build, - CILabels, - CIStages, - JobNames, - StatusNames, -) +from ci_config import CI from ci_metadata import CiMetadata from ci_utils import GHActions, normalize_string from clickhouse_helper import ( @@ -41,7 +34,7 @@ ) from digest_helper import DockerDigester from env_helper import ( - CI, + IS_CI, GITHUB_JOB_API_URL, GITHUB_REPOSITORY, GITHUB_RUN_ID, @@ -295,7 +288,7 @@ def _mark_success_action( batch: int, ) -> None: ci_cache = CiCache(s3, indata["jobs_data"]["digests"]) - job_config = CI_CONFIG.get_job_config(job) + job_config = CI.get_job_config(job) num_batches = job_config.num_batches # if batch is not provided - set to 0 batch = 0 if batch == -1 else batch @@ -305,7 +298,7 @@ def _mark_success_action( # FIXME: find generic design for propagating and handling job status (e.g. stop using statuses in GH api) # now job ca be build job w/o status data, any other job that exit with 0 with or w/o status data - if CI_CONFIG.is_build_job(job): + if CI.is_build_job(job): # there is no CommitStatus for build jobs # create dummy status relying on JobReport # FIXME: consider creating commit status for build jobs too, to treat everything the same way @@ -425,6 +418,7 @@ def _configure_jobs( pr_info: PRInfo, ci_settings: CiSettings, skip_jobs: bool, + dry_run: bool = False, ) -> CiCache: """ returns CICache instance with configured job's data @@ -436,10 +430,11 @@ def _configure_jobs( # get all jobs if not skip_jobs: - job_configs = CI_CONFIG.get_workflow_jobs_with_configs( + job_configs = CI.get_workflow_jobs_with_configs( is_mq=pr_info.is_merge_queue, is_docs_only=pr_info.has_changes_in_documentation_only(), is_master=pr_info.is_master, + is_pr=pr_info.is_pr, ) else: job_configs = {} @@ -457,7 +452,8 @@ def _configure_jobs( ci_cache = CiCache.calc_digests_and_create( s3, job_configs, - cache_enabled=not ci_settings.no_ci_cache and not skip_jobs and CI, + cache_enabled=not ci_settings.no_ci_cache and not skip_jobs and IS_CI, + dry_run=dry_run, ) ci_cache.update() ci_cache.apply(job_configs, is_release=pr_info.is_release) @@ -475,14 +471,14 @@ def _generate_ci_stage_config(jobs_data: Dict[str, Any]) -> Dict[str, Dict[str, result = {} # type: Dict[str, Any] stages_to_do = [] for job in jobs_data: - stage_type = CI_CONFIG.get_job_ci_stage(job) - if stage_type == CIStages.NA: + stage_type = CI.get_job_ci_stage(job) + if stage_type == CI.WorkflowStages.NA: continue if stage_type not in result: result[stage_type] = [] stages_to_do.append(stage_type) result[stage_type].append( - {"job_name": job, "runner_type": CI_CONFIG.get_runner_type(job)} + {"job_name": job, "runner_type": CI.JOB_CONFIGS[job].runner_type} ) result["stages_to_do"] = stages_to_do return result @@ -529,10 +525,10 @@ def _concurrent_create_status(job: str, batch: int, num_batches: int) -> None: if job not in jobs_to_skip and job not in jobs_to_do: # no need to create status for job that are not supposed to be executed continue - if CI_CONFIG.is_build_job(job): + if CI.is_build_job(job): # no GH status for build jobs continue - job_config = CI_CONFIG.get_job_config(job) + job_config = CI.get_job_config(job) if not job_config: # there might be a new job that does not exist on this branch - skip it continue @@ -558,7 +554,7 @@ def _fetch_commit_tokens(message: str, pr_info: PRInfo) -> List[str]: res = [ match for match in matches - if match in CILabels or match.startswith("job_") or match.startswith("batch_") + if match in CI.Tags or match.startswith("job_") or match.startswith("batch_") ] print(f"CI modifiers from commit message: [{res}]") res_2 = [] @@ -567,7 +563,7 @@ def _fetch_commit_tokens(message: str, pr_info: PRInfo) -> List[str]: res_2 = [ match for match in matches - if match in CILabels + if match in CI.Tags or match.startswith("job_") or match.startswith("batch_") ] @@ -643,7 +639,7 @@ def _upload_build_artifacts( print(f"Report file has been uploaded to [{report_url}]") # Upload master head's binaries - static_bin_name = CI_CONFIG.build_config[build_name].static_binary_name + static_bin_name = CI.get_build_config(build_name).static_binary_name if pr_info.is_master and static_bin_name: # Full binary with debug info: s3_path_full = "/".join((pr_info.base_ref, static_bin_name, "clickhouse-full")) @@ -838,15 +834,15 @@ def _add_build_to_version_history( def _run_test(job_name: str, run_command: str) -> int: assert ( - run_command or CI_CONFIG.get_job_config(job_name).run_command + run_command or CI.get_job_config(job_name).run_command ), "Run command must be provided as input argument or be configured in job config" env = os.environ.copy() - timeout = CI_CONFIG.get_job_config(job_name).timeout or None + timeout = CI.get_job_config(job_name).timeout or None if not run_command: run_command = "/".join( - (os.path.dirname(__file__), CI_CONFIG.get_job_config(job_name).run_command) + (os.path.dirname(__file__), CI.get_job_config(job_name).run_command) ) if ".py" in run_command and not run_command.startswith("python"): run_command = "python3 " + run_command @@ -918,7 +914,7 @@ def _set_pending_statuses(pr_info: PRInfo) -> None: state=PENDING, target_url="", description="", - context=StatusNames.SYNC, + context=CI.StatusNames.SYNC, ) except Exception as ex: print(f"ERROR: failed to set GH commit status, ex: {ex}") @@ -952,7 +948,7 @@ def main() -> int: ### CONFIGURE action: start if args.configure: - if CI and pr_info.is_pr: + if IS_CI and pr_info.is_pr: # store meta on s3 (now we need it only for PRs) meta = CiMetadata(s3, pr_info.number, pr_info.head_ref) meta.run_id = int(GITHUB_RUN_ID) @@ -962,7 +958,7 @@ def main() -> int: args.commit_message or None, update_from_api=True ) - if ci_settings.no_merge_commit and CI: + if ci_settings.no_merge_commit and IS_CI: git_runner.run(f"{GIT_PREFIX} checkout {pr_info.sha}") git_ref = git_runner.run(f"{GIT_PREFIX} rev-parse HEAD") @@ -985,18 +981,19 @@ def main() -> int: ) ci_cache.print_status() - if CI and not pr_info.is_merge_queue: + if IS_CI and not pr_info.is_merge_queue: # wait for pending jobs to be finished, await_jobs is a long blocking call ci_cache.await_pending_jobs(pr_info.is_release) if pr_info.is_release: + print("Release/master: CI Cache add pending records for all todo jobs") ci_cache.push_pending_all(pr_info.is_release) # conclude results result["git_ref"] = git_ref result["version"] = version - result["build"] = ci_cache.job_digests[Build.PACKAGE_RELEASE] - result["docs"] = ci_cache.job_digests[JobNames.DOCS_CHECK] + result["build"] = ci_cache.job_digests[CI.BuildNames.PACKAGE_RELEASE] + result["docs"] = ci_cache.job_digests[CI.JobNames.DOCS_CHECK] result["ci_settings"] = ci_settings.as_dict() if not args.skip_jobs: result["stages_data"] = _generate_ci_stage_config(ci_cache.jobs_to_do) @@ -1027,7 +1024,7 @@ def main() -> int: f"Check if rerun for name: [{check_name}], extended name [{check_name_with_group}]" ) previous_status = None - if CI_CONFIG.is_build_job(check_name): + if CI.is_build_job(check_name): # this is a build job - check if a build report is present build_result = ( BuildResult.load_any(check_name, pr_info.number, pr_info.head_ref) @@ -1055,10 +1052,8 @@ def main() -> int: # rerun helper check # FIXME: remove rerun_helper check and rely on ci cache only if check_name not in ( - # we might want to rerun reports' jobs - disable rerun check for them - JobNames.BUILD_CHECK, - JobNames.BUILD_CHECK_SPECIAL, - ): + CI.JobNames.BUILD_CHECK, + ): # we might want to rerun build report job rerun_helper = RerunHelper(commit, check_name_with_group) if rerun_helper.is_already_finished_by_status(): status = rerun_helper.get_finished_status() @@ -1071,7 +1066,7 @@ def main() -> int: # ci cache check if not previous_status and not ci_settings.no_ci_cache: ci_cache = CiCache(s3, indata["jobs_data"]["digests"]).update() - job_config = CI_CONFIG.get_job_config(check_name) + job_config = CI.get_job_config(check_name) if ci_cache.is_successful( check_name, args.batch, @@ -1111,7 +1106,7 @@ def main() -> int: ch_helper = ClickHouseHelper() check_url = "" - if CI_CONFIG.is_build_job(args.job_name): + if CI.is_build_job(args.job_name): assert ( indata ), f"--infile with config must be provided for POST action of a build type job [{args.job_name}]" @@ -1119,8 +1114,7 @@ def main() -> int: # upload binaries only for normal builds in PRs upload_binary = ( not pr_info.is_pr - or args.job_name - not in CI_CONFIG.get_builds_for_report(JobNames.BUILD_CHECK_SPECIAL) + or CI.get_job_ci_stage(args.job_name) == CI.WorkflowStages.BUILDS_1 or CiSettings.create_from_run_config(indata).upload_all ) diff --git a/tests/ci/ci_cache.py b/tests/ci/ci_cache.py index 56a84272a630..8ee0ae543856 100644 --- a/tests/ci/ci_cache.py +++ b/tests/ci/ci_cache.py @@ -5,7 +5,8 @@ from pathlib import Path from typing import Dict, Optional, Any, Union, Sequence, List, Set -from ci_config import JobNames, Build, CI_CONFIG, JobConfig +from ci_config import CI + from ci_utils import is_hex, GHActions from commit_status_helper import CommitStatusData from env_helper import ( @@ -41,7 +42,7 @@ class CiCache: release - for jobs being executed on the release branch including master branch (not a PR branch) """ - _REQUIRED_DIGESTS = [JobNames.DOCS_CHECK, Build.PACKAGE_RELEASE] + _REQUIRED_DIGESTS = [CI.JobNames.DOCS_CHECK, CI.BuildNames.PACKAGE_RELEASE] _S3_CACHE_PREFIX = "CI_cache_v1" _CACHE_BUILD_REPORT_PREFIX = "build_report" _RECORD_FILE_EXTENSION = ".ci" @@ -80,7 +81,7 @@ class JobType(Enum): @classmethod def is_docs_job(cls, job_name: str) -> bool: - return job_name == JobNames.DOCS_CHECK + return job_name == CI.JobNames.DOCS_CHECK @classmethod def is_srcs_job(cls, job_name: str) -> bool: @@ -105,8 +106,8 @@ def __init__( ): self.enabled = cache_enabled self.jobs_to_skip = [] # type: List[str] - self.jobs_to_wait = {} # type: Dict[str, JobConfig] - self.jobs_to_do = {} # type: Dict[str, JobConfig] + self.jobs_to_wait = {} # type: Dict[str, CI.JobConfig] + self.jobs_to_do = {} # type: Dict[str, CI.JobConfig] self.s3 = s3 self.job_digests = job_digests self.cache_s3_paths = { @@ -127,9 +128,13 @@ def __init__( @classmethod def calc_digests_and_create( - cls, s3: S3Helper, job_configs: Dict[str, JobConfig], cache_enabled: bool = True + cls, + s3: S3Helper, + job_configs: Dict[str, CI.JobConfig], + cache_enabled: bool = True, + dry_run: bool = False, ) -> "CiCache": - job_digester = JobDigester() + job_digester = JobDigester(dry_run=dry_run) digests = {} print("::group::Job Digests") @@ -140,9 +145,7 @@ def calc_digests_and_create( for job in cls._REQUIRED_DIGESTS: if job not in job_configs: - digest = job_digester.get_job_digest( - CI_CONFIG.get_job_config(job).digest - ) + digest = job_digester.get_job_digest(CI.get_job_config(job).digest) digests[job] = digest print( f" job [{job.rjust(50)}] required for CI Cache has digest [{digest}]" @@ -154,10 +157,10 @@ def _get_digest_for_job_type( self, job_digests: Dict[str, str], job_type: JobType ) -> str: if job_type == self.JobType.DOCS: - res = job_digests[JobNames.DOCS_CHECK] + res = job_digests[CI.JobNames.DOCS_CHECK] elif job_type == self.JobType.SRCS: - if Build.PACKAGE_RELEASE in job_digests: - res = job_digests[Build.PACKAGE_RELEASE] + if CI.BuildNames.PACKAGE_RELEASE in job_digests: + res = job_digests[CI.BuildNames.PACKAGE_RELEASE] else: assert False, "BUG, no build job in digest' list" else: @@ -648,7 +651,7 @@ def download_build_reports(self, file_prefix: str = "") -> List[str]: report_path = Path(REPORT_PATH) report_path.mkdir(exist_ok=True, parents=True) path = ( - self._get_record_s3_path(Build.PACKAGE_RELEASE) + self._get_record_s3_path(CI.BuildNames.PACKAGE_RELEASE) + self._CACHE_BUILD_REPORT_PREFIX ) if file_prefix: @@ -664,13 +667,14 @@ def download_build_reports(self, file_prefix: str = "") -> List[str]: def upload_build_report(self, build_result: BuildResult) -> str: result_json_path = build_result.write_json(Path(TEMP_PATH)) s3_path = ( - self._get_record_s3_path(Build.PACKAGE_RELEASE) + result_json_path.name + self._get_record_s3_path(CI.BuildNames.PACKAGE_RELEASE) + + result_json_path.name ) return self.s3.upload_file( bucket=S3_BUILDS_BUCKET, file_path=result_json_path, s3_path=s3_path ) - def await_pending_jobs(self, is_release: bool) -> None: + def await_pending_jobs(self, is_release: bool, dry_run: bool = False) -> None: """ await pending jobs to be finished @jobs_with_params - jobs to await. {JOB_NAME: {"batches": [BATCHES...], "num_batches": NUM_BATCHES}} @@ -687,15 +691,9 @@ def await_pending_jobs(self, is_release: bool) -> None: MAX_JOB_NUM_TO_WAIT = 3 round_cnt = 0 - # FIXME: temporary experiment: lets enable await for PR' workflows awaiting on build' jobs only + # FIXME: temporary experiment: lets enable await for PR' workflows but for a shorter time if not is_release: - MAX_ROUNDS_TO_WAIT = 1 - remove_from_wait = [] - for job in self.jobs_to_wait: - if job not in Build: - remove_from_wait.append(job) - for job in remove_from_wait: - del self.jobs_to_wait[job] + MAX_ROUNDS_TO_WAIT = 3 while ( len(self.jobs_to_wait) > MAX_JOB_NUM_TO_WAIT @@ -713,11 +711,12 @@ def await_pending_jobs(self, is_release: bool) -> None: start_at = int(time.time()) while expired_sec < TIMEOUT and self.jobs_to_wait: await_finished: Set[str] = set() - time.sleep(poll_interval_sec) + if not dry_run: + time.sleep(poll_interval_sec) self.update() for job_name, job_config in self.jobs_to_wait.items(): num_batches = job_config.num_batches - job_config = CI_CONFIG.get_job_config(job_name) + job_config = CI.get_job_config(job_name) assert job_config.pending_batches assert job_config.batches pending_batches = list(job_config.pending_batches) @@ -741,12 +740,11 @@ def await_pending_jobs(self, is_release: bool) -> None: f"Job [{job_name}_[{batch}/{num_batches}]] is not pending anymore" ) job_config.batches.remove(batch) - job_config.pending_batches.remove(batch) else: print( f"NOTE: Job [{job_name}:{batch}] finished failed - do not add to ready" ) - job_config.pending_batches.remove(batch) + job_config.pending_batches.remove(batch) if not job_config.pending_batches: await_finished.add(job_name) @@ -754,18 +752,25 @@ def await_pending_jobs(self, is_release: bool) -> None: for job in await_finished: self.jobs_to_skip.append(job) del self.jobs_to_wait[job] + del self.jobs_to_do[job] - expired_sec = int(time.time()) - start_at - print( - f"...awaiting continues... seconds left [{TIMEOUT - expired_sec}]" - ) + if not dry_run: + expired_sec = int(time.time()) - start_at + print( + f"...awaiting continues... seconds left [{TIMEOUT - expired_sec}]" + ) + else: + # make up for 2 iterations in dry_run + expired_sec += int(TIMEOUT / 2) + 1 GHActions.print_in_group( "Remaining jobs:", [list(self.jobs_to_wait)], ) - def apply(self, job_configs: Dict[str, JobConfig], is_release: bool) -> "CiCache": + def apply( + self, job_configs: Dict[str, CI.JobConfig], is_release: bool + ) -> "CiCache": if not self.enabled: self.jobs_to_do = job_configs return self diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 6ab1eb8bac41..3dc0040663b0 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -1,1611 +1,655 @@ -#!/usr/bin/env python3 - -import logging import random import re from argparse import ArgumentDefaultsHelpFormatter, ArgumentParser -from copy import deepcopy -from dataclasses import dataclass, field -from pathlib import Path -from typing import Callable, Dict, Iterable, List, Literal, Optional, Union - -from ci_utils import WithIter, normalize_string -from integration_test_images import IMAGES - - -class WorkFlows(metaclass=WithIter): - PULL_REQUEST = "PULL_REQUEST" - MASTER = "MASTER" - BACKPORT = "BACKPORT" - RELEASE = "RELEASE" - SYNC = "SYNC" - +from typing import Dict, Optional, List -class CIStages(metaclass=WithIter): - NA = "UNKNOWN" - BUILDS_1 = "Builds_1" - BUILDS_2 = "Builds_2" - TESTS_1 = "Tests_1" - TESTS_2 = "Tests_2" - TESTS_3 = "Tests_3" +from ci_utils import normalize_string +from ci_definitions import * -class Runners(metaclass=WithIter): - BUILDER = "builder" - STYLE_CHECKER = "style-checker" - STYLE_CHECKER_ARM = "style-checker-aarch64" - FUNC_TESTER = "func-tester" - FUNC_TESTER_ARM = "func-tester-aarch64" - STRESS_TESTER = "stress-tester" - FUZZER_UNIT_TESTER = "fuzzer-unit-tester" - - -class CILabels(metaclass=WithIter): - """ - Label names or commit tokens in normalized form - """ - - DO_NOT_TEST_LABEL = "do_not_test" - NO_MERGE_COMMIT = "no_merge_commit" - NO_CI_CACHE = "no_ci_cache" - # to upload all binaries from build jobs - UPLOAD_ALL_ARTIFACTS = "upload_all" - CI_SET_SYNC = "ci_set_sync" - CI_SET_ARM = "ci_set_arm" - CI_SET_REQUIRED = "ci_set_required" - CI_SET_NORMAL_BUILDS = "ci_set_normal_builds" - CI_SET_SPECIAL_BUILDS = "ci_set_special_builds" - CI_SET_NON_REQUIRED = "ci_set_non_required" - CI_SET_OLD_ANALYZER = "ci_set_old_analyzer" - - libFuzzer = "libFuzzer" - - -class Build(metaclass=WithIter): - PACKAGE_RELEASE = "package_release" - PACKAGE_AARCH64 = "package_aarch64" - PACKAGE_ASAN = "package_asan" - PACKAGE_UBSAN = "package_ubsan" - PACKAGE_TSAN = "package_tsan" - PACKAGE_MSAN = "package_msan" - PACKAGE_DEBUG = "package_debug" - PACKAGE_RELEASE_COVERAGE = "package_release_coverage" - BINARY_RELEASE = "binary_release" - BINARY_TIDY = "binary_tidy" - BINARY_DARWIN = "binary_darwin" - BINARY_AARCH64 = "binary_aarch64" - BINARY_AARCH64_V80COMPAT = "binary_aarch64_v80compat" - BINARY_FREEBSD = "binary_freebsd" - BINARY_DARWIN_AARCH64 = "binary_darwin_aarch64" - BINARY_PPC64LE = "binary_ppc64le" - BINARY_AMD64_COMPAT = "binary_amd64_compat" - BINARY_AMD64_MUSL = "binary_amd64_musl" - BINARY_RISCV64 = "binary_riscv64" - BINARY_S390X = "binary_s390x" - BINARY_LOONGARCH64 = "binary_loongarch64" - FUZZERS = "fuzzers" - - -class JobNames(metaclass=WithIter): - STYLE_CHECK = "Style check" - FAST_TEST = "Fast test" - DOCKER_SERVER = "Docker server image" - DOCKER_KEEPER = "Docker keeper image" - INSTALL_TEST_AMD = "Install packages (amd64)" - INSTALL_TEST_ARM = "Install packages (arm64)" - - STATELESS_TEST_DEBUG = "Stateless tests (debug)" - STATELESS_TEST_RELEASE = "Stateless tests (release)" - STATELESS_TEST_RELEASE_COVERAGE = "Stateless tests (coverage)" - STATELESS_TEST_AARCH64 = "Stateless tests (aarch64)" - STATELESS_TEST_ASAN = "Stateless tests (asan)" - STATELESS_TEST_TSAN = "Stateless tests (tsan)" - STATELESS_TEST_MSAN = "Stateless tests (msan)" - STATELESS_TEST_UBSAN = "Stateless tests (ubsan)" - STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE = ( - "Stateless tests (release, old analyzer, s3, DatabaseReplicated)" - ) - # merged into STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: - # STATELESS_TEST_OLD_ANALYZER_RELEASE = "Stateless tests (release, analyzer)" - # STATELESS_TEST_DB_REPL_RELEASE = "Stateless tests (release, DatabaseReplicated)" - # STATELESS_TEST_S3_RELEASE = "Stateless tests (release, s3 storage)" - STATELESS_TEST_S3_DEBUG = "Stateless tests (debug, s3 storage)" - STATELESS_TEST_S3_TSAN = "Stateless tests (tsan, s3 storage)" - STATELESS_TEST_AZURE_ASAN = "Stateless tests (azure, asan)" - STATELESS_TEST_FLAKY_ASAN = "Stateless tests flaky check (asan)" - - STATEFUL_TEST_DEBUG = "Stateful tests (debug)" - STATEFUL_TEST_RELEASE = "Stateful tests (release)" - STATEFUL_TEST_RELEASE_COVERAGE = "Stateful tests (coverage)" - STATEFUL_TEST_AARCH64 = "Stateful tests (aarch64)" - STATEFUL_TEST_ASAN = "Stateful tests (asan)" - STATEFUL_TEST_TSAN = "Stateful tests (tsan)" - STATEFUL_TEST_MSAN = "Stateful tests (msan)" - STATEFUL_TEST_UBSAN = "Stateful tests (ubsan)" - STATEFUL_TEST_PARALLEL_REPL_RELEASE = "Stateful tests (release, ParallelReplicas)" - STATEFUL_TEST_PARALLEL_REPL_DEBUG = "Stateful tests (debug, ParallelReplicas)" - STATEFUL_TEST_PARALLEL_REPL_ASAN = "Stateful tests (asan, ParallelReplicas)" - STATEFUL_TEST_PARALLEL_REPL_MSAN = "Stateful tests (msan, ParallelReplicas)" - STATEFUL_TEST_PARALLEL_REPL_UBSAN = "Stateful tests (ubsan, ParallelReplicas)" - STATEFUL_TEST_PARALLEL_REPL_TSAN = "Stateful tests (tsan, ParallelReplicas)" - - STRESS_TEST_ASAN = "Stress test (asan)" - STRESS_TEST_TSAN = "Stress test (tsan)" - STRESS_TEST_UBSAN = "Stress test (ubsan)" - STRESS_TEST_MSAN = "Stress test (msan)" - STRESS_TEST_DEBUG = "Stress test (debug)" - STRESS_TEST_AZURE_TSAN = "Stress test (azure, tsan)" - STRESS_TEST_AZURE_MSAN = "Stress test (azure, msan)" - - INTEGRATION_TEST = "Integration tests (release)" - INTEGRATION_TEST_ASAN = "Integration tests (asan)" - INTEGRATION_TEST_ASAN_OLD_ANALYZER = "Integration tests (asan, old analyzer)" - INTEGRATION_TEST_TSAN = "Integration tests (tsan)" - INTEGRATION_TEST_ARM = "Integration tests (aarch64)" - INTEGRATION_TEST_FLAKY = "Integration tests flaky check (asan)" - - UPGRADE_TEST_DEBUG = "Upgrade check (debug)" - UPGRADE_TEST_ASAN = "Upgrade check (asan)" - UPGRADE_TEST_TSAN = "Upgrade check (tsan)" - UPGRADE_TEST_MSAN = "Upgrade check (msan)" - - UNIT_TEST = "Unit tests (release)" - UNIT_TEST_ASAN = "Unit tests (asan)" - UNIT_TEST_MSAN = "Unit tests (msan)" - UNIT_TEST_TSAN = "Unit tests (tsan)" - UNIT_TEST_UBSAN = "Unit tests (ubsan)" - - AST_FUZZER_TEST_DEBUG = "AST fuzzer (debug)" - AST_FUZZER_TEST_ASAN = "AST fuzzer (asan)" - AST_FUZZER_TEST_MSAN = "AST fuzzer (msan)" - AST_FUZZER_TEST_TSAN = "AST fuzzer (tsan)" - AST_FUZZER_TEST_UBSAN = "AST fuzzer (ubsan)" - - JEPSEN_KEEPER = "ClickHouse Keeper Jepsen" - JEPSEN_SERVER = "ClickHouse Server Jepsen" - - PERFORMANCE_TEST_AMD64 = "Performance Comparison" - PERFORMANCE_TEST_ARM64 = "Performance Comparison Aarch64" - - SQL_LOGIC_TEST = "Sqllogic test (release)" - - SQLANCER = "SQLancer (release)" - SQLANCER_DEBUG = "SQLancer (debug)" - SQLTEST = "SQLTest" - - COMPATIBILITY_TEST = "Compatibility check (amd64)" - COMPATIBILITY_TEST_ARM = "Compatibility check (aarch64)" - - CLICKBENCH_TEST = "ClickBench (amd64)" - CLICKBENCH_TEST_ARM = "ClickBench (aarch64)" - - LIBFUZZER_TEST = "libFuzzer tests" - - BUILD_CHECK = "ClickHouse build check" - BUILD_CHECK_SPECIAL = "ClickHouse special build check" - - DOCS_CHECK = "Docs check" - BUGFIX_VALIDATE = "Bugfix validation" - - -class StatusNames(metaclass=WithIter): - "Class with statuses that aren't related to particular jobs" - CI = "CI running" - MERGEABLE = "Mergeable Check" - SYNC = "A Sync" - - -# dynamically update JobName with Build jobs -for attr_name in dir(Build): - if not attr_name.startswith("__") and not callable(getattr(Build, attr_name)): - setattr(JobNames, attr_name, getattr(Build, attr_name)) - - -@dataclass -class DigestConfig: - # all files, dirs to include into digest, glob supported - include_paths: List[Union[str, Path]] = field(default_factory=list) - # file suffixes to exclude from digest - exclude_files: List[str] = field(default_factory=list) - # directories to exclude from digest - exclude_dirs: List[Union[str, Path]] = field(default_factory=list) - # docker names to include into digest - docker: List[str] = field(default_factory=list) - # git submodules digest - git_submodules: bool = False - - -@dataclass -class LabelConfig: - """ - configures different CI scenarios per GH label - """ - - run_jobs: Iterable[str] = frozenset() - - -@dataclass -class JobConfig: - """ - contains config parameters for job execution in CI workflow - """ - - # configures digest calculation for the job - digest: DigestConfig = field(default_factory=DigestConfig) - # will be triggered for the job if omitted in CI workflow yml - run_command: str = "" - # job timeout, seconds - timeout: Optional[int] = None - # sets number of batches for a multi-batch job - num_batches: int = 1 - # label that enables job in CI, if set digest isn't used - run_by_label: str = "" - # to run always regardless of the job digest or/and label - run_always: bool = False - # if the job needs to be run on the release branch, including master (building packages, docker server). - # NOTE: Subsequent runs on the same branch with the similar digest are still considered skip-able. - required_on_release_branch: bool = False - # job is for pr workflow only - pr_only: bool = False - # job is for release/master branches only - release_only: bool = False - # to randomly pick and run one job among jobs in the same @random_bucket (PR branches only). - random_bucket: str = "" - # Do not set it. A list of batches to run. It will be set in runtime in accordance with ci cache and ci settings - batches: Optional[List[int]] = None - # Do not set it. A list of batches to await. It will be set in runtime in accordance with ci cache and ci settings - pending_batches: Optional[List[int]] = None - - -builds_job_config = JobConfig( - required_on_release_branch=True, - digest=DigestConfig( - include_paths=[ - "./src", - "./contrib/*-cmake", - "./contrib/consistent-hashing", - "./contrib/murmurhash", - "./contrib/libfarmhash", - "./contrib/pdqsort", - "./contrib/cityhash102", - "./contrib/sparse-checkout", - "./contrib/libmetrohash", - "./contrib/update-submodules.sh", - "./contrib/CMakeLists.txt", - "./CMakeLists.txt", - "./PreLoad.cmake", - "./cmake", - "./base", - "./programs", - "./packages", - "./docker/packager/packager", - "./rust", - "./tests/ci/version_helper.py", - # FIXME: This is a WA to rebuild the CH and recreate the Performance.tar.zst artifact - # when there are changes in performance test scripts. - # Due to the current design of the perf test we need to rebuild CH when the performance test changes, - # otherwise the changes will not be visible in the PerformanceTest job in CI - "./tests/performance", - ], - exclude_files=[".md"], - docker=["clickhouse/binary-builder"], - git_submodules=True, - ), - run_command="build_check.py $BUILD_NAME", -) -fuzzer_build_job_config = deepcopy(builds_job_config) -fuzzer_build_job_config.run_by_label = CILabels.libFuzzer - - -@dataclass -class BuildConfig: - name: str - compiler: str - package_type: Literal["deb", "binary", "fuzzers"] - additional_pkgs: bool = False - debug_build: bool = False - coverage: bool = False - sanitizer: str = "" - tidy: bool = False - # sparse_checkout is needed only to test the option itself. - # No particular sense to use it in every build, since it slows down the job. - sparse_checkout: bool = False - comment: str = "" - static_binary_name: str = "" - job_config: JobConfig = field(default_factory=lambda: deepcopy(builds_job_config)) - - def export_env(self, export: bool = False) -> str: - def process(field_name: str, field: Union[bool, str]) -> str: - if isinstance(field, bool): - field = str(field).lower() - elif not isinstance(field, str): - field = "" - if export: - return f"export BUILD_{field_name.upper()}={repr(field)}" - return f"BUILD_{field_name.upper()}={field}" - - return "\n".join(process(k, v) for k, v in self.__dict__.items()) - - -@dataclass -class BuildReportConfig: - builds: List[str] - job_config: JobConfig = field( - default_factory=lambda: JobConfig( - run_command='build_report_check.py "$CHECK_NAME"', - digest=DigestConfig( - include_paths=[ - "./tests/ci/build_report_check.py", - "./tests/ci/upload_result_helper.py", - ], - ), - ) - ) - - -@dataclass -class TestConfig: - required_build: str - job_config: JobConfig = field(default_factory=JobConfig) - - -BuildConfigs = Dict[str, BuildConfig] -BuildsReportConfig = Dict[str, BuildReportConfig] -TestConfigs = Dict[str, TestConfig] -LabelConfigs = Dict[str, LabelConfig] - -# common digests configs -compatibility_check_digest = DigestConfig( - include_paths=["./tests/ci/compatibility_check.py"], - docker=["clickhouse/test-old-ubuntu", "clickhouse/test-old-centos"], -) -install_check_digest = DigestConfig( - include_paths=["./tests/ci/install_check.py"], - docker=["clickhouse/install-deb-test", "clickhouse/install-rpm-test"], -) -stateless_check_digest = DigestConfig( - include_paths=[ - "./tests/ci/functional_test_check.py", - "./tests/queries/0_stateless/", - "./tests/clickhouse-test", - "./tests/config", - "./tests/*.txt", - ], - exclude_files=[".md"], - docker=["clickhouse/stateless-test"], -) -stateful_check_digest = DigestConfig( - include_paths=[ - "./tests/ci/functional_test_check.py", - "./tests/queries/1_stateful/", - "./tests/clickhouse-test", - "./tests/config", - "./tests/*.txt", - ], - exclude_files=[".md"], - docker=["clickhouse/stateful-test"], -) - -stress_check_digest = DigestConfig( - include_paths=[ - "./tests/queries/0_stateless/", - "./tests/queries/1_stateful/", - "./tests/clickhouse-test", - "./tests/config", - "./tests/*.txt", - ], - exclude_files=[".md"], - docker=["clickhouse/stress-test"], -) -# FIXME: which tests are upgrade? just python? -upgrade_check_digest = DigestConfig( - include_paths=["./tests/ci/upgrade_check.py"], - exclude_files=[".md"], - docker=["clickhouse/upgrade-check"], -) -integration_check_digest = DigestConfig( - include_paths=[ - "./tests/ci/integration_test_check.py", - "./tests/ci/integration_tests_runner.py", - "./tests/integration/", - ], - exclude_files=[".md"], - docker=IMAGES.copy(), -) - -ast_fuzzer_check_digest = DigestConfig( - # include_paths=["./tests/ci/ast_fuzzer_check.py"], - # exclude_files=[".md"], - # docker=["clickhouse/fuzzer"], -) -unit_check_digest = DigestConfig( - include_paths=["./tests/ci/unit_tests_check.py"], - exclude_files=[".md"], - docker=["clickhouse/unit-test"], -) -perf_check_digest = DigestConfig( - include_paths=[ - "./tests/ci/performance_comparison_check.py", - "./tests/performance/", - ], - exclude_files=[".md"], - docker=["clickhouse/performance-comparison"], -) -sqllancer_check_digest = DigestConfig( - # include_paths=["./tests/ci/sqlancer_check.py"], - # exclude_files=[".md"], - # docker=["clickhouse/sqlancer-test"], -) -sqllogic_check_digest = DigestConfig( - include_paths=["./tests/ci/sqllogic_test.py"], - exclude_files=[".md"], - docker=["clickhouse/sqllogic-test"], -) -sqltest_check_digest = DigestConfig( - include_paths=["./tests/ci/sqltest.py"], - exclude_files=[".md"], - docker=["clickhouse/sqltest"], -) -bugfix_validate_check = DigestConfig( - include_paths=[ - "./tests/queries/0_stateless/", - "./tests/ci/integration_test_check.py", - "./tests/ci/functional_test_check.py", - "./tests/ci/bugfix_validate_check.py", - ], - exclude_files=[".md"], - docker=IMAGES.copy() - + [ - "clickhouse/stateless-test", - ], -) -# common test params -docker_server_job_config = JobConfig( - required_on_release_branch=True, - run_command='docker_server.py --check-name "$CHECK_NAME" --release-type head --allow-build-reuse', - digest=DigestConfig( - include_paths=[ - "tests/ci/docker_server.py", - "./docker/server", - ] - ), -) -compatibility_test_common_params = { - "digest": compatibility_check_digest, - "run_command": "compatibility_check.py", -} -stateless_test_common_params = { - "digest": stateless_check_digest, - "run_command": 'functional_test_check.py "$CHECK_NAME"', - "timeout": 10800, -} -stateful_test_common_params = { - "digest": stateful_check_digest, - "run_command": 'functional_test_check.py "$CHECK_NAME"', - "timeout": 3600, -} -stress_test_common_params = { - "digest": stress_check_digest, - "run_command": "stress_check.py", - "timeout": 9000, -} -upgrade_test_common_params = { - "digest": upgrade_check_digest, - "run_command": "upgrade_check.py", -} -astfuzzer_test_common_params = { - "digest": ast_fuzzer_check_digest, - "run_command": "ast_fuzzer_check.py", - "run_always": True, -} -integration_test_common_params = { - "digest": integration_check_digest, - "run_command": 'integration_test_check.py "$CHECK_NAME"', -} -unit_test_common_params = { - "digest": unit_check_digest, - "run_command": "unit_tests_check.py", -} -perf_test_common_params = { - "digest": perf_check_digest, - "run_command": "performance_comparison_check.py", -} -sqllancer_test_common_params = JobConfig( - digest=sqllancer_check_digest, - run_command="sqlancer_check.py", - release_only=True, - run_always=True, -) -sqllogic_test_params = JobConfig( - digest=sqllogic_check_digest, - run_command="sqllogic_test.py", - timeout=10800, - release_only=True, -) -sql_test_params = JobConfig( - digest=sqltest_check_digest, - run_command="sqltest.py", - timeout=10800, - release_only=True, -) -clickbench_test_params = { - "digest": DigestConfig( - include_paths=[ - "tests/ci/clickbench.py", - ], - docker=["clickhouse/clickbench"], - ), - "run_command": 'clickbench.py "$CHECK_NAME"', - "timeout": 900, -} -install_test_params = JobConfig( - digest=install_check_digest, - run_command='install_check.py "$CHECK_NAME"', - timeout=900, -) - - -@dataclass -class CIConfig: +class CI: """ Contains configs for all jobs in the CI pipeline each config item in the below dicts should be an instance of JobConfig class or inherited from it """ - build_config: BuildConfigs - builds_report_config: BuildsReportConfig - test_configs: TestConfigs - other_jobs_configs: TestConfigs - label_configs: LabelConfigs + # reimport types to CI class so that they visible as CI.* and mypy is happy + # pylint:disable=useless-import-alias,reimported,import-outside-toplevel + from ci_definitions import BuildConfig as BuildConfig + from ci_definitions import DigestConfig as DigestConfig + from ci_definitions import JobConfig as JobConfig + from ci_definitions import CheckDescription as CheckDescription + from ci_definitions import Tags as Tags + from ci_definitions import JobNames as JobNames + from ci_definitions import BuildNames as BuildNames + from ci_definitions import StatusNames as StatusNames + from ci_definitions import CHECK_DESCRIPTIONS as CHECK_DESCRIPTIONS + from ci_definitions import MQ_JOBS as MQ_JOBS + from ci_definitions import WorkflowStages as WorkflowStages + from ci_definitions import Runners as Runners # Jobs that run for doc related updates _DOCS_CHECK_JOBS = [JobNames.DOCS_CHECK, JobNames.STYLE_CHECK] - # Jobs that run in Merge Queue if it's enabled - _MQ_JOBS = [ - JobNames.STYLE_CHECK, - JobNames.FAST_TEST, - Build.BINARY_RELEASE, - JobNames.UNIT_TEST, - ] - - def get_label_config(self, label_name: str) -> Optional[LabelConfig]: - for label, config in self.label_configs.items(): - if normalize_string(label_name) == normalize_string(label): - return config - return None - - def get_job_ci_stage(self, job_name: str) -> str: - if job_name in [ - JobNames.STYLE_CHECK, - JobNames.FAST_TEST, - JobNames.JEPSEN_KEEPER, - JobNames.BUILD_CHECK, - JobNames.BUILD_CHECK_SPECIAL, - ]: - # FIXME: we can't currently handle Jepsen in the Stage as it's job has concurrency directive - # BUILD_CHECK and BUILD_CHECK_SPECIAL runs not in stage because we need them even if Builds stage failed - return CIStages.NA - stage_type = None - if self.is_build_job(job_name): - stage_type = CIStages.BUILDS_1 - if job_name in CI_CONFIG.get_builds_for_report( - JobNames.BUILD_CHECK_SPECIAL - ): - # special builds go to Build_2 stage to not delay Builds_1/Test_1 - stage_type = CIStages.BUILDS_2 - elif self.is_docs_job(job_name): - stage_type = CIStages.TESTS_1 - elif self.is_test_job(job_name): - if job_name in CI_CONFIG.test_configs: - required_build = CI_CONFIG.test_configs[job_name].required_build - assert required_build - if required_build in CI_CONFIG.get_builds_for_report( - JobNames.BUILD_CHECK - ): - stage_type = CIStages.TESTS_1 - else: - stage_type = CIStages.TESTS_2 - else: - stage_type = CIStages.TESTS_1 - if job_name not in REQUIRED_CHECKS: - stage_type = CIStages.TESTS_3 - assert stage_type, f"BUG [{job_name}]" - return stage_type - - def get_job_config(self, check_name: str) -> JobConfig: - res = None - for config in ( - self.build_config, - self.builds_report_config, - self.test_configs, - self.other_jobs_configs, - ): - if check_name in config: # type: ignore - res = config[check_name].job_config # type: ignore - break - return res # type: ignore - - def get_runner_type(self, check_name: str) -> str: - result = None - if self.is_build_job(check_name) or check_name == JobNames.FAST_TEST: - result = Runners.BUILDER - elif any( - words in check_name.lower() - for words in [ - "install packages", - "compatibility check", - "docker", - "build check", - "jepsen", - "style check", - ] - ): - result = Runners.STYLE_CHECKER - elif check_name == JobNames.DOCS_CHECK: - # docs job is demanding - result = Runners.FUNC_TESTER_ARM - elif any( - words in check_name.lower() - for words in [ - "stateless", - "stateful", - "clickbench", - "sqllogic test", - "libfuzzer", - "bugfix validation", - ] - ): - result = Runners.FUNC_TESTER - elif any( - words in check_name.lower() - for words in ["stress", "upgrade", "integration", "performance comparison"] - ): - result = Runners.STRESS_TESTER - elif any( - words in check_name.lower() - for words in ["ast fuzzer", "unit tests", "sqlancer", "sqltest"] - ): - result = Runners.FUZZER_UNIT_TESTER - - assert result, f"BUG, no runner for [{check_name}]" - - if ( - "aarch" in check_name.lower() or "arm64" in check_name.lower() - ) and "aarch" not in result: - if result == Runners.STRESS_TESTER: - # FIXME: no arm stress tester group atm - result = Runners.FUNC_TESTER_ARM - elif result == Runners.BUILDER: - # crosscompile - no arm required - pass - else: - # switch to aarch64 runner - result += "-aarch64" - - return result - - def get_job_parents(self, check_name: str) -> List[str]: - res = [] - check_name = normalize_string(check_name) - for config in ( - self.build_config, - self.test_configs, - self.other_jobs_configs, - ): - for job_name in config: # type: ignore - if check_name == normalize_string(job_name): - if isinstance(config[job_name], TestConfig): # type: ignore - if config[job_name].required_build: # type: ignore - res.append(config[job_name].required_build) # type: ignore - return res - - def get_digest_config(self, check_name: str) -> DigestConfig: - res = None - for config in ( - self.other_jobs_configs, - self.build_config, - self.builds_report_config, - self.test_configs, - ): - if check_name in config: # type: ignore - res = config[check_name].job_config.digest # type: ignore - assert ( - res - ), f"Invalid check_name or CI_CONFIG outdated, config not found for [{check_name}]" - return res # type: ignore - - def get_workflow_jobs_with_configs( - self, is_mq: bool, is_docs_only: bool, is_master: bool - ) -> Dict[str, JobConfig]: - """ - get a list of all jobs for a workflow with configs - """ - jobs = [] - if is_mq: - jobs = self._MQ_JOBS - elif is_docs_only: - jobs = self._DOCS_CHECK_JOBS - else: - for config in ( - self.other_jobs_configs, - self.build_config, - self.builds_report_config, - self.test_configs, - ): - jobs += list(config) # type:ignore - if is_master: - for job in self._MQ_JOBS: - jobs.remove(job) - - randomization_bucket_jobs = {} # type: Dict[str, Dict[str, JobConfig]] - res = {} # type: Dict[str, JobConfig] - for job in jobs: - job_config = self.get_job_config(job) - - if job_config.random_bucket: - if job_config.random_bucket not in randomization_bucket_jobs: - randomization_bucket_jobs[job_config.random_bucket] = {} - randomization_bucket_jobs[job_config.random_bucket][job] = job_config - continue - - res[job] = job_config - - # add to the result a random job from each random bucket, if any - for bucket, jobs_configs in randomization_bucket_jobs.items(): - job = random.choice(list(jobs_configs)) - print(f"Pick job [{job}] from randomization bucket [{bucket}]") - res[job] = jobs_configs[job] - - return res - - def get_builds_for_report( - self, report_name: str, release: bool = False, backport: bool = False - ) -> List[str]: - # hack to modify build list for release and bp wf - assert not (release and backport), "Invalid input" - if backport and report_name == JobNames.BUILD_CHECK: - return [ - Build.PACKAGE_RELEASE, - Build.PACKAGE_AARCH64, - Build.PACKAGE_ASAN, - Build.PACKAGE_TSAN, - Build.PACKAGE_DEBUG, - ] - if (release or backport) and report_name == JobNames.BUILD_CHECK_SPECIAL: - return [ - Build.BINARY_DARWIN, - Build.BINARY_DARWIN_AARCH64, - ] - - return self.builds_report_config[report_name].builds - - @classmethod - def is_build_job(cls, job: str) -> bool: - return job in Build - - @classmethod - def is_test_job(cls, job: str) -> bool: - return not cls.is_build_job(job) and job != JobNames.STYLE_CHECK - - @classmethod - def is_docs_job(cls, job: str) -> bool: - return job == JobNames.DOCS_CHECK - - @staticmethod - def is_required(check_name: str) -> bool: - """Checks if a check_name is in REQUIRED_CHECKS, including batched jobs""" - _BATCH_REGEXP = re.compile(r"\s+\[[0-9/]+\]$") - if check_name in REQUIRED_CHECKS: - return True - if batch := _BATCH_REGEXP.search(check_name): - return check_name[: batch.start()] in REQUIRED_CHECKS - return False - - def validate(self) -> None: - errors = [] - for name, build_config in self.build_config.items(): - build_in_reports = False - for _, report_config in self.builds_report_config.items(): - if name in report_config.builds: - build_in_reports = True - break - # All build configs must belong to build_report_config - if not build_in_reports: - logging.error("Build name %s does not belong to build reports", name) - errors.append(f"Build name {name} does not belong to build reports") - # The name should be the same as build_config.name - if not build_config.name == name: - logging.error( - "Build name '%s' does not match the config 'name' value '%s'", - name, - build_config.name, - ) - errors.append( - f"Build name {name} does not match 'name' value '{build_config.name}'" - ) - # All build_report_config values should be in build_config.keys() - for build_report_name, build_report_config in self.builds_report_config.items(): - build_names = build_report_config.builds - missed_names = [ - name for name in build_names if name not in self.build_config.keys() - ] - if missed_names: - logging.error( - "The following names of the build report '%s' " - "are missed in build_config: %s", - build_report_name, - missed_names, - ) - errors.append( - f"The following names of the build report '{build_report_name}' " - f"are missed in build_config: {missed_names}", - ) - # And finally, all tests' requirements must be in the builds - for test_name, test_config in self.test_configs.items(): - if test_config.required_build not in self.build_config.keys(): - logging.error( - "The requirement '%s' for '%s' is not found in builds", - test_config, - test_name, - ) - errors.append( - f"The requirement '{test_config}' for " - f"'{test_name}' is not found in builds" - ) - if ( - test_config.required_build - and test_config.required_build - not in self.builds_report_config[JobNames.BUILD_CHECK].builds - ): - errors.append( - f"Test job' required build must be from [{JobNames.BUILD_CHECK}] list" - ) - - if errors: - raise KeyError("config contains errors", errors) - - -# checks required by Mergeable Check -REQUIRED_CHECKS = [ - "PR Check", - StatusNames.SYNC, - JobNames.BUILD_CHECK, - JobNames.BUILD_CHECK_SPECIAL, - JobNames.DOCS_CHECK, - JobNames.FAST_TEST, - JobNames.STATEFUL_TEST_RELEASE, - JobNames.STATELESS_TEST_RELEASE, - JobNames.STATELESS_TEST_ASAN, - JobNames.STATELESS_TEST_FLAKY_ASAN, - JobNames.STATEFUL_TEST_ASAN, - JobNames.STYLE_CHECK, - JobNames.UNIT_TEST_ASAN, - JobNames.UNIT_TEST_MSAN, - JobNames.UNIT_TEST, - JobNames.UNIT_TEST_TSAN, - JobNames.UNIT_TEST_UBSAN, - JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER, - JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE, -] - -CI_CONFIG = CIConfig( - label_configs={ - CILabels.DO_NOT_TEST_LABEL: LabelConfig(run_jobs=[JobNames.STYLE_CHECK]), - CILabels.CI_SET_ARM: LabelConfig( + TAG_CONFIGS = { + Tags.DO_NOT_TEST_LABEL: LabelConfig(run_jobs=[JobNames.STYLE_CHECK]), + Tags.CI_SET_ARM: LabelConfig( run_jobs=[ JobNames.STYLE_CHECK, - Build.PACKAGE_AARCH64, + BuildNames.PACKAGE_AARCH64, JobNames.INTEGRATION_TEST_ARM, ] ), - CILabels.CI_SET_REQUIRED: LabelConfig(run_jobs=REQUIRED_CHECKS), - CILabels.CI_SET_NORMAL_BUILDS: LabelConfig( - run_jobs=[ - JobNames.STYLE_CHECK, - JobNames.BUILD_CHECK, - Build.PACKAGE_RELEASE, - Build.PACKAGE_AARCH64, - Build.PACKAGE_ASAN, - Build.PACKAGE_UBSAN, - Build.PACKAGE_TSAN, - Build.PACKAGE_MSAN, - Build.PACKAGE_DEBUG, - Build.BINARY_RELEASE, - Build.PACKAGE_RELEASE_COVERAGE, - Build.FUZZERS, - ] - ), - CILabels.CI_SET_SPECIAL_BUILDS: LabelConfig( - run_jobs=[ - JobNames.STYLE_CHECK, - JobNames.BUILD_CHECK_SPECIAL, - Build.BINARY_TIDY, - Build.BINARY_DARWIN, - Build.BINARY_AARCH64, - Build.BINARY_AARCH64_V80COMPAT, - Build.BINARY_FREEBSD, - Build.BINARY_DARWIN_AARCH64, - Build.BINARY_PPC64LE, - Build.BINARY_RISCV64, - Build.BINARY_S390X, - Build.BINARY_LOONGARCH64, - Build.BINARY_AMD64_COMPAT, - Build.BINARY_AMD64_MUSL, - ] + Tags.CI_SET_REQUIRED: LabelConfig(run_jobs=REQUIRED_CHECKS), + Tags.CI_SET_BUILDS: LabelConfig( + run_jobs=[JobNames.STYLE_CHECK, JobNames.BUILD_CHECK] + + [build for build in BuildNames if build != BuildNames.FUZZERS] ), - CILabels.CI_SET_NON_REQUIRED: LabelConfig( + Tags.CI_SET_NON_REQUIRED: LabelConfig( run_jobs=[job for job in JobNames if job not in REQUIRED_CHECKS] ), - CILabels.CI_SET_OLD_ANALYZER: LabelConfig( + Tags.CI_SET_OLD_ANALYZER: LabelConfig( run_jobs=[ JobNames.STYLE_CHECK, JobNames.FAST_TEST, - Build.PACKAGE_RELEASE, - Build.PACKAGE_ASAN, + BuildNames.PACKAGE_RELEASE, + BuildNames.PACKAGE_ASAN, JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE, JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER, ] ), - CILabels.CI_SET_SYNC: LabelConfig( + Tags.CI_SET_SYNC: LabelConfig( run_jobs=[ - Build.PACKAGE_ASAN, + BuildNames.PACKAGE_ASAN, JobNames.STYLE_CHECK, JobNames.BUILD_CHECK, JobNames.UNIT_TEST_ASAN, JobNames.STATEFUL_TEST_ASAN, ] ), - }, - build_config={ - Build.PACKAGE_RELEASE: BuildConfig( - name=Build.PACKAGE_RELEASE, - compiler="clang-18", - package_type="deb", - static_binary_name="amd64", - additional_pkgs=True, - ), - Build.PACKAGE_AARCH64: BuildConfig( - name=Build.PACKAGE_AARCH64, - compiler="clang-18-aarch64", - package_type="deb", - static_binary_name="aarch64", - additional_pkgs=True, - ), - Build.PACKAGE_ASAN: BuildConfig( - name=Build.PACKAGE_ASAN, - compiler="clang-18", - sanitizer="address", - package_type="deb", - ), - Build.PACKAGE_UBSAN: BuildConfig( - name=Build.PACKAGE_UBSAN, - compiler="clang-18", - sanitizer="undefined", - package_type="deb", - ), - Build.PACKAGE_TSAN: BuildConfig( - name=Build.PACKAGE_TSAN, - compiler="clang-18", - sanitizer="thread", - package_type="deb", - ), - Build.PACKAGE_MSAN: BuildConfig( - name=Build.PACKAGE_MSAN, - compiler="clang-18", - sanitizer="memory", - package_type="deb", - ), - Build.PACKAGE_DEBUG: BuildConfig( - name=Build.PACKAGE_DEBUG, - compiler="clang-18", - debug_build=True, - package_type="deb", - sparse_checkout=True, # Check that it works with at least one build, see also update-submodules.sh - ), - Build.PACKAGE_RELEASE_COVERAGE: BuildConfig( - name=Build.PACKAGE_RELEASE_COVERAGE, - compiler="clang-18", - coverage=True, - package_type="deb", - ), - Build.BINARY_RELEASE: BuildConfig( - name=Build.BINARY_RELEASE, - compiler="clang-18", - package_type="binary", - ), - Build.BINARY_TIDY: BuildConfig( - name=Build.BINARY_TIDY, - compiler="clang-18", - debug_build=True, - package_type="binary", - static_binary_name="debug-amd64", - tidy=True, - comment="clang-tidy is used for static analysis", - ), - Build.BINARY_DARWIN: BuildConfig( - name=Build.BINARY_DARWIN, - compiler="clang-18-darwin", - package_type="binary", - static_binary_name="macos", - ), - Build.BINARY_AARCH64: BuildConfig( - name=Build.BINARY_AARCH64, - compiler="clang-18-aarch64", - package_type="binary", - ), - Build.BINARY_AARCH64_V80COMPAT: BuildConfig( - name=Build.BINARY_AARCH64_V80COMPAT, - compiler="clang-18-aarch64-v80compat", - package_type="binary", - static_binary_name="aarch64v80compat", - comment="For ARMv8.1 and older", - ), - Build.BINARY_FREEBSD: BuildConfig( - name=Build.BINARY_FREEBSD, - compiler="clang-18-freebsd", - package_type="binary", - static_binary_name="freebsd", - ), - Build.BINARY_DARWIN_AARCH64: BuildConfig( - name=Build.BINARY_DARWIN_AARCH64, - compiler="clang-18-darwin-aarch64", - package_type="binary", - static_binary_name="macos-aarch64", - ), - Build.BINARY_PPC64LE: BuildConfig( - name=Build.BINARY_PPC64LE, - compiler="clang-18-ppc64le", - package_type="binary", - static_binary_name="powerpc64le", - ), - Build.BINARY_AMD64_COMPAT: BuildConfig( - name=Build.BINARY_AMD64_COMPAT, - compiler="clang-18-amd64-compat", - package_type="binary", - static_binary_name="amd64compat", - comment="SSE2-only build", - ), - Build.BINARY_AMD64_MUSL: BuildConfig( - name=Build.BINARY_AMD64_MUSL, - compiler="clang-18-amd64-musl", - package_type="binary", - static_binary_name="amd64musl", - comment="Build with Musl", - ), - Build.BINARY_RISCV64: BuildConfig( - name=Build.BINARY_RISCV64, - compiler="clang-18-riscv64", - package_type="binary", - static_binary_name="riscv64", - ), - Build.BINARY_S390X: BuildConfig( - name=Build.BINARY_S390X, - compiler="clang-18-s390x", - package_type="binary", - static_binary_name="s390x", - ), - Build.BINARY_LOONGARCH64: BuildConfig( - name=Build.BINARY_LOONGARCH64, - compiler="clang-18-loongarch64", - package_type="binary", - static_binary_name="loongarch64", - ), - Build.FUZZERS: BuildConfig( - name=Build.FUZZERS, - compiler="clang-18", - package_type="fuzzers", - job_config=fuzzer_build_job_config, - ), - }, - builds_report_config={ - JobNames.BUILD_CHECK: BuildReportConfig( - builds=[ - Build.PACKAGE_RELEASE, - Build.PACKAGE_AARCH64, - Build.PACKAGE_ASAN, - Build.PACKAGE_UBSAN, - Build.PACKAGE_TSAN, - Build.PACKAGE_MSAN, - Build.PACKAGE_DEBUG, - Build.BINARY_RELEASE, - Build.PACKAGE_RELEASE_COVERAGE, - Build.FUZZERS, - ] + } + + JOB_CONFIGS: Dict[str, JobConfig] = { + BuildNames.PACKAGE_RELEASE: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_RELEASE, + compiler="clang-18", + package_type="deb", + static_binary_name="amd64", + additional_pkgs=True, + ) + ), + BuildNames.PACKAGE_AARCH64: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_AARCH64, + compiler="clang-18-aarch64", + package_type="deb", + static_binary_name="aarch64", + additional_pkgs=True, + ) + ), + BuildNames.PACKAGE_ASAN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_ASAN, + compiler="clang-18", + sanitizer="address", + package_type="deb", + ), ), - JobNames.BUILD_CHECK_SPECIAL: BuildReportConfig( - builds=[ - Build.BINARY_TIDY, - Build.BINARY_DARWIN, - Build.BINARY_AARCH64, - Build.BINARY_AARCH64_V80COMPAT, - Build.BINARY_FREEBSD, - Build.BINARY_DARWIN_AARCH64, - Build.BINARY_PPC64LE, - Build.BINARY_RISCV64, - Build.BINARY_S390X, - Build.BINARY_LOONGARCH64, - Build.BINARY_AMD64_COMPAT, - Build.BINARY_AMD64_MUSL, - ] + BuildNames.PACKAGE_UBSAN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_UBSAN, + compiler="clang-18", + sanitizer="undefined", + package_type="deb", + ), + ), + BuildNames.PACKAGE_TSAN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_TSAN, + compiler="clang-18", + sanitizer="thread", + package_type="deb", + ), + ), + BuildNames.PACKAGE_MSAN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_MSAN, + compiler="clang-18", + sanitizer="memory", + package_type="deb", + ), + ), + BuildNames.PACKAGE_DEBUG: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_DEBUG, + compiler="clang-18", + debug_build=True, + package_type="deb", + sparse_checkout=True, # Check that it works with at least one build, see also update-submodules.sh + ), + ), + BuildNames.PACKAGE_RELEASE_COVERAGE: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.PACKAGE_RELEASE_COVERAGE, + compiler="clang-18", + coverage=True, + package_type="deb", + ), + ), + BuildNames.BINARY_RELEASE: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_RELEASE, + compiler="clang-18", + package_type="binary", + ), + ), + BuildNames.BINARY_TIDY: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_TIDY, + compiler="clang-18", + debug_build=True, + package_type="binary", + static_binary_name="debug-amd64", + tidy=True, + comment="clang-tidy is used for static analysis", + ), + ), + BuildNames.BINARY_DARWIN: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_DARWIN, + compiler="clang-18-darwin", + package_type="binary", + static_binary_name="macos", + ), ), - }, - other_jobs_configs={ - JobNames.DOCKER_SERVER: TestConfig("", job_config=docker_server_job_config), - JobNames.DOCKER_KEEPER: TestConfig("", job_config=docker_server_job_config), - JobNames.DOCS_CHECK: TestConfig( - "", - job_config=JobConfig( - digest=DigestConfig( - include_paths=["**/*.md", "./docs", "tests/ci/docs_check.py"], - docker=["clickhouse/docs-builder"], - ), - run_command="docs_check.py", + BuildNames.BINARY_AARCH64: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_AARCH64, + compiler="clang-18-aarch64", + package_type="binary", ), ), - JobNames.FAST_TEST: TestConfig( - "", - job_config=JobConfig( - pr_only=True, - digest=DigestConfig( - include_paths=["./tests/queries/0_stateless/"], - exclude_files=[".md"], - docker=["clickhouse/fasttest"], - ), - timeout=2400, + BuildNames.BINARY_AARCH64_V80COMPAT: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_AARCH64_V80COMPAT, + compiler="clang-18-aarch64-v80compat", + package_type="binary", + static_binary_name="aarch64v80compat", + comment="For ARMv8.1 and older", ), ), - JobNames.STYLE_CHECK: TestConfig( - "", - job_config=JobConfig( - run_always=True, + BuildNames.BINARY_FREEBSD: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_FREEBSD, + compiler="clang-18-freebsd", + package_type="binary", + static_binary_name="freebsd", ), ), - JobNames.BUGFIX_VALIDATE: TestConfig( - "", - # we run this check by label - no digest required - job_config=JobConfig( - run_by_label="pr-bugfix", - run_command="bugfix_validate_check.py", - timeout=900, + BuildNames.BINARY_DARWIN_AARCH64: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_DARWIN_AARCH64, + compiler="clang-18-darwin-aarch64", + package_type="binary", + static_binary_name="macos-aarch64", ), ), - }, - test_configs={ - JobNames.INSTALL_TEST_AMD: TestConfig( - Build.PACKAGE_RELEASE, job_config=install_test_params + BuildNames.BINARY_PPC64LE: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_PPC64LE, + compiler="clang-18-ppc64le", + package_type="binary", + static_binary_name="powerpc64le", + ), ), - JobNames.INSTALL_TEST_ARM: TestConfig( - Build.PACKAGE_AARCH64, job_config=install_test_params + BuildNames.BINARY_AMD64_COMPAT: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_AMD64_COMPAT, + compiler="clang-18-amd64-compat", + package_type="binary", + static_binary_name="amd64compat", + comment="SSE2-only build", + ), ), - JobNames.STATEFUL_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(**stateful_test_common_params) # type: ignore + BuildNames.BINARY_AMD64_MUSL: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_AMD64_MUSL, + compiler="clang-18-amd64-musl", + package_type="binary", + static_binary_name="amd64musl", + comment="Build with Musl", + ), ), - JobNames.STATEFUL_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(**stateful_test_common_params) # type: ignore + BuildNames.BINARY_RISCV64: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_RISCV64, + compiler="clang-18-riscv64", + package_type="binary", + static_binary_name="riscv64", + ), ), - JobNames.STATEFUL_TEST_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(**stateful_test_common_params) # type: ignore + BuildNames.BINARY_S390X: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_S390X, + compiler="clang-18-s390x", + package_type="binary", + static_binary_name="s390x", + ), ), - JobNames.STATEFUL_TEST_UBSAN: TestConfig( - Build.PACKAGE_UBSAN, job_config=JobConfig(**stateful_test_common_params) # type: ignore + BuildNames.BINARY_LOONGARCH64: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.BINARY_LOONGARCH64, + compiler="clang-18-loongarch64", + package_type="binary", + static_binary_name="loongarch64", + ), ), - JobNames.STATEFUL_TEST_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, job_config=JobConfig(**stateful_test_common_params) # type: ignore + BuildNames.FUZZERS: CommonJobConfigs.BUILD.with_properties( + build_config=BuildConfig( + name=BuildNames.FUZZERS, + compiler="clang-18", + package_type="fuzzers", + ), + run_by_label=Tags.libFuzzer, ), - JobNames.STATEFUL_TEST_RELEASE: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(**stateful_test_common_params) # type: ignore + JobNames.BUILD_CHECK: CommonJobConfigs.BUILD_REPORT.with_properties(), + JobNames.INSTALL_TEST_AMD: CommonJobConfigs.INSTALL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE] ), - JobNames.STATEFUL_TEST_RELEASE_COVERAGE: TestConfig( - Build.PACKAGE_RELEASE_COVERAGE, job_config=JobConfig(**stateful_test_common_params) # type: ignore + JobNames.INSTALL_TEST_ARM: CommonJobConfigs.INSTALL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64] ), - JobNames.STATEFUL_TEST_AARCH64: TestConfig( - Build.PACKAGE_AARCH64, job_config=JobConfig(**stateful_test_common_params) # type: ignore + JobNames.STATEFUL_TEST_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN] ), - # Stateful tests for parallel replicas - JobNames.STATEFUL_TEST_PARALLEL_REPL_RELEASE: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(**stateful_test_common_params) # type: ignore + JobNames.STATEFUL_TEST_TSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN] ), - JobNames.STATEFUL_TEST_PARALLEL_REPL_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, job_config=JobConfig(**stateful_test_common_params) # type: ignore + JobNames.STATEFUL_TEST_MSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN] ), - JobNames.STATEFUL_TEST_PARALLEL_REPL_ASAN: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(random_bucket="parrepl_with_sanitizer", **stateful_test_common_params) # type: ignore + JobNames.STATEFUL_TEST_UBSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_UBSAN] ), - JobNames.STATEFUL_TEST_PARALLEL_REPL_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(random_bucket="parrepl_with_sanitizer", **stateful_test_common_params) # type: ignore + JobNames.STATEFUL_TEST_DEBUG: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG] ), - JobNames.STATEFUL_TEST_PARALLEL_REPL_UBSAN: TestConfig( - Build.PACKAGE_UBSAN, job_config=JobConfig(random_bucket="parrepl_with_sanitizer", **stateful_test_common_params) # type: ignore + JobNames.STATEFUL_TEST_RELEASE: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE] ), - JobNames.STATEFUL_TEST_PARALLEL_REPL_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(random_bucket="parrepl_with_sanitizer", **stateful_test_common_params) # type: ignore + JobNames.STATEFUL_TEST_RELEASE_COVERAGE: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE_COVERAGE] ), - # End stateful tests for parallel replicas - JobNames.STATELESS_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, - job_config=JobConfig(num_batches=4, **stateless_test_common_params), # type: ignore + JobNames.STATEFUL_TEST_AARCH64: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64] ), - JobNames.STATELESS_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, - job_config=JobConfig(num_batches=5, **stateless_test_common_params), # type: ignore + JobNames.STATEFUL_TEST_PARALLEL_REPL_RELEASE: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE] ), - JobNames.STATELESS_TEST_MSAN: TestConfig( - Build.PACKAGE_MSAN, - job_config=JobConfig(num_batches=6, **stateless_test_common_params), # type: ignore + JobNames.STATEFUL_TEST_PARALLEL_REPL_DEBUG: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG] ), - JobNames.STATELESS_TEST_UBSAN: TestConfig( - Build.PACKAGE_UBSAN, - job_config=JobConfig(num_batches=2, **stateless_test_common_params), # type: ignore + JobNames.STATEFUL_TEST_PARALLEL_REPL_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], + random_bucket="parrepl_with_sanitizer", ), - JobNames.STATELESS_TEST_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, - job_config=JobConfig(num_batches=5, **stateless_test_common_params), # type: ignore + JobNames.STATEFUL_TEST_PARALLEL_REPL_MSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], + random_bucket="parrepl_with_sanitizer", ), - JobNames.STATELESS_TEST_RELEASE: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(**stateless_test_common_params) # type: ignore + JobNames.STATEFUL_TEST_PARALLEL_REPL_UBSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_UBSAN], + random_bucket="parrepl_with_sanitizer", ), - JobNames.STATELESS_TEST_RELEASE_COVERAGE: TestConfig( - Build.PACKAGE_RELEASE_COVERAGE, - job_config=JobConfig(num_batches=6, **stateless_test_common_params), # type: ignore + JobNames.STATEFUL_TEST_PARALLEL_REPL_TSAN: CommonJobConfigs.STATEFUL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], + random_bucket="parrepl_with_sanitizer", ), - JobNames.STATELESS_TEST_AARCH64: TestConfig( - Build.PACKAGE_AARCH64, job_config=JobConfig(**stateless_test_common_params) # type: ignore + JobNames.STATELESS_TEST_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4 ), - JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: TestConfig( - Build.PACKAGE_RELEASE, - job_config=JobConfig(num_batches=4, **stateless_test_common_params), # type: ignore + JobNames.STATELESS_TEST_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], num_batches=5 ), - JobNames.STATELESS_TEST_S3_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, - job_config=JobConfig(num_batches=6, **stateless_test_common_params), # type: ignore + JobNames.STATELESS_TEST_MSAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], num_batches=6 ), - JobNames.STATELESS_TEST_AZURE_ASAN: TestConfig( - Build.PACKAGE_ASAN, - job_config=JobConfig(num_batches=4, **stateless_test_common_params, release_only=True), # type: ignore + JobNames.STATELESS_TEST_UBSAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_UBSAN], num_batches=2 ), - JobNames.STATELESS_TEST_S3_TSAN: TestConfig( - Build.PACKAGE_TSAN, - job_config=JobConfig(num_batches=5, **stateless_test_common_params), # type: ignore + JobNames.STATELESS_TEST_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=5 ), - JobNames.STRESS_TEST_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, job_config=JobConfig(**stress_test_common_params) # type: ignore + JobNames.STATELESS_TEST_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], ), - JobNames.STRESS_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(**stress_test_common_params) # type: ignore + JobNames.STATELESS_TEST_RELEASE_COVERAGE: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE_COVERAGE], num_batches=6 ), - JobNames.STRESS_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(random_bucket="stress_with_sanitizer", **stress_test_common_params) # type: ignore + JobNames.STATELESS_TEST_AARCH64: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64], ), - JobNames.STRESS_TEST_UBSAN: TestConfig( - Build.PACKAGE_UBSAN, job_config=JobConfig(random_bucket="stress_with_sanitizer", **stress_test_common_params) # type: ignore + JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4 ), - JobNames.STRESS_TEST_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(random_bucket="stress_with_sanitizer", **stress_test_common_params) # type: ignore + JobNames.STATELESS_TEST_S3_DEBUG: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG], num_batches=6 ), - JobNames.UPGRADE_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore + JobNames.STATELESS_TEST_AZURE_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], num_batches=4, release_only=True ), - JobNames.STRESS_TEST_AZURE_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(**stress_test_common_params, release_only=True) # type: ignore + JobNames.STATELESS_TEST_S3_TSAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], + num_batches=5, ), - JobNames.STRESS_TEST_AZURE_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(**stress_test_common_params, release_only=True) # type: ignore + JobNames.STRESS_TEST_DEBUG: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG], ), - JobNames.UPGRADE_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore + JobNames.STRESS_TEST_TSAN: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], ), - JobNames.UPGRADE_TEST_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(pr_only=True, random_bucket="upgrade_with_sanitizer", **upgrade_test_common_params) # type: ignore + JobNames.STRESS_TEST_ASAN: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], + random_bucket="stress_with_sanitizer", ), - JobNames.UPGRADE_TEST_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, job_config=JobConfig(pr_only=True, **upgrade_test_common_params) # type: ignore + JobNames.STRESS_TEST_UBSAN: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_UBSAN], + random_bucket="stress_with_sanitizer", ), - JobNames.INTEGRATION_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, - job_config=JobConfig(num_batches=4, **integration_test_common_params, release_only=True), # type: ignore + JobNames.STRESS_TEST_MSAN: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], + random_bucket="stress_with_sanitizer", ), - JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER: TestConfig( - Build.PACKAGE_ASAN, - job_config=JobConfig(num_batches=6, **integration_test_common_params), # type: ignore + JobNames.STRESS_TEST_AZURE_TSAN: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], release_only=True ), - JobNames.INTEGRATION_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, - job_config=JobConfig(num_batches=6, **integration_test_common_params), # type: ignore + JobNames.STRESS_TEST_AZURE_MSAN: CommonJobConfigs.STRESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], release_only=True ), - JobNames.INTEGRATION_TEST_ARM: TestConfig( - Build.PACKAGE_AARCH64, - job_config=JobConfig(num_batches=6, **integration_test_common_params), # type: ignore + JobNames.UPGRADE_TEST_ASAN: CommonJobConfigs.UPGRADE_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], + random_bucket="upgrade_with_sanitizer", + pr_only=True, ), - JobNames.INTEGRATION_TEST: TestConfig( - Build.PACKAGE_RELEASE, - job_config=JobConfig(num_batches=4, **integration_test_common_params, release_only=True), # type: ignore + JobNames.UPGRADE_TEST_TSAN: CommonJobConfigs.UPGRADE_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], + random_bucket="upgrade_with_sanitizer", + pr_only=True, ), - JobNames.INTEGRATION_TEST_FLAKY: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(pr_only=True, **integration_test_common_params) # type: ignore + JobNames.UPGRADE_TEST_MSAN: CommonJobConfigs.UPGRADE_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], + random_bucket="upgrade_with_sanitizer", + pr_only=True, ), - JobNames.COMPATIBILITY_TEST: TestConfig( - Build.PACKAGE_RELEASE, - job_config=JobConfig( - required_on_release_branch=True, **compatibility_test_common_params # type: ignore - ), + JobNames.UPGRADE_TEST_DEBUG: CommonJobConfigs.UPGRADE_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG], pr_only=True ), - JobNames.COMPATIBILITY_TEST_ARM: TestConfig( - Build.PACKAGE_AARCH64, - job_config=JobConfig( - required_on_release_branch=True, **compatibility_test_common_params # type: ignore - ), + JobNames.INTEGRATION_TEST_ASAN: CommonJobConfigs.INTEGRATION_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], release_only=True, num_batches=4 ), - JobNames.UNIT_TEST: TestConfig( - Build.BINARY_RELEASE, job_config=JobConfig(**unit_test_common_params) # type: ignore + JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER: CommonJobConfigs.INTEGRATION_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], num_batches=6 ), - JobNames.UNIT_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(**unit_test_common_params) # type: ignore + JobNames.INTEGRATION_TEST_TSAN: CommonJobConfigs.INTEGRATION_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], num_batches=6 ), - JobNames.UNIT_TEST_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(**unit_test_common_params) # type: ignore + JobNames.INTEGRATION_TEST_ARM: CommonJobConfigs.INTEGRATION_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64], num_batches=6 ), - JobNames.UNIT_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(**unit_test_common_params) # type: ignore + JobNames.INTEGRATION_TEST: CommonJobConfigs.INTEGRATION_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], + num_batches=4, + release_only=True, ), - JobNames.UNIT_TEST_UBSAN: TestConfig( - Build.PACKAGE_UBSAN, job_config=JobConfig(**unit_test_common_params) # type: ignore + JobNames.INTEGRATION_TEST_FLAKY: CommonJobConfigs.INTEGRATION_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True ), - JobNames.AST_FUZZER_TEST_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, job_config=JobConfig(**astfuzzer_test_common_params) # type: ignore + JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], + required_on_release_branch=True, ), - JobNames.AST_FUZZER_TEST_ASAN: TestConfig( - Build.PACKAGE_ASAN, job_config=JobConfig(**astfuzzer_test_common_params) # type: ignore + JobNames.COMPATIBILITY_TEST_ARM: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64], + required_on_release_branch=True, ), - JobNames.AST_FUZZER_TEST_MSAN: TestConfig( - Build.PACKAGE_MSAN, job_config=JobConfig(**astfuzzer_test_common_params) # type: ignore + JobNames.UNIT_TEST: CommonJobConfigs.UNIT_TEST.with_properties( + required_builds=[BuildNames.BINARY_RELEASE], ), - JobNames.AST_FUZZER_TEST_TSAN: TestConfig( - Build.PACKAGE_TSAN, job_config=JobConfig(**astfuzzer_test_common_params) # type: ignore + JobNames.UNIT_TEST_ASAN: CommonJobConfigs.UNIT_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], ), - JobNames.AST_FUZZER_TEST_UBSAN: TestConfig( - Build.PACKAGE_UBSAN, job_config=JobConfig(**astfuzzer_test_common_params) # type: ignore + JobNames.UNIT_TEST_MSAN: CommonJobConfigs.UNIT_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], ), - JobNames.STATELESS_TEST_FLAKY_ASAN: TestConfig( - # replace to non-default - Build.PACKAGE_ASAN, - job_config=JobConfig(pr_only=True, **{**stateless_test_common_params, "timeout": 3600}), # type: ignore + JobNames.UNIT_TEST_TSAN: CommonJobConfigs.UNIT_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], ), - JobNames.JEPSEN_KEEPER: TestConfig( - Build.BINARY_RELEASE, - job_config=JobConfig( - run_by_label="jepsen-test", run_command="jepsen_check.py keeper" - ), + JobNames.UNIT_TEST_UBSAN: CommonJobConfigs.UNIT_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_UBSAN], ), - JobNames.JEPSEN_SERVER: TestConfig( - Build.BINARY_RELEASE, - job_config=JobConfig( - run_by_label="jepsen-test", run_command="jepsen_check.py server" - ), + JobNames.AST_FUZZER_TEST_DEBUG: CommonJobConfigs.ASTFUZZER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG], + ), + JobNames.AST_FUZZER_TEST_ASAN: CommonJobConfigs.ASTFUZZER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], + ), + JobNames.AST_FUZZER_TEST_MSAN: CommonJobConfigs.ASTFUZZER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_MSAN], + ), + JobNames.AST_FUZZER_TEST_TSAN: CommonJobConfigs.ASTFUZZER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_TSAN], + ), + JobNames.AST_FUZZER_TEST_UBSAN: CommonJobConfigs.ASTFUZZER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_UBSAN], + ), + JobNames.STATELESS_TEST_FLAKY_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True, timeout=3600 + ), + JobNames.JEPSEN_KEEPER: JobConfig( + required_builds=[BuildNames.BINARY_RELEASE], + run_by_label="jepsen-test", + run_command="jepsen_check.py keeper", + runner_type=Runners.STYLE_CHECKER_ARM, + ), + JobNames.JEPSEN_SERVER: JobConfig( + required_builds=[BuildNames.BINARY_RELEASE], + run_by_label="jepsen-test", + run_command="jepsen_check.py server", + runner_type=Runners.STYLE_CHECKER_ARM, ), - JobNames.PERFORMANCE_TEST_AMD64: TestConfig( - Build.PACKAGE_RELEASE, - job_config=JobConfig(num_batches=4, **perf_test_common_params), # type: ignore + JobNames.PERFORMANCE_TEST_AMD64: CommonJobConfigs.PERF_TESTS.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4 ), - JobNames.PERFORMANCE_TEST_ARM64: TestConfig( - Build.PACKAGE_AARCH64, - job_config=JobConfig(num_batches=4, run_by_label="pr-performance", **perf_test_common_params), # type: ignore + JobNames.PERFORMANCE_TEST_ARM64: CommonJobConfigs.PERF_TESTS.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64], + num_batches=4, + run_by_label="pr-performance", ), - JobNames.SQLANCER: TestConfig( - Build.PACKAGE_RELEASE, job_config=sqllancer_test_common_params + JobNames.SQLANCER: CommonJobConfigs.SQLLANCER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], ), - JobNames.SQLANCER_DEBUG: TestConfig( - Build.PACKAGE_DEBUG, job_config=sqllancer_test_common_params + JobNames.SQLANCER_DEBUG: CommonJobConfigs.SQLLANCER_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_DEBUG], ), - JobNames.SQL_LOGIC_TEST: TestConfig( - Build.PACKAGE_RELEASE, job_config=sqllogic_test_params + JobNames.SQL_LOGIC_TEST: CommonJobConfigs.SQLLOGIC_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], ), - JobNames.SQLTEST: TestConfig(Build.PACKAGE_RELEASE, job_config=sql_test_params), - JobNames.CLICKBENCH_TEST: TestConfig( - Build.PACKAGE_RELEASE, job_config=JobConfig(**clickbench_test_params) # type: ignore + JobNames.SQLTEST: CommonJobConfigs.SQL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], ), - JobNames.CLICKBENCH_TEST_ARM: TestConfig( - Build.PACKAGE_AARCH64, job_config=JobConfig(**clickbench_test_params) # type: ignore + JobNames.CLICKBENCH_TEST: CommonJobConfigs.SQL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE], ), - JobNames.LIBFUZZER_TEST: TestConfig( - Build.FUZZERS, - job_config=JobConfig( - run_by_label=CILabels.libFuzzer, - timeout=10800, - run_command='libfuzzer_test_check.py "$CHECK_NAME"', + JobNames.CLICKBENCH_TEST_ARM: CommonJobConfigs.SQL_TEST.with_properties( + required_builds=[BuildNames.PACKAGE_AARCH64], + ), + JobNames.LIBFUZZER_TEST: JobConfig( + required_builds=[BuildNames.FUZZERS], + run_by_label=Tags.libFuzzer, + timeout=10800, + run_command='libfuzzer_test_check.py "$CHECK_NAME"', + runner_type=Runners.STYLE_CHECKER, + ), + JobNames.DOCKER_SERVER: CommonJobConfigs.DOCKER_SERVER.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE] + ), + JobNames.DOCKER_KEEPER: CommonJobConfigs.DOCKER_SERVER.with_properties( + required_builds=[BuildNames.PACKAGE_RELEASE] + ), + JobNames.DOCS_CHECK: JobConfig( + digest=DigestConfig( + include_paths=["**/*.md", "./docs", "tests/ci/docs_check.py"], + docker=["clickhouse/docs-builder"], ), - ), # type: ignore - }, -) -CI_CONFIG.validate() + run_command="docs_check.py", + runner_type=Runners.FUNC_TESTER, + ), + JobNames.FAST_TEST: JobConfig( + pr_only=True, + digest=DigestConfig( + include_paths=["./tests/queries/0_stateless/"], + exclude_files=[".md"], + docker=["clickhouse/fasttest"], + ), + timeout=2400, + runner_type=Runners.BUILDER, + ), + JobNames.STYLE_CHECK: JobConfig( + run_always=True, + runner_type=Runners.STYLE_CHECKER_ARM, + ), + JobNames.BUGFIX_VALIDATE: JobConfig( + run_by_label="pr-bugfix", + run_command="bugfix_validate_check.py", + timeout=900, + runner_type=Runners.STYLE_CHECKER, + ), + } + + @classmethod + def get_tag_config(cls, label_name: str) -> Optional[LabelConfig]: + for label, config in cls.TAG_CONFIGS.items(): + if normalize_string(label_name) == normalize_string(label): + return config + return None + + @classmethod + def get_job_ci_stage(cls, job_name: str) -> str: + if job_name in [ + JobNames.STYLE_CHECK, + JobNames.FAST_TEST, + JobNames.JEPSEN_SERVER, + JobNames.JEPSEN_KEEPER, + JobNames.BUILD_CHECK, + ]: + return WorkflowStages.NA + + stage_type = None + if cls.is_build_job(job_name): + for _job, config in cls.JOB_CONFIGS.items(): + if config.required_builds and job_name in config.required_builds: + stage_type = WorkflowStages.BUILDS_1 + break + else: + stage_type = WorkflowStages.BUILDS_2 + elif cls.is_docs_job(job_name): + stage_type = WorkflowStages.TESTS_1 + elif cls.is_test_job(job_name): + if job_name in CI.JOB_CONFIGS: + if job_name in REQUIRED_CHECKS: + stage_type = WorkflowStages.TESTS_1 + else: + stage_type = WorkflowStages.TESTS_3 + assert stage_type, f"BUG [{job_name}]" + return stage_type + + @classmethod + def get_job_config(cls, check_name: str) -> JobConfig: + return cls.JOB_CONFIGS[check_name] + + @classmethod + def get_required_build_name(cls, check_name: str) -> str: + assert check_name in cls.JOB_CONFIGS + required_builds = cls.JOB_CONFIGS[check_name].required_builds + assert required_builds and len(required_builds) == 1 + return required_builds[0] + + @classmethod + def get_job_parents(cls, check_name: str) -> List[str]: + return cls.JOB_CONFIGS[check_name].required_builds or [] + + @classmethod + def get_workflow_jobs_with_configs( + cls, is_mq: bool, is_docs_only: bool, is_master: bool, is_pr: bool + ) -> Dict[str, JobConfig]: + """ + get a list of all jobs for a workflow with configs + """ + jobs = [] + if is_mq: + jobs = MQ_JOBS + elif is_docs_only: + jobs = cls._DOCS_CHECK_JOBS + else: + # add all jobs + jobs = list(cls.JOB_CONFIGS) + if is_master: + for job in MQ_JOBS: + jobs.remove(job) + + randomization_bucket_jobs = {} # type: Dict[str, Dict[str, JobConfig]] + res = {} # type: Dict[str, JobConfig] + for job in jobs: + job_config = cls.JOB_CONFIGS[job] + if job_config.random_bucket and is_pr: + if job_config.random_bucket not in randomization_bucket_jobs: + randomization_bucket_jobs[job_config.random_bucket] = {} + randomization_bucket_jobs[job_config.random_bucket][job] = job_config + continue + + res[job] = job_config + + # add to the result a random job from each random bucket, if any + for bucket, jobs_configs in randomization_bucket_jobs.items(): + job = random.choice(list(jobs_configs)) + print(f"Pick job [{job}] from randomization bucket [{bucket}]") + res[job] = jobs_configs[job] + + return res -@dataclass -class CheckDescription: - name: str - description: str # the check descriptions, will be put into the status table - match_func: Callable[[str], bool] # the function to check vs the commit status + @classmethod + def is_build_job(cls, job: str) -> bool: + return job in cls.BuildNames + + @classmethod + def is_test_job(cls, job: str) -> bool: + return not cls.is_build_job(job) and job != cls.JobNames.STYLE_CHECK - def __hash__(self) -> int: - return hash(self.name + self.description) + @classmethod + def is_docs_job(cls, job: str) -> bool: + return job == JobNames.DOCS_CHECK + @classmethod + def is_required(cls, check_name: str) -> bool: + """Checks if a check_name is in REQUIRED_CHECKS, including batched jobs""" + _BATCH_REGEXP = re.compile(r"\s+\[[0-9/]+\]$") + if check_name in REQUIRED_CHECKS: + return True + if batch := _BATCH_REGEXP.search(check_name): + return check_name[: batch.start()] in REQUIRED_CHECKS + return False -CHECK_DESCRIPTIONS = [ - CheckDescription( - "PR Check", - "Checks correctness of the PR's body", - lambda x: x == "PR Check", - ), - CheckDescription( - StatusNames.SYNC, - "If it fails, ask a maintainer for help", - lambda x: x == StatusNames.SYNC, - ), - CheckDescription( - "AST fuzzer", - "Runs randomly generated queries to catch program errors. " - "The build type is optionally given in parenthesis. " - "If it fails, ask a maintainer for help", - lambda x: x.startswith("AST fuzzer"), - ), - CheckDescription( - JobNames.BUGFIX_VALIDATE, - "Checks that either a new test (functional or integration) or there " - "some changed tests that fail with the binary built on master branch", - lambda x: x == JobNames.BUGFIX_VALIDATE, - ), - CheckDescription( - "CI running", - "A meta-check that indicates the running CI. Normally, it's in success or " - "pending state. The failed status indicates some problems with the PR", - lambda x: x == "CI running", - ), - CheckDescription( - "ClickHouse build check", - "Builds ClickHouse in various configurations for use in further steps. " - "You have to fix the builds that fail. Build logs often has enough " - "information to fix the error, but you might have to reproduce the failure " - "locally. The cmake options can be found in the build log, grepping for " - 'cmake. Use these options and follow the general build process', - lambda x: x.startswith("ClickHouse") and x.endswith("build check"), - ), - CheckDescription( - "Compatibility check", - "Checks that clickhouse binary runs on distributions with old libc " - "versions. If it fails, ask a maintainer for help", - lambda x: x.startswith("Compatibility check"), - ), - CheckDescription( - JobNames.DOCKER_SERVER, - "The check to build and optionally push the mentioned image to docker hub", - lambda x: x.startswith("Docker server"), - ), - CheckDescription( - JobNames.DOCKER_KEEPER, - "The check to build and optionally push the mentioned image to docker hub", - lambda x: x.startswith("Docker keeper"), - ), - CheckDescription( - JobNames.DOCS_CHECK, - "Builds and tests the documentation", - lambda x: x == JobNames.DOCS_CHECK, - ), - CheckDescription( - JobNames.FAST_TEST, - "Normally this is the first check that is ran for a PR. It builds ClickHouse " - 'and runs most of stateless functional tests, ' - "omitting some. If it fails, further checks are not started until it is fixed. " - "Look at the report to see which tests fail, then reproduce the failure " - 'locally as described here', - lambda x: x == JobNames.FAST_TEST, - ), - CheckDescription( - "Flaky tests", - "Checks if new added or modified tests are flaky by running them repeatedly, " - "in parallel, with more randomization. Functional tests are run 100 times " - "with address sanitizer, and additional randomization of thread scheduling. " - "Integration tests are run up to 10 times. If at least once a new test has " - "failed, or was too long, this check will be red. We don't allow flaky tests, " - 'read the doc', - lambda x: "tests flaky check" in x, - ), - CheckDescription( - "Install packages", - "Checks that the built packages are installable in a clear environment", - lambda x: x.startswith("Install packages ("), - ), - CheckDescription( - "Integration tests", - "The integration tests report. In parenthesis the package type is given, " - "and in square brackets are the optional part/total tests", - lambda x: x.startswith("Integration tests ("), - ), - CheckDescription( - StatusNames.MERGEABLE, - "Checks if all other necessary checks are successful", - lambda x: x == StatusNames.MERGEABLE, - ), - CheckDescription( - "Performance Comparison", - "Measure changes in query performance. The performance test report is " - 'described in detail here. ' - "In square brackets are the optional part/total tests", - lambda x: x.startswith("Performance Comparison"), - ), - CheckDescription( - "Push to Dockerhub", - "The check for building and pushing the CI related docker images to docker hub", - lambda x: x.startswith("Push") and "to Dockerhub" in x, - ), - CheckDescription( - "Sqllogic", - "Run clickhouse on the " - 'sqllogic ' - "test set against sqlite and checks that all statements are passed", - lambda x: x.startswith("Sqllogic test"), - ), - CheckDescription( - "SQLancer", - "Fuzzing tests that detect logical bugs with " - 'SQLancer tool', - lambda x: x.startswith("SQLancer"), - ), - CheckDescription( - "Stateful tests", - "Runs stateful functional tests for ClickHouse binaries built in various " - "configurations -- release, debug, with sanitizers, etc", - lambda x: x.startswith("Stateful tests ("), - ), - CheckDescription( - "Stateless tests", - "Runs stateless functional tests for ClickHouse binaries built in various " - "configurations -- release, debug, with sanitizers, etc", - lambda x: x.startswith("Stateless tests ("), - ), - CheckDescription( - "Stress test", - "Runs stateless functional tests concurrently from several clients to detect " - "concurrency-related errors", - lambda x: x.startswith("Stress test ("), - ), - CheckDescription( - JobNames.STYLE_CHECK, - "Runs a set of checks to keep the code style clean. If some of tests failed, " - "see the related log from the report", - lambda x: x == JobNames.STYLE_CHECK, - ), - CheckDescription( - "Unit tests", - "Runs the unit tests for different release types", - lambda x: x.startswith("Unit tests ("), - ), - CheckDescription( - "Upgrade check", - "Runs stress tests on server version from last release and then tries to " - "upgrade it to the version from the PR. It checks if the new server can " - "successfully startup without any errors, crashes or sanitizer asserts", - lambda x: x.startswith("Upgrade check ("), - ), - CheckDescription( - "ClickBench", - "Runs [ClickBench](https://github.com/ClickHouse/ClickBench/) with instant-attach table", - lambda x: x.startswith("ClickBench"), - ), - CheckDescription( - "Fallback for unknown", - "There's no description for the check yet, please add it to " - "tests/ci/ci_config.py:CHECK_DESCRIPTIONS", - lambda x: True, - ), -] + @classmethod + def get_build_config(cls, build_name: str) -> BuildConfig: + assert build_name in cls.JOB_CONFIGS, f"Invalid build name [{build_name}]" + res = cls.JOB_CONFIGS[build_name].build_config + assert res, f"not a build [{build_name}] or invalid JobConfig" + return res -def main() -> None: +if __name__ == "__main__": parser = ArgumentParser( formatter_class=ArgumentDefaultsHelpFormatter, description="The script provides build config for GITHUB_ENV or shell export", @@ -1617,10 +661,9 @@ def main() -> None: help="if set, the ENV parameters are provided for shell export", ) args = parser.parse_args() - build_config = CI_CONFIG.build_config.get(args.build_name) - if build_config: - print(build_config.export_env(args.export)) - - -if __name__ == "__main__": - main() + assert ( + args.build_name in CI.JOB_CONFIGS + ), f"Build name [{args.build_name}] is not valid" + build_config = CI.JOB_CONFIGS[args.build_name].build_config + assert build_config, "--export must not be used for non-build jobs" + print(build_config.export_env(args.export)) diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py new file mode 100644 index 000000000000..e427d48505e5 --- /dev/null +++ b/tests/ci/ci_definitions.py @@ -0,0 +1,754 @@ +import copy +from dataclasses import dataclass, field +from pathlib import Path +from typing import Callable, List, Union, Iterable, Optional, Literal, Any + +from ci_utils import WithIter +from integration_test_images import IMAGES + + +class WorkflowStages(metaclass=WithIter): + """ + Stages of GitHUb actions workflow + """ + + # for jobs that do not belong to any stage, e.g. Build Report Check + NA = "UNKNOWN" + # normal builds (builds that required for further testing) + BUILDS_1 = "Builds_1" + # special builds + BUILDS_2 = "Builds_2" + # all tests required for merge + TESTS_1 = "Tests_1" + # not used atm + TESTS_2 = "Tests_2" + # all tests not required for merge + TESTS_3 = "Tests_3" + + +class Runners(metaclass=WithIter): + """ + GitHub runner's labels + """ + + BUILDER = "builder" + STYLE_CHECKER = "style-checker" + STYLE_CHECKER_ARM = "style-checker-aarch64" + FUNC_TESTER = "func-tester" + FUNC_TESTER_ARM = "func-tester-aarch64" + STRESS_TESTER = "stress-tester" + FUZZER_UNIT_TESTER = "fuzzer-unit-tester" + + +class Tags(metaclass=WithIter): + """ + CI Customization tags (set via PR body or some of them in GH labels, e.g. libFuzzer) + """ + + DO_NOT_TEST_LABEL = "do_not_test" + NO_MERGE_COMMIT = "no_merge_commit" + NO_CI_CACHE = "no_ci_cache" + # to upload all binaries from build jobs + UPLOAD_ALL_ARTIFACTS = "upload_all" + CI_SET_SYNC = "ci_set_sync" + CI_SET_ARM = "ci_set_arm" + CI_SET_REQUIRED = "ci_set_required" + CI_SET_BUILDS = "ci_set_builds" + CI_SET_NON_REQUIRED = "ci_set_non_required" + CI_SET_OLD_ANALYZER = "ci_set_old_analyzer" + + libFuzzer = "libFuzzer" + + +class BuildNames(metaclass=WithIter): + """ + Build' job names + """ + + PACKAGE_RELEASE = "package_release" + PACKAGE_AARCH64 = "package_aarch64" + PACKAGE_ASAN = "package_asan" + PACKAGE_UBSAN = "package_ubsan" + PACKAGE_TSAN = "package_tsan" + PACKAGE_MSAN = "package_msan" + PACKAGE_DEBUG = "package_debug" + PACKAGE_RELEASE_COVERAGE = "package_release_coverage" + BINARY_RELEASE = "binary_release" + BINARY_TIDY = "binary_tidy" + BINARY_DARWIN = "binary_darwin" + BINARY_AARCH64 = "binary_aarch64" + BINARY_AARCH64_V80COMPAT = "binary_aarch64_v80compat" + BINARY_FREEBSD = "binary_freebsd" + BINARY_DARWIN_AARCH64 = "binary_darwin_aarch64" + BINARY_PPC64LE = "binary_ppc64le" + BINARY_AMD64_COMPAT = "binary_amd64_compat" + BINARY_AMD64_MUSL = "binary_amd64_musl" + BINARY_RISCV64 = "binary_riscv64" + BINARY_S390X = "binary_s390x" + BINARY_LOONGARCH64 = "binary_loongarch64" + FUZZERS = "fuzzers" + + +class JobNames(metaclass=WithIter): + """ + All CI non-build jobs (Build jobs are concatenated to this list via python hack) + """ + + STYLE_CHECK = "Style check" + FAST_TEST = "Fast test" + DOCKER_SERVER = "Docker server image" + DOCKER_KEEPER = "Docker keeper image" + INSTALL_TEST_AMD = "Install packages (amd64)" + INSTALL_TEST_ARM = "Install packages (aarch64)" + + STATELESS_TEST_DEBUG = "Stateless tests (debug)" + STATELESS_TEST_RELEASE = "Stateless tests (release)" + STATELESS_TEST_RELEASE_COVERAGE = "Stateless tests (coverage)" + STATELESS_TEST_AARCH64 = "Stateless tests (aarch64)" + STATELESS_TEST_ASAN = "Stateless tests (asan)" + STATELESS_TEST_TSAN = "Stateless tests (tsan)" + STATELESS_TEST_MSAN = "Stateless tests (msan)" + STATELESS_TEST_UBSAN = "Stateless tests (ubsan)" + STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE = ( + "Stateless tests (release, old analyzer, s3, DatabaseReplicated)" + ) + STATELESS_TEST_S3_DEBUG = "Stateless tests (debug, s3 storage)" + STATELESS_TEST_S3_TSAN = "Stateless tests (tsan, s3 storage)" + STATELESS_TEST_AZURE_ASAN = "Stateless tests (azure, asan)" + STATELESS_TEST_FLAKY_ASAN = "Stateless tests flaky check (asan)" + + STATEFUL_TEST_DEBUG = "Stateful tests (debug)" + STATEFUL_TEST_RELEASE = "Stateful tests (release)" + STATEFUL_TEST_RELEASE_COVERAGE = "Stateful tests (coverage)" + STATEFUL_TEST_AARCH64 = "Stateful tests (aarch64)" + STATEFUL_TEST_ASAN = "Stateful tests (asan)" + STATEFUL_TEST_TSAN = "Stateful tests (tsan)" + STATEFUL_TEST_MSAN = "Stateful tests (msan)" + STATEFUL_TEST_UBSAN = "Stateful tests (ubsan)" + STATEFUL_TEST_PARALLEL_REPL_RELEASE = "Stateful tests (release, ParallelReplicas)" + STATEFUL_TEST_PARALLEL_REPL_DEBUG = "Stateful tests (debug, ParallelReplicas)" + STATEFUL_TEST_PARALLEL_REPL_ASAN = "Stateful tests (asan, ParallelReplicas)" + STATEFUL_TEST_PARALLEL_REPL_MSAN = "Stateful tests (msan, ParallelReplicas)" + STATEFUL_TEST_PARALLEL_REPL_UBSAN = "Stateful tests (ubsan, ParallelReplicas)" + STATEFUL_TEST_PARALLEL_REPL_TSAN = "Stateful tests (tsan, ParallelReplicas)" + + STRESS_TEST_ASAN = "Stress test (asan)" + STRESS_TEST_TSAN = "Stress test (tsan)" + STRESS_TEST_UBSAN = "Stress test (ubsan)" + STRESS_TEST_MSAN = "Stress test (msan)" + STRESS_TEST_DEBUG = "Stress test (debug)" + STRESS_TEST_AZURE_TSAN = "Stress test (azure, tsan)" + STRESS_TEST_AZURE_MSAN = "Stress test (azure, msan)" + + INTEGRATION_TEST = "Integration tests (release)" + INTEGRATION_TEST_ASAN = "Integration tests (asan)" + INTEGRATION_TEST_ASAN_OLD_ANALYZER = "Integration tests (asan, old analyzer)" + INTEGRATION_TEST_TSAN = "Integration tests (tsan)" + INTEGRATION_TEST_ARM = "Integration tests (aarch64)" + INTEGRATION_TEST_FLAKY = "Integration tests flaky check (asan)" + + UPGRADE_TEST_DEBUG = "Upgrade check (debug)" + UPGRADE_TEST_ASAN = "Upgrade check (asan)" + UPGRADE_TEST_TSAN = "Upgrade check (tsan)" + UPGRADE_TEST_MSAN = "Upgrade check (msan)" + + UNIT_TEST = "Unit tests (release)" + UNIT_TEST_ASAN = "Unit tests (asan)" + UNIT_TEST_MSAN = "Unit tests (msan)" + UNIT_TEST_TSAN = "Unit tests (tsan)" + UNIT_TEST_UBSAN = "Unit tests (ubsan)" + + AST_FUZZER_TEST_DEBUG = "AST fuzzer (debug)" + AST_FUZZER_TEST_ASAN = "AST fuzzer (asan)" + AST_FUZZER_TEST_MSAN = "AST fuzzer (msan)" + AST_FUZZER_TEST_TSAN = "AST fuzzer (tsan)" + AST_FUZZER_TEST_UBSAN = "AST fuzzer (ubsan)" + + JEPSEN_KEEPER = "ClickHouse Keeper Jepsen" + JEPSEN_SERVER = "ClickHouse Server Jepsen" + + PERFORMANCE_TEST_AMD64 = "Performance Comparison" + PERFORMANCE_TEST_ARM64 = "Performance Comparison Aarch64" + + SQL_LOGIC_TEST = "Sqllogic test (release)" + + SQLANCER = "SQLancer (release)" + SQLANCER_DEBUG = "SQLancer (debug)" + SQLTEST = "SQLTest" + + COMPATIBILITY_TEST = "Compatibility check (amd64)" + COMPATIBILITY_TEST_ARM = "Compatibility check (aarch64)" + + CLICKBENCH_TEST = "ClickBench (amd64)" + CLICKBENCH_TEST_ARM = "ClickBench (aarch64)" + + LIBFUZZER_TEST = "libFuzzer tests" + + BUILD_CHECK = "ClickHouse build check" + # BUILD_CHECK_SPECIAL = "ClickHouse special build check" + + DOCS_CHECK = "Docs check" + BUGFIX_VALIDATE = "Bugfix validation" + + +# hack to concatenate Build and non-build jobs under JobNames class +for attr_name in dir(BuildNames): + if not attr_name.startswith("__") and not callable(getattr(BuildNames, attr_name)): + setattr(JobNames, attr_name, getattr(BuildNames, attr_name)) + + +class StatusNames(metaclass=WithIter): + """ + Class with statuses that aren't related to particular jobs + """ + + # overall CI report + CI = "CI running" + # mergeable status + MERGEABLE = "Mergeable Check" + # status of a sync pr + SYNC = "A Sync" + # PR formatting check status + PR_CHECK = "PR Check" + + +@dataclass +class DigestConfig: + # all files, dirs to include into digest, glob supported + include_paths: List[Union[str, Path]] = field(default_factory=list) + # file suffixes to exclude from digest + exclude_files: List[str] = field(default_factory=list) + # directories to exclude from digest + exclude_dirs: List[Union[str, Path]] = field(default_factory=list) + # docker names to include into digest + docker: List[str] = field(default_factory=list) + # git submodules digest + git_submodules: bool = False + + +@dataclass +class LabelConfig: + """ + configures different CI scenarios per CI Tag/GH label + """ + + run_jobs: Iterable[str] = frozenset() + + +@dataclass +class BuildConfig: + name: str + compiler: str + package_type: Literal["deb", "binary", "fuzzers"] + additional_pkgs: bool = False + debug_build: bool = False + coverage: bool = False + sanitizer: str = "" + tidy: bool = False + # sparse_checkout is needed only to test the option itself. + # No particular sense to use it in every build, since it slows down the job. + sparse_checkout: bool = False + comment: str = "" + static_binary_name: str = "" + + def export_env(self, export: bool = False) -> str: + def process(field_name: str, field: Union[bool, str]) -> str: + if isinstance(field, bool): + field = str(field).lower() + elif not isinstance(field, str): + field = "" + if export: + return f"export BUILD_{field_name.upper()}={repr(field)}" + return f"BUILD_{field_name.upper()}={field}" + + return "\n".join(process(k, v) for k, v in self.__dict__.items()) + + +@dataclass +class JobConfig: + """ + contains config parameters for job execution in CI workflow + """ + + # GH Runner type (tag from @Runners) + runner_type: str + # builds required for the job (applicable for test jobs) + required_builds: Optional[List[str]] = None + # build config for the build job (applicable for builds) + build_config: Optional[BuildConfig] = None + # configures digest calculation for the job + digest: DigestConfig = field(default_factory=DigestConfig) + # will be triggered for the job if omitted in CI workflow yml + run_command: str = "" + # job timeout, seconds + timeout: Optional[int] = None + # sets number of batches for a multi-batch job + num_batches: int = 1 + # label that enables job in CI, if set digest isn't used + run_by_label: str = "" + # to run always regardless of the job digest or/and label + run_always: bool = False + # if the job needs to be run on the release branch, including master (building packages, docker server). + # NOTE: Subsequent runs on the same branch with the similar digest are still considered skip-able. + required_on_release_branch: bool = False + # job is for pr workflow only + pr_only: bool = False + # job is for release/master branches only + release_only: bool = False + # to randomly pick and run one job among jobs in the same @random_bucket (PR branches only). + random_bucket: str = "" + # Do not set it. A list of batches to run. It will be set in runtime in accordance with ci cache and ci settings + batches: Optional[List[int]] = None + # Do not set it. A list of batches to await. It will be set in runtime in accordance with ci cache and ci settings + pending_batches: Optional[List[int]] = None + + def with_properties(self, **kwargs: Any) -> "JobConfig": + res = copy.deepcopy(self) + for k, v in kwargs.items(): + assert hasattr(self, k), f"Setting invalid attribute [{k}]" + setattr(res, k, v) + return res + + def get_required_build(self) -> str: + assert self.required_builds + return self.required_builds[0] + + +class CommonJobConfigs: + """ + Common job configs + """ + + BUILD_REPORT = JobConfig( + run_command="build_report_check.py", + digest=DigestConfig( + include_paths=[ + "./tests/ci/build_report_check.py", + "./tests/ci/upload_result_helper.py", + ], + ), + runner_type=Runners.STYLE_CHECKER_ARM, + ) + COMPATIBILITY_TEST = JobConfig( + digest=DigestConfig( + include_paths=["./tests/ci/compatibility_check.py"], + docker=["clickhouse/test-old-ubuntu", "clickhouse/test-old-centos"], + ), + run_command="compatibility_check.py", + runner_type=Runners.STYLE_CHECKER_ARM, + ) + INSTALL_TEST = JobConfig( + digest=DigestConfig( + include_paths=["./tests/ci/install_check.py"], + docker=["clickhouse/install-deb-test", "clickhouse/install-rpm-test"], + ), + run_command='install_check.py "$CHECK_NAME"', + runner_type=Runners.STYLE_CHECKER_ARM, + timeout=900, + ) + STATELESS_TEST = JobConfig( + digest=DigestConfig( + include_paths=[ + "./tests/ci/functional_test_check.py", + "./tests/queries/0_stateless/", + "./tests/clickhouse-test", + "./tests/config", + "./tests/*.txt", + ], + exclude_files=[".md"], + docker=["clickhouse/stateless-test"], + ), + run_command='functional_test_check.py "$CHECK_NAME"', + runner_type=Runners.FUNC_TESTER, + timeout=10800, + ) + STATEFUL_TEST = JobConfig( + digest=DigestConfig( + include_paths=[ + "./tests/ci/functional_test_check.py", + "./tests/queries/1_stateful/", + "./tests/clickhouse-test", + "./tests/config", + "./tests/*.txt", + ], + exclude_files=[".md"], + docker=["clickhouse/stateful-test"], + ), + run_command='functional_test_check.py "$CHECK_NAME"', + runner_type=Runners.FUNC_TESTER, + timeout=3600, + ) + STRESS_TEST = JobConfig( + digest=DigestConfig( + include_paths=[ + "./tests/queries/0_stateless/", + "./tests/queries/1_stateful/", + "./tests/clickhouse-test", + "./tests/config", + "./tests/*.txt", + ], + exclude_files=[".md"], + docker=["clickhouse/stress-test"], + ), + run_command="stress_check.py", + runner_type=Runners.STRESS_TESTER, + timeout=9000, + ) + UPGRADE_TEST = JobConfig( + digest=DigestConfig( + include_paths=["./tests/ci/upgrade_check.py"], + exclude_files=[".md"], + docker=["clickhouse/upgrade-check"], + ), + run_command="upgrade_check.py", + runner_type=Runners.STRESS_TESTER, + ) + INTEGRATION_TEST = JobConfig( + digest=DigestConfig( + include_paths=[ + "./tests/ci/integration_test_check.py", + "./tests/ci/integration_tests_runner.py", + "./tests/integration/", + ], + exclude_files=[".md"], + docker=IMAGES.copy(), + ), + run_command='integration_test_check.py "$CHECK_NAME"', + runner_type=Runners.STRESS_TESTER, + ) + ASTFUZZER_TEST = JobConfig( + digest=DigestConfig(), + run_command="ast_fuzzer_check.py", + run_always=True, + runner_type=Runners.FUZZER_UNIT_TESTER, + ) + UNIT_TEST = JobConfig( + digest=DigestConfig( + include_paths=["./tests/ci/unit_tests_check.py"], + exclude_files=[".md"], + docker=["clickhouse/unit-test"], + ), + run_command="unit_tests_check.py", + runner_type=Runners.FUZZER_UNIT_TESTER, + ) + PERF_TESTS = JobConfig( + digest=DigestConfig( + include_paths=[ + "./tests/ci/performance_comparison_check.py", + "./tests/performance/", + ], + exclude_files=[".md"], + docker=["clickhouse/performance-comparison"], + ), + run_command="performance_comparison_check.py", + runner_type=Runners.STRESS_TESTER, + ) + SQLLANCER_TEST = JobConfig( + digest=DigestConfig(), + run_command="sqlancer_check.py", + release_only=True, + run_always=True, + runner_type=Runners.FUZZER_UNIT_TESTER, + ) + SQLLOGIC_TEST = JobConfig( + digest=DigestConfig( + include_paths=["./tests/ci/sqllogic_test.py"], + exclude_files=[".md"], + docker=["clickhouse/sqllogic-test"], + ), + run_command="sqllogic_test.py", + timeout=10800, + release_only=True, + runner_type=Runners.STYLE_CHECKER_ARM, + ) + SQL_TEST = JobConfig( + digest=DigestConfig( + include_paths=["./tests/ci/sqltest.py"], + exclude_files=[".md"], + docker=["clickhouse/sqltest"], + ), + run_command="sqltest.py", + timeout=10800, + release_only=True, + runner_type=Runners.FUZZER_UNIT_TESTER, + ) + BUGFIX_TEST = JobConfig( + digest=DigestConfig(), + run_command="bugfix_validate_check.py", + timeout=900, + runner_type=Runners.FUNC_TESTER, + ) + DOCKER_SERVER = JobConfig( + required_on_release_branch=True, + run_command='docker_server.py --check-name "$CHECK_NAME" --release-type head --allow-build-reuse', + digest=DigestConfig( + include_paths=[ + "tests/ci/docker_server.py", + "./docker/server", + ] + ), + runner_type=Runners.STYLE_CHECKER, + ) + CLICKBENCH_TEST = JobConfig( + digest=DigestConfig( + include_paths=[ + "tests/ci/clickbench.py", + ], + docker=["clickhouse/clickbench"], + ), + run_command='clickbench.py "$CHECK_NAME"', + timeout=900, + runner_type=Runners.FUNC_TESTER, + ) + BUILD = JobConfig( + required_on_release_branch=True, + digest=DigestConfig( + include_paths=[ + "./src", + "./contrib/*-cmake", + "./contrib/consistent-hashing", + "./contrib/murmurhash", + "./contrib/libfarmhash", + "./contrib/pdqsort", + "./contrib/cityhash102", + "./contrib/sparse-checkout", + "./contrib/libmetrohash", + "./contrib/update-submodules.sh", + "./contrib/CMakeLists.txt", + "./CMakeLists.txt", + "./PreLoad.cmake", + "./cmake", + "./base", + "./programs", + "./packages", + "./docker/packager/packager", + "./rust", + "./tests/ci/version_helper.py", + # FIXME: This is a WA to rebuild the CH and recreate the Performance.tar.zst artifact + # when there are changes in performance test scripts. + # Due to the current design of the perf test we need to rebuild CH when the performance test changes, + # otherwise the changes will not be visible in the PerformanceTest job in CI + "./tests/performance", + ], + exclude_files=[".md"], + docker=["clickhouse/binary-builder"], + git_submodules=True, + ), + run_command="build_check.py $BUILD_NAME", + runner_type=Runners.BUILDER, + ) + + +REQUIRED_CHECKS = [ + StatusNames.PR_CHECK, + StatusNames.SYNC, + JobNames.BUILD_CHECK, + JobNames.DOCS_CHECK, + JobNames.FAST_TEST, + JobNames.STATEFUL_TEST_RELEASE, + JobNames.STATELESS_TEST_RELEASE, + JobNames.STATELESS_TEST_ASAN, + JobNames.STATELESS_TEST_FLAKY_ASAN, + JobNames.STATEFUL_TEST_ASAN, + JobNames.STYLE_CHECK, + JobNames.UNIT_TEST_ASAN, + JobNames.UNIT_TEST_MSAN, + JobNames.UNIT_TEST, + JobNames.UNIT_TEST_TSAN, + JobNames.UNIT_TEST_UBSAN, + JobNames.INTEGRATION_TEST_ASAN_OLD_ANALYZER, + JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE, +] + +# Jobs that run in Merge Queue if it's enabled +MQ_JOBS = [ + JobNames.STYLE_CHECK, + JobNames.FAST_TEST, + BuildNames.BINARY_RELEASE, + JobNames.UNIT_TEST, +] + + +@dataclass +class CheckDescription: + name: str + description: str # the check descriptions, will be put into the status table + match_func: Callable[[str], bool] # the function to check vs the commit status + + def __hash__(self) -> int: + return hash(self.name + self.description) + + +CHECK_DESCRIPTIONS = [ + CheckDescription( + StatusNames.PR_CHECK, + "Checks correctness of the PR's body", + lambda x: x == "PR Check", + ), + CheckDescription( + StatusNames.SYNC, + "If it fails, ask a maintainer for help", + lambda x: x == StatusNames.SYNC, + ), + CheckDescription( + "AST fuzzer", + "Runs randomly generated queries to catch program errors. " + "The build type is optionally given in parenthesis. " + "If it fails, ask a maintainer for help", + lambda x: x.startswith("AST fuzzer"), + ), + CheckDescription( + JobNames.BUGFIX_VALIDATE, + "Checks that either a new test (functional or integration) or there " + "some changed tests that fail with the binary built on master branch", + lambda x: x == JobNames.BUGFIX_VALIDATE, + ), + CheckDescription( + StatusNames.CI, + "A meta-check that indicates the running CI. Normally, it's in success or " + "pending state. The failed status indicates some problems with the PR", + lambda x: x == "CI running", + ), + CheckDescription( + "ClickHouse build check", + "Builds ClickHouse in various configurations for use in further steps. " + "You have to fix the builds that fail. Build logs often has enough " + "information to fix the error, but you might have to reproduce the failure " + "locally. The cmake options can be found in the build log, grepping for " + 'cmake. Use these options and follow the general build process', + lambda x: x.startswith("ClickHouse") and x.endswith("build check"), + ), + CheckDescription( + "Compatibility check", + "Checks that clickhouse binary runs on distributions with old libc " + "versions. If it fails, ask a maintainer for help", + lambda x: x.startswith("Compatibility check"), + ), + CheckDescription( + JobNames.DOCKER_SERVER, + "The check to build and optionally push the mentioned image to docker hub", + lambda x: x.startswith("Docker server"), + ), + CheckDescription( + JobNames.DOCKER_KEEPER, + "The check to build and optionally push the mentioned image to docker hub", + lambda x: x.startswith("Docker keeper"), + ), + CheckDescription( + JobNames.DOCS_CHECK, + "Builds and tests the documentation", + lambda x: x == JobNames.DOCS_CHECK, + ), + CheckDescription( + JobNames.FAST_TEST, + "Normally this is the first check that is ran for a PR. It builds ClickHouse " + 'and runs most of stateless functional tests, ' + "omitting some. If it fails, further checks are not started until it is fixed. " + "Look at the report to see which tests fail, then reproduce the failure " + 'locally as described here', + lambda x: x == JobNames.FAST_TEST, + ), + CheckDescription( + "Flaky tests", + "Checks if new added or modified tests are flaky by running them repeatedly, " + "in parallel, with more randomization. Functional tests are run 100 times " + "with address sanitizer, and additional randomization of thread scheduling. " + "Integration tests are run up to 10 times. If at least once a new test has " + "failed, or was too long, this check will be red. We don't allow flaky tests, " + 'read the doc', + lambda x: "tests flaky check" in x, + ), + CheckDescription( + "Install packages", + "Checks that the built packages are installable in a clear environment", + lambda x: x.startswith("Install packages ("), + ), + CheckDescription( + "Integration tests", + "The integration tests report. In parenthesis the package type is given, " + "and in square brackets are the optional part/total tests", + lambda x: x.startswith("Integration tests ("), + ), + CheckDescription( + StatusNames.MERGEABLE, + "Checks if all other necessary checks are successful", + lambda x: x == StatusNames.MERGEABLE, + ), + CheckDescription( + "Performance Comparison", + "Measure changes in query performance. The performance test report is " + 'described in detail here. ' + "In square brackets are the optional part/total tests", + lambda x: x.startswith("Performance Comparison"), + ), + CheckDescription( + "Push to Dockerhub", + "The check for building and pushing the CI related docker images to docker hub", + lambda x: x.startswith("Push") and "to Dockerhub" in x, + ), + CheckDescription( + "Sqllogic", + "Run clickhouse on the " + 'sqllogic ' + "test set against sqlite and checks that all statements are passed", + lambda x: x.startswith("Sqllogic test"), + ), + CheckDescription( + "SQLancer", + "Fuzzing tests that detect logical bugs with " + 'SQLancer tool', + lambda x: x.startswith("SQLancer"), + ), + CheckDescription( + "Stateful tests", + "Runs stateful functional tests for ClickHouse binaries built in various " + "configurations -- release, debug, with sanitizers, etc", + lambda x: x.startswith("Stateful tests ("), + ), + CheckDescription( + "Stateless tests", + "Runs stateless functional tests for ClickHouse binaries built in various " + "configurations -- release, debug, with sanitizers, etc", + lambda x: x.startswith("Stateless tests ("), + ), + CheckDescription( + "Stress test", + "Runs stateless functional tests concurrently from several clients to detect " + "concurrency-related errors", + lambda x: x.startswith("Stress test ("), + ), + CheckDescription( + JobNames.STYLE_CHECK, + "Runs a set of checks to keep the code style clean. If some of tests failed, " + "see the related log from the report", + lambda x: x == JobNames.STYLE_CHECK, + ), + CheckDescription( + "Unit tests", + "Runs the unit tests for different release types", + lambda x: x.startswith("Unit tests ("), + ), + CheckDescription( + "Upgrade check", + "Runs stress tests on server version from last release and then tries to " + "upgrade it to the version from the PR. It checks if the new server can " + "successfully startup without any errors, crashes or sanitizer asserts", + lambda x: x.startswith("Upgrade check ("), + ), + CheckDescription( + "ClickBench", + "Runs [ClickBench](https://github.com/ClickHouse/ClickBench/) with instant-attach table", + lambda x: x.startswith("ClickBench"), + ), + CheckDescription( + "Fallback for unknown", + "There's no description for the check yet, please add it to " + "tests/ci/ci_config.py:CHECK_DESCRIPTIONS", + lambda x: True, + ), +] diff --git a/tests/ci/ci_settings.py b/tests/ci/ci_settings.py index f25344c77019..a36fcf953ae3 100644 --- a/tests/ci/ci_settings.py +++ b/tests/ci/ci_settings.py @@ -3,7 +3,7 @@ from typing import Optional, List, Dict, Any, Iterable from ci_utils import normalize_string -from ci_config import CILabels, CI_CONFIG, JobConfig, JobNames +from ci_config import CI from git_helper import Runner as GitRunner, GIT_PREFIX from pr_info import PRInfo @@ -80,7 +80,7 @@ def create_from_pr_message( if not res.ci_jobs: res.ci_jobs = [] res.ci_jobs.append(match.removeprefix("job_")) - elif match.startswith("ci_set_") and match in CILabels: + elif match.startswith("ci_set_") and match in CI.Tags: if not res.ci_sets: res.ci_sets = [] res.ci_sets.append(match) @@ -97,15 +97,15 @@ def create_from_pr_message( res.exclude_keywords += [ normalize_string(keyword) for keyword in keywords ] - elif match == CILabels.NO_CI_CACHE: + elif match == CI.Tags.NO_CI_CACHE: res.no_ci_cache = True print("NOTE: CI Cache will be disabled") - elif match == CILabels.UPLOAD_ALL_ARTIFACTS: + elif match == CI.Tags.UPLOAD_ALL_ARTIFACTS: res.upload_all = True print("NOTE: All binary artifacts will be uploaded") - elif match == CILabels.DO_NOT_TEST_LABEL: + elif match == CI.Tags.DO_NOT_TEST_LABEL: res.do_not_test = True - elif match == CILabels.NO_MERGE_COMMIT: + elif match == CI.Tags.NO_MERGE_COMMIT: res.no_merge_commit = True print("NOTE: Merge Commit will be disabled") elif match.startswith("batch_"): @@ -131,18 +131,18 @@ def create_from_pr_message( def _check_if_selected( self, job: str, - job_config: JobConfig, + job_config: CI.JobConfig, is_release: bool, is_pr: bool, is_mq: bool, labels: Iterable[str], ) -> bool: # type: ignore #too-many-return-statements if self.do_not_test: - label_config = CI_CONFIG.get_label_config(CILabels.DO_NOT_TEST_LABEL) - assert label_config, f"Unknown tag [{CILabels.DO_NOT_TEST_LABEL}]" + label_config = CI.get_tag_config(CI.Tags.DO_NOT_TEST_LABEL) + assert label_config, f"Unknown tag [{CI.Tags.DO_NOT_TEST_LABEL}]" if job in label_config.run_jobs: print( - f"Job [{job}] present in CI set [{CILabels.DO_NOT_TEST_LABEL}] - pass" + f"Job [{job}] present in CI set [{CI.Tags.DO_NOT_TEST_LABEL}] - pass" ) return True return False @@ -164,7 +164,7 @@ def _check_if_selected( to_deny = False if self.include_keywords: - if job == JobNames.STYLE_CHECK: + if job == CI.JobNames.STYLE_CHECK: # never exclude Style Check by include keywords return True for keyword in self.include_keywords: @@ -175,7 +175,7 @@ def _check_if_selected( if self.ci_sets: for tag in self.ci_sets: - label_config = CI_CONFIG.get_label_config(tag) + label_config = CI.get_tag_config(tag) assert label_config, f"Unknown tag [{tag}]" if job in label_config.run_jobs: print(f"Job [{job}] present in CI set [{tag}] - pass") @@ -197,12 +197,12 @@ def _check_if_selected( def apply( self, - job_configs: Dict[str, JobConfig], + job_configs: Dict[str, CI.JobConfig], is_release: bool, is_pr: bool, is_mq: bool, labels: Iterable[str], - ) -> Dict[str, JobConfig]: + ) -> Dict[str, CI.JobConfig]: """ Apply CI settings from pr body """ @@ -220,7 +220,7 @@ def apply( add_parents = [] for job in list(res): - parent_jobs = CI_CONFIG.get_job_parents(job) + parent_jobs = CI.get_job_parents(job) for parent_job in parent_jobs: if parent_job not in res: add_parents.append(parent_job) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index a0d6495452f0..96cf700ed77f 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -17,7 +17,7 @@ from github.IssueComment import IssueComment from github.Repository import Repository -from ci_config import CHECK_DESCRIPTIONS, CheckDescription, StatusNames, CIConfig +from ci_config import CI from env_helper import GITHUB_REPOSITORY, GITHUB_UPSTREAM_REPOSITORY, TEMP_PATH from lambda_shared_package.lambda_shared.pr import Labels from pr_info import PRInfo @@ -160,7 +160,7 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: if not statuses: return - if not [status for status in statuses if status.context == StatusNames.CI]: + if not [status for status in statuses if status.context == CI.StatusNames.CI]: # This is the case, when some statuses already exist for the check, # but not the StatusNames.CI. We should create it as pending. # W/o pr_info to avoid recursion, and yes, one extra create_ci_report @@ -169,7 +169,7 @@ def set_status_comment(commit: Commit, pr_info: PRInfo) -> None: PENDING, create_ci_report(pr_info, statuses), "The report for running CI", - StatusNames.CI, + CI.StatusNames.CI, ) # We update the report in generate_status_comment function, so do it each @@ -212,20 +212,20 @@ def generate_status_comment(pr_info: PRInfo, statuses: CommitStatuses) -> str: f"\n" ) # group checks by the name to get the worst one per each - grouped_statuses = {} # type: Dict[CheckDescription, CommitStatuses] + grouped_statuses = {} # type: Dict[CI.CheckDescription, CommitStatuses] for status in statuses: cd = None - for c in CHECK_DESCRIPTIONS: + for c in CI.CHECK_DESCRIPTIONS: if c.match_func(status.context): cd = c break - if cd is None or cd == CHECK_DESCRIPTIONS[-1]: + if cd is None or cd == CI.CHECK_DESCRIPTIONS[-1]: # This is the case for either non-found description or a fallback - cd = CheckDescription( + cd = CI.CheckDescription( status.context, - CHECK_DESCRIPTIONS[-1].description, - CHECK_DESCRIPTIONS[-1].match_func, + CI.CHECK_DESCRIPTIONS[-1].description, + CI.CHECK_DESCRIPTIONS[-1].match_func, ) if cd in grouped_statuses: @@ -301,7 +301,7 @@ def create_ci_report(pr_info: PRInfo, statuses: CommitStatuses) -> str: ) ) return upload_results( - S3Helper(), pr_info.number, pr_info.sha, test_results, [], StatusNames.CI + S3Helper(), pr_info.number, pr_info.sha, test_results, [], CI.StatusNames.CI ) @@ -435,7 +435,7 @@ def set_mergeable_check( state, report_url, format_description(description), - StatusNames.MERGEABLE, + CI.StatusNames.MERGEABLE, ) @@ -443,7 +443,7 @@ def update_mergeable_check(commit: Commit, pr_info: PRInfo, check_name: str) -> "check if the check_name in REQUIRED_CHECKS and then trigger update" not_run = ( pr_info.labels.intersection({Labels.SKIP_MERGEABLE_CHECK, Labels.RELEASE}) - or not CIConfig.is_required(check_name) + or not CI.is_required(check_name) or pr_info.release_pr or pr_info.number == 0 ) @@ -465,13 +465,11 @@ def trigger_mergeable_check( workflow_failed: bool = False, ) -> StatusType: """calculate and update StatusNames.MERGEABLE""" - required_checks = [ - status for status in statuses if CIConfig.is_required(status.context) - ] + required_checks = [status for status in statuses if CI.is_required(status.context)] mergeable_status = None for status in statuses: - if status.context == StatusNames.MERGEABLE: + if status.context == CI.StatusNames.MERGEABLE: mergeable_status = status break @@ -548,7 +546,7 @@ def update_upstream_sync_status( "Using commit %s to post the %s status `%s`: [%s]", last_synced_upstream_commit.sha, sync_status, - StatusNames.SYNC, + CI.StatusNames.SYNC, "", ) post_commit_status( @@ -556,7 +554,7 @@ def update_upstream_sync_status( sync_status, "", "", - StatusNames.SYNC, + CI.StatusNames.SYNC, ) trigger_mergeable_check( last_synced_upstream_commit, diff --git a/tests/ci/digest_helper.py b/tests/ci/digest_helper.py index 8d6ec127f6ee..4dcfb03c04fa 100644 --- a/tests/ci/digest_helper.py +++ b/tests/ci/digest_helper.py @@ -9,10 +9,10 @@ from sys import modules from docker_images_helper import get_images_info -from ci_config import DigestConfig from git_helper import Runner from env_helper import ROOT_DIR from ci_utils import cd +from ci_config import CI DOCKER_DIGEST_LEN = 12 JOB_DIGEST_LEN = 10 @@ -139,20 +139,21 @@ def get_all_digests(self) -> Dict: class JobDigester: - def __init__(self): + def __init__(self, dry_run: bool = False): self.dd = DockerDigester() self.cache: Dict[str, str] = {} + self.dry_run = dry_run @staticmethod - def _get_config_hash(digest_config: DigestConfig) -> str: + def _get_config_hash(digest_config: CI.DigestConfig) -> str: data_dict = asdict(digest_config) hash_obj = md5() hash_obj.update(str(data_dict).encode()) hash_string = hash_obj.hexdigest() return hash_string - def get_job_digest(self, digest_config: DigestConfig) -> str: - if not digest_config.include_paths: + def get_job_digest(self, digest_config: CI.DigestConfig) -> str: + if not digest_config.include_paths or self.dry_run: # job is not for digest return "f" * JOB_DIGEST_LEN diff --git a/tests/ci/download_binary.py b/tests/ci/download_binary.py index 79db1e57d62d..b0b5659ca831 100755 --- a/tests/ci/download_binary.py +++ b/tests/ci/download_binary.py @@ -8,7 +8,7 @@ from pathlib import Path from build_download_helper import download_build_with_progress -from ci_config import CI_CONFIG +from ci_config import CI from env_helper import RUNNER_TEMP, S3_ARTIFACT_DOWNLOAD_TEMPLATE from git_helper import Git, commit from version_helper import get_version_from_repo, version_arg @@ -59,7 +59,8 @@ def main(): temp_path.mkdir(parents=True, exist_ok=True) for build in args.build_names: # check if it's in CI_CONFIG - config = CI_CONFIG.build_config[build] + config = CI.JOB_CONFIGS[build].build_config + assert config if args.rename and config.static_binary_name: path = temp_path / f"clickhouse-{config.static_binary_name}" else: diff --git a/tests/ci/env_helper.py b/tests/ci/env_helper.py index 36732bd7c9fa..5217e4035da1 100644 --- a/tests/ci/env_helper.py +++ b/tests/ci/env_helper.py @@ -9,8 +9,9 @@ module_dir = p.abspath(p.dirname(__file__)) git_root = p.abspath(p.join(module_dir, "..", "..")) + ROOT_DIR = git_root -CI = bool(os.getenv("CI")) +IS_CI = bool(os.getenv("CI")) TEMP_PATH = os.getenv("TEMP_PATH", p.abspath(p.join(module_dir, "./tmp"))) REPORT_PATH = f"{TEMP_PATH}/reports" # FIXME: latest should not be used in CI, set temporary for transition to "docker with digest as a tag" diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 904b565ad86a..bd83f8757901 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -4,7 +4,7 @@ from github import Github -from ci_config import StatusNames +from ci_config import CI from commit_status_helper import ( get_commit, get_commit_filtered_statuses, @@ -71,7 +71,7 @@ def main(): can_set_green_mergeable_status=True, ) - ci_running_statuses = [s for s in statuses if s.context == StatusNames.CI] + ci_running_statuses = [s for s in statuses if s.context == CI.StatusNames.CI] if not ci_running_statuses: return # Take the latest status @@ -81,7 +81,11 @@ def main(): has_pending = False error_cnt = 0 for status in statuses: - if status.context in (StatusNames.MERGEABLE, StatusNames.CI, StatusNames.SYNC): + if status.context in ( + CI.StatusNames.MERGEABLE, + CI.StatusNames.CI, + CI.StatusNames.SYNC, + ): # do not account these statuses continue if status.state == PENDING: @@ -108,7 +112,7 @@ def main(): ci_state, ci_status.target_url, description, - StatusNames.CI, + CI.StatusNames.CI, pr_info, dump_to_file=True, ) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 4abaeac30b7d..87f721cfde73 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -18,7 +18,7 @@ from itertools import chain from typing import Any, Dict -from env_helper import CI +from env_helper import IS_CI from integration_test_images import IMAGES MAX_RETRY = 1 @@ -1004,7 +1004,7 @@ def run(): logging.info("Running tests") - if CI: + if IS_CI: # Avoid overlaps with previous runs logging.info("Clearing dmesg before run") subprocess.check_call("sudo -E dmesg --clear", shell=True) @@ -1012,7 +1012,7 @@ def run(): state, description, test_results, _ = runner.run_impl(repo_path, build_path) logging.info("Tests finished") - if CI: + if IS_CI: # Dump dmesg (to capture possible OOMs) logging.info("Dumping dmesg") subprocess.check_call("sudo -E dmesg -T", shell=True) diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 1e61fd9fab75..f91a3f080c0c 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -13,7 +13,6 @@ from build_download_helper import ( download_build_with_progress, - get_build_name_for_check, read_build_urls, ) from compress_files import compress_fast @@ -25,6 +24,7 @@ from ssh import SSHKey from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_config import CI JEPSEN_GROUP_NAME = "jepsen_group" @@ -224,7 +224,7 @@ def main(): head = requests.head(build_url, timeout=60) assert head.status_code == 200, f"Clickhouse binary not found: {build_url}" else: - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) urls = read_build_urls(build_name, REPORT_PATH) build_url = None for url in urls: diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index 0c779b515bdf..b3488ac0af21 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -12,7 +12,7 @@ from github import Github from build_download_helper import download_builds_filter -from ci_config import CI_CONFIG +from ci_config import CI from clickhouse_helper import get_instance_id, get_instance_type from commit_status_helper import get_commit from docker_images_helper import get_docker_image, pull_image @@ -83,7 +83,7 @@ def main(): assert ( check_name ), "Check name must be provided as an input arg or in CHECK_NAME env" - required_build = CI_CONFIG.test_configs[check_name].required_build + required_build = CI.JOB_CONFIGS[check_name].get_required_build() with open(GITHUB_EVENT_PATH, "r", encoding="utf-8") as event_file: event = json.load(event_file) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index dda5b30f1e3d..fb25a29cc575 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -316,7 +316,9 @@ def __init__( @property def is_master(self) -> bool: - return self.number == 0 and self.head_ref == "master" + return ( + self.number == 0 and self.head_ref == "master" and not self.is_merge_queue + ) @property def is_release(self) -> bool: @@ -324,7 +326,10 @@ def is_release(self) -> bool: @property def is_pr(self): - return self.event_type == EventType.PULL_REQUEST + if self.event_type == EventType.PULL_REQUEST: + assert self.number + return True + return False @property def is_scheduled(self) -> bool: @@ -353,9 +358,6 @@ def fetch_changed_files(self): if self.changed_files_requested: return - if not getattr(self, "diff_urls", False): - raise TypeError("The event does not have diff URLs") - for diff_url in self.diff_urls: response = get_gh_api( diff_url, diff --git a/tests/ci/report.py b/tests/ci/report.py index ee58efdba524..bdaa2e151300 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -21,7 +21,7 @@ ) from build_download_helper import get_gh_api -from ci_config import CI_CONFIG, BuildConfig +from ci_config import CI from ci_utils import normalize_string from env_helper import REPORT_PATH, TEMP_PATH @@ -412,6 +412,7 @@ def load_any(cls, build_name: str, pr_number: int, head_ref: str): # type: igno ref_report = None master_report = None any_report = None + Path(REPORT_PATH).mkdir(parents=True, exist_ok=True) for file in Path(REPORT_PATH).iterdir(): if f"{build_name}.json" in file.name: any_report = file @@ -448,8 +449,10 @@ def as_json(self) -> str: return json.dumps(asdict(self), indent=2) @property - def build_config(self) -> Optional[BuildConfig]: - return CI_CONFIG.build_config.get(self.build_name, None) + def build_config(self) -> Optional[CI.BuildConfig]: + if self.build_name not in CI.JOB_CONFIGS: + return None + return CI.JOB_CONFIGS[self.build_name].build_config @property def comment(self) -> str: diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 131cbeef7860..00942352dde7 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -5,7 +5,6 @@ from github import Github -from ci_config import StatusNames from commit_status_helper import ( create_ci_report, format_description, @@ -24,6 +23,7 @@ ) from pr_info import PRInfo from report import FAILURE, PENDING, SUCCESS, StatusType +from ci_config import CI TRUSTED_ORG_IDS = { 54801242, # clickhouse @@ -208,7 +208,7 @@ def main(): PENDING, ci_report_url, description, - StatusNames.CI, + CI.StatusNames.CI, pr_info, ) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index bff53f00ad35..da0ccb2b74d8 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -11,7 +11,7 @@ import botocore # type: ignore from compress_files import compress_file_fast from env_helper import ( - CI, + IS_CI, RUNNER_TEMP, S3_BUILDS_BUCKET, S3_DOWNLOAD, @@ -111,13 +111,13 @@ def delete_file_from_s3(self, bucket_name: str, s3_path: str) -> None: self.client.delete_object(Bucket=bucket_name, Key=s3_path) def upload_test_report_to_s3(self, file_path: Path, s3_path: str) -> str: - if CI: + if IS_CI: return self._upload_file_to_s3(S3_TEST_REPORTS_BUCKET, file_path, s3_path) return S3Helper.copy_file_to_local(S3_TEST_REPORTS_BUCKET, file_path, s3_path) def upload_build_file_to_s3(self, file_path: Path, s3_path: str) -> str: - if CI: + if IS_CI: return self._upload_file_to_s3(S3_BUILDS_BUCKET, file_path, s3_path) return S3Helper.copy_file_to_local(S3_BUILDS_BUCKET, file_path, s3_path) @@ -255,7 +255,7 @@ def task(file_path: Path) -> Union[str, List[str]]: if full_fs_path.is_symlink(): if upload_symlinks: - if CI: + if IS_CI: return self._upload_file_to_s3( bucket_name, full_fs_path, @@ -266,7 +266,7 @@ def task(file_path: Path) -> Union[str, List[str]]: ) return [] - if CI: + if IS_CI: return self._upload_file_to_s3( bucket_name, full_fs_path, full_s3_path + "/" + file_path.name ) @@ -331,7 +331,7 @@ def list_prefix_non_recursive( return result def url_if_exists(self, key: str, bucket: str = S3_BUILDS_BUCKET) -> str: - if not CI: + if not IS_CI: local_path = self.local_path(bucket, key) if local_path.exists(): return local_path.as_uri() @@ -345,7 +345,7 @@ def url_if_exists(self, key: str, bucket: str = S3_BUILDS_BUCKET) -> str: @staticmethod def get_url(bucket: str, key: str) -> str: - if CI: + if IS_CI: return S3Helper.s3_url(bucket, key) return S3Helper.local_path(bucket, key).as_uri() diff --git a/tests/ci/sqlancer_check.py b/tests/ci/sqlancer_check.py index 9d33c4805985..a68db8b97919 100644 --- a/tests/ci/sqlancer_check.py +++ b/tests/ci/sqlancer_check.py @@ -6,12 +6,13 @@ import sys from pathlib import Path -from build_download_helper import get_build_name_for_check, read_build_urls +from build_download_helper import read_build_urls from docker_images_helper import DockerImage, get_docker_image, pull_image from env_helper import REPORT_PATH, TEMP_PATH from report import FAILURE, SUCCESS, JobReport, TestResult, TestResults from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_config import CI IMAGE_NAME = "clickhouse/sqlancer-test" @@ -43,7 +44,7 @@ def main(): docker_image = pull_image(get_docker_image(IMAGE_NAME)) - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) urls = read_build_urls(build_name, reports_path) if not urls: raise ValueError("No build URLs found") diff --git a/tests/ci/sqltest.py b/tests/ci/sqltest.py index c8c2adbbd564..8e6ca6ff87fa 100644 --- a/tests/ci/sqltest.py +++ b/tests/ci/sqltest.py @@ -6,12 +6,13 @@ import sys from pathlib import Path -from build_download_helper import get_build_name_for_check, read_build_urls +from build_download_helper import read_build_urls from docker_images_helper import get_docker_image, pull_image from env_helper import REPORT_PATH, TEMP_PATH from pr_info import PRInfo from report import SUCCESS, JobReport, TestResult from stopwatch import Stopwatch +from ci_config import CI IMAGE_NAME = "clickhouse/sqltest" @@ -49,7 +50,7 @@ def main(): docker_image = pull_image(get_docker_image(IMAGE_NAME)) - build_name = get_build_name_for_check(check_name) + build_name = CI.get_required_build_name(check_name) print(build_name) urls = read_build_urls(build_name, reports_path) if not urls: diff --git a/tests/ci/ssh.py b/tests/ci/ssh.py index 321826fcf447..89d90d724d2d 100644 --- a/tests/ci/ssh.py +++ b/tests/ci/ssh.py @@ -37,9 +37,9 @@ def start(self): ssh_options = ( "," + os.environ["SSH_OPTIONS"] if os.environ.get("SSH_OPTIONS") else "" ) - os.environ[ - "SSH_OPTIONS" - ] = f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no" + os.environ["SSH_OPTIONS"] = ( + f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no" + ) def add(self, key): key_pub = self._key_pub(key) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 9906d87a8c07..35e48246be98 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -13,7 +13,7 @@ import magic from docker_images_helper import get_docker_image, pull_image -from env_helper import CI, REPO_COPY, TEMP_PATH +from env_helper import IS_CI, REPO_COPY, TEMP_PATH from git_helper import GIT_PREFIX, git_runner from pr_info import PRInfo from report import ERROR, FAILURE, SUCCESS, JobReport, TestResults, read_test_results @@ -152,7 +152,7 @@ def main(): run_cpp_check = True run_shell_check = True run_python_check = True - if CI and pr_info.number > 0: + if IS_CI and pr_info.number > 0: pr_info.fetch_changed_files() run_cpp_check = any( not (is_python(file) or is_shell(file)) for file in pr_info.changed_files diff --git a/tests/ci/sync_pr.py b/tests/ci/sync_pr.py index f1073603e8df..8251ccbaf382 100644 --- a/tests/ci/sync_pr.py +++ b/tests/ci/sync_pr.py @@ -5,12 +5,12 @@ import argparse import sys -from ci_config import StatusNames from commit_status_helper import get_commit, post_commit_status from get_robot_token import get_best_robot_token from github_helper import GitHub from pr_info import PRInfo from report import SUCCESS +from ci_config import CI def parse_args() -> argparse.Namespace: @@ -75,7 +75,7 @@ def set_sync_status(gh, pr_info, sync_pr): if sync_pr.mergeable_state == "clean": print(f"Sync PR [{sync_pr.number}] is clean") post_commit_status( - get_commit(gh, pr_info.sha), SUCCESS, "", "", StatusNames.SYNC + get_commit(gh, pr_info.sha), SUCCESS, "", "", CI.StatusNames.SYNC ) else: print( diff --git a/tests/ci/test_ci_cache.py b/tests/ci/test_ci_cache.py index b1be07098033..81d649b246b7 100644 --- a/tests/ci/test_ci_cache.py +++ b/tests/ci/test_ci_cache.py @@ -5,12 +5,12 @@ import shutil from typing import Dict, Set import unittest -from ci_config import Build, JobNames from s3_helper import S3Helper from ci_cache import CiCache from digest_helper import JOB_DIGEST_LEN from commit_status_helper import CommitStatusData from env_helper import S3_BUILDS_BUCKET, TEMP_PATH +from ci_config import CI def _create_mock_digest_1(string): @@ -21,8 +21,8 @@ def _create_mock_digest_2(string): return md5((string + "+nonce").encode("utf-8")).hexdigest()[:JOB_DIGEST_LEN] -DIGESTS = {job: _create_mock_digest_1(job) for job in JobNames} -DIGESTS2 = {job: _create_mock_digest_2(job) for job in JobNames} +DIGESTS = {job: _create_mock_digest_1(job) for job in CI.JobNames} +DIGESTS2 = {job: _create_mock_digest_2(job) for job in CI.JobNames} # pylint:disable=protected-access @@ -84,8 +84,10 @@ def test_cache(self): NUM_BATCHES = 10 DOCS_JOBS_NUM = 1 - assert len(set(job for job in JobNames)) == len(list(job for job in JobNames)) - NONDOCS_JOBS_NUM = len(set(job for job in JobNames)) - DOCS_JOBS_NUM + assert len(set(job for job in CI.JobNames)) == len( + list(job for job in CI.JobNames) + ) + NONDOCS_JOBS_NUM = len(set(job for job in CI.JobNames)) - DOCS_JOBS_NUM PR_NUM = 123456 status = CommitStatusData( @@ -97,13 +99,13 @@ def test_cache(self): ) ### add some pending statuses for two batches, non-release branch - for job in JobNames: + for job in CI.JobNames: ci_cache.push_pending(job, [0, 1, 2], NUM_BATCHES, release_branch=False) ci_cache_2.push_pending(job, [0, 1, 2], NUM_BATCHES, release_branch=False) ### add success status for 0 batch, non-release branch batch = 0 - for job in JobNames: + for job in CI.JobNames: ci_cache.push_successful( job, batch, NUM_BATCHES, status, release_branch=False ) @@ -113,21 +115,17 @@ def test_cache(self): ### add failed status for 2 batch, non-release branch batch = 2 - for job in JobNames: + for job in CI.JobNames: ci_cache.push_failed(job, batch, NUM_BATCHES, status, release_branch=False) ci_cache_2.push_failed( job, batch, NUM_BATCHES, status, release_branch=False ) ### check all expected directories were created on s3 mock - expected_build_path_1 = f"{CiCache.JobType.SRCS.value}-{_create_mock_digest_1(Build.PACKAGE_RELEASE)}" - expected_docs_path_1 = ( - f"{CiCache.JobType.DOCS.value}-{_create_mock_digest_1(JobNames.DOCS_CHECK)}" - ) - expected_build_path_2 = f"{CiCache.JobType.SRCS.value}-{_create_mock_digest_2(Build.PACKAGE_RELEASE)}" - expected_docs_path_2 = ( - f"{CiCache.JobType.DOCS.value}-{_create_mock_digest_2(JobNames.DOCS_CHECK)}" - ) + expected_build_path_1 = f"{CiCache.JobType.SRCS.value}-{_create_mock_digest_1(CI.BuildNames.PACKAGE_RELEASE)}" + expected_docs_path_1 = f"{CiCache.JobType.DOCS.value}-{_create_mock_digest_1(CI.JobNames.DOCS_CHECK)}" + expected_build_path_2 = f"{CiCache.JobType.SRCS.value}-{_create_mock_digest_2(CI.BuildNames.PACKAGE_RELEASE)}" + expected_docs_path_2 = f"{CiCache.JobType.DOCS.value}-{_create_mock_digest_2(CI.JobNames.DOCS_CHECK)}" self.assertCountEqual( list(s3_mock.files_on_s3_paths.keys()), [ @@ -174,7 +172,7 @@ def test_cache(self): ) ### check statuses for all jobs in cache - for job in JobNames: + for job in CI.JobNames: self.assertEqual( ci_cache.is_successful(job, 0, NUM_BATCHES, release_branch=False), True ) @@ -212,7 +210,7 @@ def test_cache(self): assert status2 is None ### add some more pending statuses for two batches and for a release branch - for job in JobNames: + for job in CI.JobNames: ci_cache.push_pending( job, batches=[0, 1], num_batches=NUM_BATCHES, release_branch=True ) @@ -226,7 +224,7 @@ def test_cache(self): sha="deadbeaf2", pr_num=PR_NUM, ) - for job in JobNames: + for job in CI.JobNames: ci_cache.push_successful(job, 0, NUM_BATCHES, status, release_branch=True) ### check number of cache files is as expected @@ -249,7 +247,7 @@ def test_cache(self): ) ### check statuses - for job in JobNames: + for job in CI.JobNames: self.assertEqual(ci_cache.is_successful(job, 0, NUM_BATCHES, False), True) self.assertEqual(ci_cache.is_successful(job, 0, NUM_BATCHES, True), True) self.assertEqual(ci_cache.is_successful(job, 1, NUM_BATCHES, False), False) @@ -273,7 +271,7 @@ def test_cache(self): ### create new cache object and verify the same checks ci_cache = CiCache(s3_mock, DIGESTS) - for job in JobNames: + for job in CI.JobNames: self.assertEqual(ci_cache.is_successful(job, 0, NUM_BATCHES, False), True) self.assertEqual(ci_cache.is_successful(job, 0, NUM_BATCHES, True), True) self.assertEqual(ci_cache.is_successful(job, 1, NUM_BATCHES, False), False) diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index badbc4c5dcfc..76ab3b82fa5c 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -1,30 +1,306 @@ #!/usr/bin/env python3 import unittest -from ci_config import CIStages, JobNames, CI_CONFIG, Runners +from ci_config import CI +import ci as CIPY +from ci_settings import CiSettings +from pr_info import PRInfo, EventType +from s3_helper import S3Helper +from ci_cache import CiCache +from ci_utils import normalize_string + + +_TEST_EVENT_JSON = {"dummy": "dummy"} + +# pylint:disable=protected-access class TestCIConfig(unittest.TestCase): def test_runner_config(self): """check runner is provided w/o exception""" - for job in JobNames: - runner = CI_CONFIG.get_runner_type(job) - self.assertIn(runner, Runners) + for job in CI.JobNames: + self.assertIn(CI.JOB_CONFIGS[job].runner_type, CI.Runners) def test_job_stage_config(self): - """check runner is provided w/o exception""" - for job in JobNames: - stage = CI_CONFIG.get_job_ci_stage(job) - if job in [ - JobNames.STYLE_CHECK, - JobNames.FAST_TEST, - JobNames.JEPSEN_KEEPER, - JobNames.BUILD_CHECK, - JobNames.BUILD_CHECK_SPECIAL, - ]: - assert ( - stage == CIStages.NA - ), "These jobs are not in CI stages, must be NA" + """ + check runner is provided w/o exception + """ + # check stages + for job in CI.JobNames: + if job in CI.BuildNames: + self.assertTrue( + CI.get_job_ci_stage(job) + in (CI.WorkflowStages.BUILDS_1, CI.WorkflowStages.BUILDS_2) + ) + else: + if job in ( + CI.JobNames.STYLE_CHECK, + CI.JobNames.FAST_TEST, + CI.JobNames.JEPSEN_SERVER, + CI.JobNames.JEPSEN_KEEPER, + CI.JobNames.BUILD_CHECK, + ): + self.assertEqual( + CI.get_job_ci_stage(job), + CI.WorkflowStages.NA, + msg=f"Stage for [{job}] is not correct", + ) + else: + self.assertTrue( + CI.get_job_ci_stage(job) + in (CI.WorkflowStages.TESTS_1, CI.WorkflowStages.TESTS_3), + msg=f"Stage for [{job}] is not correct", + ) + + def test_build_jobs_configs(self): + """ + check build jobs have non-None build_config attribute + check test jobs have None build_config attribute + """ + for job in CI.JobNames: + if job in CI.BuildNames: + self.assertTrue( + isinstance(CI.JOB_CONFIGS[job].build_config, CI.BuildConfig) + ) else: - assert stage != CIStages.NA, f"stage not found for [{job}]" - self.assertIn(stage, CIStages) + self.assertTrue(CI.JOB_CONFIGS[job].build_config is None) + + def test_ci_py_for_pull_request(self): + """ + checks ci.py job configuration + """ + settings = CiSettings() + settings.no_ci_cache = True + settings.ci_sets = [CI.Tags.CI_SET_BUILDS] + settings.include_keywords = [ + "package", + "integration", + "upgrade", + "clickHouse_build_check", + "stateless", + ] + settings.exclude_keywords = ["asan", "aarch64"] + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + # make it pull request info + pr_info.event_type = EventType.PULL_REQUEST + pr_info.number = 12345 + assert pr_info.is_pr and not pr_info.is_release and not pr_info.is_master + assert not pr_info.is_merge_queue + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + actual_jobs_to_do = list(ci_cache.jobs_to_do) + expected_jobs_to_do = [] + for set_ in settings.ci_sets: + tag_config = CI.get_tag_config(set_) + assert tag_config + set_jobs = tag_config.run_jobs + for job in set_jobs: + if any(k in normalize_string(job) for k in settings.exclude_keywords): + continue + expected_jobs_to_do.append(job) + for job, config in CI.JOB_CONFIGS.items(): + if not any( + keyword in normalize_string(job) + for keyword in settings.include_keywords + ): + continue + if any( + keyword in normalize_string(job) + for keyword in settings.exclude_keywords + ): + continue + if config.random_bucket: + continue + if job not in expected_jobs_to_do: + expected_jobs_to_do.append(job) + + random_buckets = [] + for job, config in ci_cache.jobs_to_do.items(): + if config.random_bucket: + self.assertTrue( + config.random_bucket not in random_buckets, + "Only one job must be picked up from each random bucket", + ) + random_buckets.append(config.random_bucket) + actual_jobs_to_do.remove(job) + + self.assertCountEqual(expected_jobs_to_do, actual_jobs_to_do) + + def test_ci_py_for_pull_request_no_settings(self): + """ + checks ci.py job configuration in PR with empty ci_settings + """ + settings = CiSettings() + settings.no_ci_cache = True + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + # make it pull request info + pr_info.event_type = EventType.PULL_REQUEST + pr_info.number = 12345 + assert pr_info.is_pr and not pr_info.is_release and not pr_info.is_master + assert not pr_info.is_merge_queue + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + actual_jobs_to_do = list(ci_cache.jobs_to_do) + expected_jobs_to_do = [] + for job, config in CI.JOB_CONFIGS.items(): + if config.random_bucket: + continue + if config.release_only: + continue + if config.run_by_label: + continue + expected_jobs_to_do.append(job) + + random_buckets = [] + for job, config in ci_cache.jobs_to_do.items(): + if config.random_bucket: + self.assertTrue( + config.random_bucket not in random_buckets, + "Only one job must be picked up from each random bucket", + ) + random_buckets.append(config.random_bucket) + actual_jobs_to_do.remove(job) + + self.assertCountEqual(expected_jobs_to_do, actual_jobs_to_do) + + def test_ci_py_for_master(self): + """ + checks ci.py job configuration + """ + settings = CiSettings() + settings.no_ci_cache = True + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + pr_info.event_type = EventType.PUSH + assert pr_info.number == 0 and pr_info.is_release and not pr_info.is_merge_queue + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + actual_jobs_to_do = list(ci_cache.jobs_to_do) + expected_jobs_to_do = [] + for job, config in CI.JOB_CONFIGS.items(): + if config.pr_only: + continue + if config.run_by_label: + continue + if job in CI.MQ_JOBS: + continue + expected_jobs_to_do.append(job) + self.assertCountEqual(expected_jobs_to_do, actual_jobs_to_do) + + def test_ci_py_for_merge_queue(self): + """ + checks ci.py job configuration + """ + settings = CiSettings() + settings.no_ci_cache = True + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + # make it merge_queue + pr_info.event_type = EventType.MERGE_QUEUE + assert ( + pr_info.number == 0 + and pr_info.is_merge_queue + and not pr_info.is_release + and not pr_info.is_master + and not pr_info.is_pr + ) + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + actual_jobs_to_do = list(ci_cache.jobs_to_do) + expected_jobs_to_do = [ + "Style check", + "Fast test", + "binary_release", + "Unit tests (release)", + ] + self.assertCountEqual(expected_jobs_to_do, actual_jobs_to_do) + + def test_ci_py_await(self): + """ + checks ci.py job configuration + """ + settings = CiSettings() + settings.no_ci_cache = True + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + pr_info.event_type = EventType.PUSH + pr_info.number = 0 + assert pr_info.is_release and not pr_info.is_merge_queue + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + self.assertTrue(not ci_cache.jobs_to_skip, "Must be no jobs in skip list") + all_jobs_in_wf = list(ci_cache.jobs_to_do) + assert not ci_cache.jobs_to_wait + ci_cache.await_pending_jobs(is_release=pr_info.is_release, dry_run=True) + assert not ci_cache.jobs_to_skip + assert not ci_cache.jobs_to_wait + + # pretend there are pending jobs that we neet to wait + ci_cache.jobs_to_wait = dict(ci_cache.jobs_to_do) + for job, config in ci_cache.jobs_to_wait.items(): + assert not config.pending_batches + assert config.batches + config.pending_batches = list(config.batches) + for job, config in ci_cache.jobs_to_wait.items(): + for batch in range(config.num_batches): + record = CiCache.Record( + record_type=CiCache.RecordType.PENDING, + job_name=job, + job_digest=ci_cache.job_digests[job], + batch=batch, + num_batches=config.num_batches, + release_branch=True, + ) + for record_t_, records_ in ci_cache.records.items(): + if record_t_.value == CiCache.RecordType.PENDING.value: + records_[record.to_str_key()] = record + + def _test_await_for_batch( + ci_cache: CiCache, record_type: CiCache.RecordType, batch: int + ) -> None: + assert ci_cache.jobs_to_wait + for job_, config_ in ci_cache.jobs_to_wait.items(): + record = CiCache.Record( + record_type=record_type, + job_name=job_, + job_digest=ci_cache.job_digests[job_], + batch=batch, + num_batches=config_.num_batches, + release_branch=True, + ) + for record_t_, records_ in ci_cache.records.items(): + if record_t_.value == record_type.value: + records_[record.to_str_key()] = record + # await + ci_cache.await_pending_jobs(is_release=pr_info.is_release, dry_run=True) + for _, config_ in ci_cache.jobs_to_wait.items(): + assert config_.pending_batches + if ( + record_type != CiCache.RecordType.PENDING + and batch < config_.num_batches + ): + assert batch not in config_.pending_batches + else: + assert batch in config_.pending_batches + + _test_await_for_batch(ci_cache, CiCache.RecordType.SUCCESSFUL, 0) + # check all one-batch jobs are in jobs_to_skip + for job in all_jobs_in_wf: + config = CI.JOB_CONFIGS[job] + if config.num_batches == 1: + self.assertTrue(job in ci_cache.jobs_to_skip) + self.assertTrue(job not in ci_cache.jobs_to_do) + else: + self.assertTrue(job not in ci_cache.jobs_to_skip) + self.assertTrue(job in ci_cache.jobs_to_do) + + _test_await_for_batch(ci_cache, CiCache.RecordType.FAILED, 1) + _test_await_for_batch(ci_cache, CiCache.RecordType.SUCCESSFUL, 2) + + self.assertTrue(len(ci_cache.jobs_to_skip) > 0) + self.assertTrue(len(ci_cache.jobs_to_do) > 0) + self.assertCountEqual( + list(ci_cache.jobs_to_do) + ci_cache.jobs_to_skip, all_jobs_in_wf + ) diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index 608889328030..42a1460691ac 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -4,7 +4,7 @@ import unittest from ci_settings import CiSettings -from ci_config import JobConfig +from ci_config import CI _TEST_BODY_1 = """ #### Run only: @@ -166,15 +166,14 @@ def test_options_applied(self): ["tsan", "foobar", "aarch64", "analyzer", "s3_storage", "coverage"], ) - jobs_configs = {job: JobConfig() for job in _TEST_JOB_LIST} - jobs_configs[ - "fuzzers" - ].run_by_label = ( + jobs_configs = { + job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) + for job in _TEST_JOB_LIST + } + jobs_configs["fuzzers"].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs[ - "Integration tests (asan)" - ].release_only = ( + jobs_configs["Integration tests (asan)"].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( @@ -210,7 +209,10 @@ def test_options_applied(self): ) def test_options_applied_2(self): - jobs_configs = {job: JobConfig() for job in _TEST_JOB_LIST_2} + jobs_configs = { + job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) + for job in _TEST_JOB_LIST_2 + } jobs_configs["Style check"].release_only = True jobs_configs["Fast test"].pr_only = True jobs_configs["fuzzers"].run_by_label = "TEST_LABEL" @@ -252,7 +254,10 @@ def test_options_applied_2(self): def test_options_applied_3(self): ci_settings = CiSettings() ci_settings.include_keywords = ["Style"] - jobs_configs = {job: JobConfig() for job in _TEST_JOB_LIST_2} + jobs_configs = { + job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) + for job in _TEST_JOB_LIST_2 + } jobs_configs["Style check"].release_only = True jobs_configs["Fast test"].pr_only = True # no settings are set @@ -296,10 +301,13 @@ def test_options_applied_4(self): ) self.assertCountEqual(ci_options.include_keywords, ["analyzer"]) self.assertIsNone(ci_options.exclude_keywords) - jobs_configs = {job: JobConfig() for job in _TEST_JOB_LIST} - jobs_configs[ - "fuzzers" - ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result + jobs_configs = { + job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) + for job in _TEST_JOB_LIST + } + jobs_configs["fuzzers"].run_by_label = ( + "TEST_LABEL" # check "fuzzers" does not appears in the result + ) jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( diff --git a/utils/check-style/check_py.sh b/utils/check-style/check_py.sh index 2e645d2f19a4..98fdbf8bd7e0 100755 --- a/utils/check-style/check_py.sh +++ b/utils/check-style/check_py.sh @@ -4,11 +4,11 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_ou start_total=`date +%s` -start=`date +%s` -echo "Check " | ts -./check-black -n |& tee /test_output/black_output.txt -runtime=$((`date +%s`-start)) -echo "Check python formatting with black. Done. $runtime seconds." +#start=`date +%s` +#echo "Check " | ts +#./check-black -n |& tee /test_output/black_output.txt +#runtime=$((`date +%s`-start)) +#echo "Check python formatting with black. Done. $runtime seconds." start=`date +%s` ./check-pylint -n |& tee /test_output/pylint_output.txt diff --git a/utils/check-style/process_style_check_result.py b/utils/check-style/process_style_check_result.py index 2c349114a599..460b5f841ab9 100755 --- a/utils/check-style/process_style_check_result.py +++ b/utils/check-style/process_style_check_result.py @@ -17,7 +17,7 @@ def process_result(result_folder): "shellcheck", "style", "pylint", - "black", + # "black", "flake8", "mypy", "typos", From 9d416da6010db16d0cde24d0c3fba1fe4dab97a1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 Jun 2024 10:23:26 +0000 Subject: [PATCH 157/254] Another fix. --- src/Interpreters/ActionsDAG.cpp | 3 --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 4 ---- 2 files changed, 7 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 92d8e3ace9ea..c01142ff01a2 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1552,9 +1552,6 @@ ActionsDAGPtr ActionsDAG::makeConvertingActions( actions_dag->outputs.swap(projection); actions_dag->removeUnusedActions(false); - if (mode == MatchColumnsMode::Name) - appendInputsFromNamesMap(*actions_dag, source, inputs); - return actions_dag; } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 8ee3654067c9..5f94367b46de 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -2010,7 +2010,6 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons Block cur_header = pipe.getHeader(); - bool project_inputs = result_projection != nullptr; auto append_actions = [&result_projection](ActionsDAGPtr actions) { if (!result_projection) @@ -2036,9 +2035,6 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons if (result_projection) { - if (project_inputs) - result_projection->appendInputsForUnusedColumns(pipe.getHeader()); - auto projection_actions = std::make_shared(result_projection); pipe.addSimpleTransform([&](const Block & header) { From fc6c714c11ee0ba67b3ffa3bbcc4a2cb79772322 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 Jun 2024 10:54:30 +0000 Subject: [PATCH 158/254] Fix some crashes. --- src/Interpreters/ExpressionAnalyzer.cpp | 9 ++++++--- src/Interpreters/InterpreterSelectQuery.cpp | 10 +++++++--- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index b4fa852916ee..2e7bc69402f6 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -927,12 +927,15 @@ JoinPtr SelectQueryExpressionAnalyzer::appendJoin( { const ColumnsWithTypeAndName & left_sample_columns = chain.getLastStep().getResultColumns(); - auto tmp_actions = std::make_shared(std::move(converting_join_columns->actions)); - JoinPtr join = makeJoin(*syntax->ast_join, left_sample_columns, tmp_actions); - converting_join_columns->actions = std::move(*tmp_actions); + ActionsDAGPtr converting_actions; + if (converting_join_columns) + converting_actions = std::make_shared(std::move(converting_join_columns->actions)); + + JoinPtr join = makeJoin(*syntax->ast_join, left_sample_columns, converting_actions); if (converting_join_columns) { + converting_join_columns->actions = std::move(*converting_actions); chain.steps.push_back(std::make_unique(converting_join_columns)); chain.addStep(); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 68967f331f04..a1ad10ff623e 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2770,9 +2770,13 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions void InterpreterSelectQuery::executeTotalsAndHaving( QueryPlan & query_plan, bool has_having, const ActionsAndFlagsPtr & expression, bool remove_filter, bool overflow_row, bool final) { - auto dag = expression->actions.clone(); - if (expression->project_input) - dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + ActionsDAGPtr dag; + if (expression) + { + dag = expression->actions.clone(); + if (expression->project_input) + dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); + } const Settings & settings = context->getSettingsRef(); From b43240068600c806679a061c2359c728e62dd22f Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Jun 2024 13:10:04 +0000 Subject: [PATCH 159/254] better type usage --- src/Common/Scheduler/ISchedulerNode.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index 37a85666f3a7..3f699c881fb4 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -31,6 +31,7 @@ namespace ErrorCodes class ISchedulerNode; class EventQueue; +using EventId = UInt64; inline const Poco::Util::AbstractConfiguration & emptyConfig() { @@ -180,7 +181,7 @@ class ISchedulerNode : public boost::intrusive::list_base_hook<>, private boost: String basename; SchedulerNodeInfo info; ISchedulerNode * parent = nullptr; - UInt64 activation_event_id = 0; // Valid for `ISchedulerNode` placed in EventQueue::activations + EventId activation_event_id = 0; // Valid for `ISchedulerNode` placed in EventQueue::activations /// Introspection std::atomic dequeued_requests{0}; @@ -201,7 +202,6 @@ class EventQueue public: using Task = std::function; - using EventId = UInt64; static constexpr EventId not_postponed = 0; using TimePoint = std::chrono::system_clock::time_point; @@ -210,9 +210,9 @@ class EventQueue struct Event { const EventId event_id; - std::function task; + Task task; - Event(EventId event_id_, std::function && task_) + Event(EventId event_id_, Task && task_) : event_id(event_id_) , task(std::move(task_)) {} From d0f2014f139de87dfe22e2e660cd3693ca997137 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Fri, 14 Jun 2024 21:20:25 +0800 Subject: [PATCH 160/254] not count syste tables --- src/Databases/DatabasesCommon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index c590486d10e2..6426123bb4f9 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -304,7 +304,7 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c /// non-Atomic database the is_detached is set to true before RENAME. table->is_detached = false; - if (table->isSystemStorage() == false) + if (table->isSystemStorage() == false && table_id.database_name != DatabaseCatalog::SYSTEM_DATABASE) CurrentMetrics::add(getAttachedCounterForStorage(table), 1); } From da186811f037dd637ba6850aaac40858c8897fde Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 14 Jun 2024 15:28:21 +0200 Subject: [PATCH 161/254] add more tests --- tests/integration/test_table_db_num_limit/test.py | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/integration/test_table_db_num_limit/test.py b/tests/integration/test_table_db_num_limit/test.py index bde9a956d700..f2080ec4738a 100644 --- a/tests/integration/test_table_db_num_limit/test.py +++ b/tests/integration/test_table_db_num_limit/test.py @@ -28,6 +28,13 @@ def test_table_db_limit(started_cluster): node1.query("create database db_exp".format(i)) assert "TOO_MANY_DATABASES" in str(exp_info) + + for i in range(10): + node1.query("create table t{} (a Int32) Engine = Log".format(i)) + + node1.query("system flush logs"); + for i in range(10): + node1.query("drop table t{}".format(i)) for i in range(10): node1.query("create table t{} (a Int32) Engine = Log".format(i)) From 5f3535309be83cebab9fae967d6b4ab739aca9dc Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 14 Jun 2024 15:38:09 +0200 Subject: [PATCH 162/254] Add a test --- .../test_keeper_profiler/__init__.py | 0 .../configs/keeper_config1.xml | 44 +++++++++++++ .../configs/keeper_config2.xml | 44 +++++++++++++ .../configs/keeper_config3.xml | 44 +++++++++++++ .../integration/test_keeper_profiler/test.py | 65 +++++++++++++++++++ 5 files changed, 197 insertions(+) create mode 100644 tests/integration/test_keeper_profiler/__init__.py create mode 100644 tests/integration/test_keeper_profiler/configs/keeper_config1.xml create mode 100644 tests/integration/test_keeper_profiler/configs/keeper_config2.xml create mode 100644 tests/integration/test_keeper_profiler/configs/keeper_config3.xml create mode 100644 tests/integration/test_keeper_profiler/test.py diff --git a/tests/integration/test_keeper_profiler/__init__.py b/tests/integration/test_keeper_profiler/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/integration/test_keeper_profiler/configs/keeper_config1.xml b/tests/integration/test_keeper_profiler/configs/keeper_config1.xml new file mode 100644 index 000000000000..b541090bd989 --- /dev/null +++ b/tests/integration/test_keeper_profiler/configs/keeper_config1.xml @@ -0,0 +1,44 @@ + + + + http://minio1:9001/snapshots/ + minio + minio123 + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + * + + + 5000 + 10000 + 5000 + 50 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + 1000000000 + 1000000000 + diff --git a/tests/integration/test_keeper_profiler/configs/keeper_config2.xml b/tests/integration/test_keeper_profiler/configs/keeper_config2.xml new file mode 100644 index 000000000000..9ad0a772860a --- /dev/null +++ b/tests/integration/test_keeper_profiler/configs/keeper_config2.xml @@ -0,0 +1,44 @@ + + + + http://minio1:9001/snapshots/ + minio + minio123 + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + * + + + 5000 + 10000 + 5000 + 75 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + 1000000000 + 1000000000 + diff --git a/tests/integration/test_keeper_profiler/configs/keeper_config3.xml b/tests/integration/test_keeper_profiler/configs/keeper_config3.xml new file mode 100644 index 000000000000..ff31f7cdbf46 --- /dev/null +++ b/tests/integration/test_keeper_profiler/configs/keeper_config3.xml @@ -0,0 +1,44 @@ + + + + http://minio1:9001/snapshots/ + minio + minio123 + + 9181 + 3 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + * + + + 5000 + 10000 + 5000 + 75 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + true + + + 3 + node3 + 9234 + true + + + + 1000000000 + 1000000000 + diff --git a/tests/integration/test_keeper_profiler/test.py b/tests/integration/test_keeper_profiler/test.py new file mode 100644 index 000000000000..bb24f13d9c20 --- /dev/null +++ b/tests/integration/test_keeper_profiler/test.py @@ -0,0 +1,65 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV +from helpers.keeper_utils import KeeperClient, KeeperException + + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node1", + main_configs=["configs/keeper_config1.xml"], + stay_alive=True, +) +node2 = cluster.add_instance( + "node2", + main_configs=["configs/keeper_config2.xml"], + stay_alive=True, + with_minio=True, +) +node3 = cluster.add_instance( + "node3", + main_configs=["configs/keeper_config3.xml"], + stay_alive=True, + with_minio=True, +) + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_profiler(started_cluster): + node = cluster.instances["node1"] + + node.query( + "CREATE TABLE t (key UInt32, value String) Engine = ReplicatedMergeTree('/clickhouse-tables/test1', 'r1') ORDER BY key" + ) + + node.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + node.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + node.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + node.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + + node.query("system flush logs") + assert int(node.query("exists system.trace_log")) + + assert 1 < int( + node.query( + """ +set allow_introspection_functions=1; +system flush logs; +select cnt from ( + select count() as cnt, formatReadableSize(sum(size)), + arrayStringConcat( + arrayMap(x, y -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), arrayMap(x -> demangle(addressToSymbol(x)), trace)), + '\n') as trace +from system.trace_log where trace_type = ‘Real’ and trace ilike '%KeeperTCPHandler%' group by trace); + """ + ) + ) From c856dc8c66dd95d4557d1e95b07e60d8b4ff5e00 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 14 Jun 2024 13:45:00 +0000 Subject: [PATCH 163/254] Automatic style fix --- tests/integration/test_keeper_profiler/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_keeper_profiler/test.py b/tests/integration/test_keeper_profiler/test.py index bb24f13d9c20..796d77987479 100644 --- a/tests/integration/test_keeper_profiler/test.py +++ b/tests/integration/test_keeper_profiler/test.py @@ -24,6 +24,7 @@ with_minio=True, ) + @pytest.fixture(scope="module", autouse=True) def started_cluster(): try: From e0128556f273873d9b027624e01fd86cec4ff653 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 14 Jun 2024 16:30:27 +0200 Subject: [PATCH 164/254] Disable long test in coverage run --- ...581_share_big_sets_between_multiple_mutations_tasks_long.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql index ff8b9c71e928..741d01779712 100644 --- a/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql +++ b/tests/queries/0_stateless/02581_share_big_sets_between_multiple_mutations_tasks_long.sql @@ -1,4 +1,4 @@ --- Tags: long, no-debug, no-tsan, no-asan, no-ubsan, no-msan, no-parallel +-- Tags: long, no-debug, no-tsan, no-asan, no-ubsan, no-msan, no-parallel, no-sanitize-coverage -- no-parallel because the sets use a lot of memory, which may interfere with other tests From 5a577915aba1f927124d340d0b27da39a9db5bad Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 Jun 2024 15:43:43 +0000 Subject: [PATCH 165/254] Fixing more tests. --- src/Interpreters/ActionsDAG.cpp | 4 ++-- src/Interpreters/ExpressionAnalyzer.cpp | 9 ++++----- src/Storages/StorageMaterializedView.cpp | 1 + .../02576_predicate_push_down_sorting_fix.reference | 9 +++++---- .../0_stateless/03095_window_functions_qualify.reference | 5 +++-- 5 files changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index c01142ff01a2..6e11dc96769a 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -2754,8 +2754,8 @@ void ActionsDAG::removeUnusedConjunctions(NodeRawConstPtrs rejected_conjunctions std::unordered_set used_inputs; for (const auto * input : inputs) { - if (removes_filter && input == predicate) - continue; + // if (removes_filter && input == predicate) + // continue; used_inputs.insert(input); } diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 2e7bc69402f6..d0f037ad9e1f 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -928,13 +928,11 @@ JoinPtr SelectQueryExpressionAnalyzer::appendJoin( const ColumnsWithTypeAndName & left_sample_columns = chain.getLastStep().getResultColumns(); ActionsDAGPtr converting_actions; - if (converting_join_columns) - converting_actions = std::make_shared(std::move(converting_join_columns->actions)); - JoinPtr join = makeJoin(*syntax->ast_join, left_sample_columns, converting_actions); - if (converting_join_columns) + if (converting_actions) { + converting_join_columns = std::make_shared(); converting_join_columns->actions = std::move(*converting_actions); chain.steps.push_back(std::make_unique(converting_join_columns)); chain.addStep(); @@ -1720,9 +1718,9 @@ ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendProjectResult(Expression for (const auto & column : required_result_columns) result_columns.emplace_back(column, std::string{}); actions->actions.project(result_columns); - actions->project_input = actions->projected_output = true; } + actions->project_input = actions->projected_output = true; return actions; } @@ -1878,6 +1876,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( { auto dag = std::make_shared(std::move(prewhere_dag_and_flags->actions)); prewhere_info = std::make_shared(std::move(dag), query.prewhere()->getColumnName()); + prewhere_dag_and_flags.reset(); } finalize(chain, prewhere_step_num, where_step_num, having_step_num, query); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 9625c13197cc..316f398b4761 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -273,6 +273,7 @@ void StorageMaterializedView::read( * They may be added in case of distributed query with JOIN. * In that case underlying table returns joined columns as well. */ + converting_actions->removeUnusedActions(); auto converting_step = std::make_unique(query_plan.getCurrentDataStream(), converting_actions); converting_step->setStepDescription("Convert target table structure to MaterializedView structure"); query_plan.addStep(std::move(converting_step)); diff --git a/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.reference b/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.reference index dd107065380d..d391c365ea7b 100644 --- a/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.reference +++ b/tests/queries/0_stateless/02576_predicate_push_down_sorting_fix.reference @@ -1,10 +1,11 @@ Expression ((Project names + (Projection + ))) Header: number UInt64 Actions: INPUT : 0 -> __table2.number UInt64 : 0 - ALIAS __table2.number :: 0 -> number UInt64 : 1 - ALIAS number :: 1 -> __table1.number UInt64 : 0 - ALIAS __table1.number :: 0 -> number UInt64 : 1 -Positions: 1 + INPUT :: 1 -> ignore(2_UInt8) UInt8 : 1 + ALIAS __table2.number :: 0 -> number UInt64 : 2 + ALIAS number :: 2 -> __table1.number UInt64 : 0 + ALIAS __table1.number :: 0 -> number UInt64 : 2 +Positions: 2 Sorting (Sorting for ORDER BY) Header: ignore(2_UInt8) UInt8 __table2.number UInt64 diff --git a/tests/queries/0_stateless/03095_window_functions_qualify.reference b/tests/queries/0_stateless/03095_window_functions_qualify.reference index c74a212428ba..9ffbe64f85ec 100644 --- a/tests/queries/0_stateless/03095_window_functions_qualify.reference +++ b/tests/queries/0_stateless/03095_window_functions_qualify.reference @@ -48,8 +48,9 @@ Positions: 2 0 INPUT :: 1 -> count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) UInt64 : 1 INPUT : 2 -> count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) UInt64 : 2 COLUMN Const(UInt8) -> 4_UInt8 UInt8 : 3 - FUNCTION equals(count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) :: 2, 4_UInt8 :: 3) -> equals(count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)), 4_UInt8) UInt8 : 4 - Positions: 4 0 1 + INPUT :: 3 -> modulo(__table1.number, 3_UInt8) UInt8 : 4 + FUNCTION equals(count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)) :: 2, 4_UInt8 :: 3) -> equals(count() OVER (PARTITION BY modulo(__table1.number, 3_UInt8)), 4_UInt8) UInt8 : 5 + Positions: 5 0 1 Window (Window step for window \'PARTITION BY modulo(__table1.number, 3_UInt8)\') Header: modulo(__table1.number, 3_UInt8) UInt8 __table1.number UInt64 From 592d704bf96ed941bb0c8a6713967cf0c173c386 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 14 Jun 2024 18:42:37 +0200 Subject: [PATCH 166/254] Build Report check remove pending status --- tests/ci/build_report_check.py | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 5f24fd8f2242..434cf83cfb12 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -24,7 +24,7 @@ BuildResult, JobReport, create_build_html_report, - get_worst_status, + get_worst_status, FAILURE, ) from stopwatch import Stopwatch from ci_config import CI @@ -134,17 +134,16 @@ def main(): # Check if there are no builds at all, do not override bad status if summary_status == SUCCESS: if missing_builds: - summary_status = PENDING + summary_status = FAILURE elif ok_groups == 0: summary_status = ERROR - addition = "" + description = "" + if missing_builds: - addition = ( - f" ({required_builds - missing_builds} of {required_builds} builds are OK)" - ) + description = f"{missing_builds} of {required_builds} builds are missing." - description = f"{ok_groups}/{total_groups} artifact groups are OK{addition}" + description += f" {ok_groups}/{total_groups} artifact groups are OK" JobReport( description=description, From eb2af8202224587ff2c7d078639d947009e5f676 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 14 Jun 2024 18:43:59 +0200 Subject: [PATCH 167/254] reenable black --- utils/check-style/check_py.sh | 10 +++++----- utils/check-style/process_style_check_result.py | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/utils/check-style/check_py.sh b/utils/check-style/check_py.sh index 98fdbf8bd7e0..2e645d2f19a4 100755 --- a/utils/check-style/check_py.sh +++ b/utils/check-style/check_py.sh @@ -4,11 +4,11 @@ cd /ClickHouse/utils/check-style || echo -e "failure\tRepo not found" > /test_ou start_total=`date +%s` -#start=`date +%s` -#echo "Check " | ts -#./check-black -n |& tee /test_output/black_output.txt -#runtime=$((`date +%s`-start)) -#echo "Check python formatting with black. Done. $runtime seconds." +start=`date +%s` +echo "Check " | ts +./check-black -n |& tee /test_output/black_output.txt +runtime=$((`date +%s`-start)) +echo "Check python formatting with black. Done. $runtime seconds." start=`date +%s` ./check-pylint -n |& tee /test_output/pylint_output.txt diff --git a/utils/check-style/process_style_check_result.py b/utils/check-style/process_style_check_result.py index 460b5f841ab9..2c349114a599 100755 --- a/utils/check-style/process_style_check_result.py +++ b/utils/check-style/process_style_check_result.py @@ -17,7 +17,7 @@ def process_result(result_folder): "shellcheck", "style", "pylint", - # "black", + "black", "flake8", "mypy", "typos", From 7b43ede04e38a05e2e6ee46ea5ca6cb1a6d12d98 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 14 Jun 2024 18:45:08 +0200 Subject: [PATCH 168/254] style fix --- tests/ci/build_report_check.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 434cf83cfb12..664f6a7cbb99 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -24,7 +24,8 @@ BuildResult, JobReport, create_build_html_report, - get_worst_status, FAILURE, + get_worst_status, + FAILURE, ) from stopwatch import Stopwatch from ci_config import CI From ba349ac73b778e949472d49996503e440d4f2839 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 14 Jun 2024 18:50:53 +0200 Subject: [PATCH 169/254] ci test improvement --- tests/ci/test_ci_config.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 76ab3b82fa5c..3454ea1851a8 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -285,6 +285,11 @@ def _test_await_for_batch( else: assert batch in config_.pending_batches + for _, config_ in ci_cache.jobs_to_do.items(): + # jobs to do must have batches to run before/after await + # if it's an empty list after await - apparently job has not been removed after await + assert config_.batches + _test_await_for_batch(ci_cache, CiCache.RecordType.SUCCESSFUL, 0) # check all one-batch jobs are in jobs_to_skip for job in all_jobs_in_wf: From 02d76f60823b71e6b1de5f7562e749ed45b25312 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 Jun 2024 17:05:03 +0000 Subject: [PATCH 170/254] Trying to fix fuzzed test. --- src/Storages/StorageMerge.cpp | 13 ++++++++++++- .../02156_storage_merge_prewhere_2.reference | 6 ++++++ .../0_stateless/02156_storage_merge_prewhere_2.sql | 4 ++++ 3 files changed, 22 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index cae819fa9823..419e944f456e 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1172,6 +1172,16 @@ ReadFromMerge::ChildPlan ReadFromMerge::createPlanForTable( if (real_column_names.empty()) real_column_names.push_back(ExpressionActions::getSmallestColumn(storage_snapshot_->metadata->getColumns().getAllPhysical()).name); + if (allow_experimental_analyzer) + { + auto ast = modified_query_info.query_tree->toAST(); + InterpreterSelectQueryAnalyzer interpreter(ast, + modified_context, + SelectQueryOptions(processed_stage)); + + modified_query_info.query_tree = interpreter.getQueryTree(); + } + storage->read(plan, real_column_names, storage_snapshot_, @@ -1200,7 +1210,8 @@ ReadFromMerge::ChildPlan ReadFromMerge::createPlanForTable( { /// Converting query to AST because types might be different in the source table. /// Need to resolve types again. - InterpreterSelectQueryAnalyzer interpreter(modified_query_info.query_tree->toAST(), + auto ast = modified_query_info.query_tree->toAST(); + InterpreterSelectQueryAnalyzer interpreter(ast, modified_context, SelectQueryOptions(processed_stage)); diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere_2.reference b/tests/queries/0_stateless/02156_storage_merge_prewhere_2.reference index e2fa0e43e0c2..8e7596488710 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere_2.reference +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere_2.reference @@ -1,6 +1,12 @@ 1 a +1 a +2 b +2 b +1 a +1 a 2 b 2 b 1 a 2 b +1 a 2 b diff --git a/tests/queries/0_stateless/02156_storage_merge_prewhere_2.sql b/tests/queries/0_stateless/02156_storage_merge_prewhere_2.sql index 5c0751b7a358..1b4881d4e7be 100644 --- a/tests/queries/0_stateless/02156_storage_merge_prewhere_2.sql +++ b/tests/queries/0_stateless/02156_storage_merge_prewhere_2.sql @@ -5,9 +5,13 @@ DROP TABLE IF EXISTS t_02156_ololo_dist; CREATE TABLE t_02156_ololo_1 (k UInt32, v Nullable(String)) ENGINE = MergeTree order by k; CREATE TABLE t_02156_ololo_2 (k UInt32, v String) ENGINE = MergeTree order by k; CREATE TABLE t_02156_ololo_dist (k UInt32, v String) ENGINE = Distributed(test_shard_localhost, currentDatabase(), t_02156_ololo_2); +CREATE TABLE t_02156_ololo_dist2 (k UInt32, v Nullable(String)) ENGINE = Distributed(test_shard_localhost, currentDatabase(), t_02156_ololo_1); insert into t_02156_ololo_1 values (1, 'a'); insert into t_02156_ololo_2 values (2, 'b'); select * from merge('t_02156_ololo') where k != 0 and notEmpty(v) order by k settings optimize_move_to_prewhere=0; select * from merge('t_02156_ololo') where k != 0 and notEmpty(v) order by k settings optimize_move_to_prewhere=1; + +select * from merge('t_02156_ololo_dist') where k != 0 and notEmpty(v) order by k settings optimize_move_to_prewhere=0; +select * from merge('t_02156_ololo_dist') where k != 0 and notEmpty(v) order by k settings optimize_move_to_prewhere=1; From 9bf849ca73537d1493c96618abb7b92013d64ba4 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 14 Jun 2024 19:27:57 +0000 Subject: [PATCH 171/254] Automatic style fix --- tests/ci/ssh.py | 6 +++--- tests/ci/test_ci_options.py | 14 +++++++++----- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/tests/ci/ssh.py b/tests/ci/ssh.py index 89d90d724d2d..321826fcf447 100644 --- a/tests/ci/ssh.py +++ b/tests/ci/ssh.py @@ -37,9 +37,9 @@ def start(self): ssh_options = ( "," + os.environ["SSH_OPTIONS"] if os.environ.get("SSH_OPTIONS") else "" ) - os.environ["SSH_OPTIONS"] = ( - f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no" - ) + os.environ[ + "SSH_OPTIONS" + ] = f"{ssh_options}UserKnownHostsFile=/dev/null,StrictHostKeyChecking=no" def add(self, key): key_pub = self._key_pub(key) diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index 42a1460691ac..73499540b782 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -170,10 +170,14 @@ def test_options_applied(self): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( + jobs_configs[ + "fuzzers" + ].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs["Integration tests (asan)"].release_only = ( + jobs_configs[ + "Integration tests (asan)" + ].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( @@ -305,9 +309,9 @@ def test_options_applied_4(self): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( - "TEST_LABEL" # check "fuzzers" does not appears in the result - ) + jobs_configs[ + "fuzzers" + ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( From ffb9a67a11720a71b833fd2c6f1ef6333d70b018 Mon Sep 17 00:00:00 2001 From: chloro <13125187405@163.com> Date: Sat, 15 Jun 2024 16:28:56 +0800 Subject: [PATCH 172/254] fix the bug that In operator did not perform the correct type conversion --- src/Analyzer/SetUtils.cpp | 44 ++++++++++++++----- src/Interpreters/ActionsVisitor.cpp | 34 ++++++++++---- src/Interpreters/convertFieldToType.cpp | 4 +- src/Interpreters/convertFieldToType.h | 2 +- .../0_stateless/00137_in_constants.reference | 1 + .../0_stateless/00137_in_constants.sql | 1 + 6 files changed, 64 insertions(+), 22 deletions(-) diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index ceda264b5a62..9a267bfa1493 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -54,8 +54,9 @@ size_t getCompoundTypeDepth(const IDataType & type) } template -Block createBlockFromCollection(const Collection & collection, const DataTypes & block_types, bool transform_null_in) +Block createBlockFromCollection(const Collection & collection, const DataTypes& value_types, const DataTypes & block_types, bool transform_null_in) { + assert(collection.size() == value_types.size()); size_t columns_size = block_types.size(); MutableColumns columns(columns_size); for (size_t i = 0; i < columns_size; ++i) @@ -65,19 +66,24 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & } Row tuple_values; + size_t value_types_index = 0; for (const auto & value : collection) { if (columns_size == 1) { - auto field = convertFieldToTypeStrict(value, *block_types[0]); - if (!field) + DataTypePtr data_type = value_types[value_types_index]; + auto field = convertFieldToTypeStrict(value, *data_type, *block_types[0]); + if (!field) { + value_types_index += 1; continue; + } bool need_insert_null = transform_null_in && block_types[0]->isNullable(); if (!field->isNull() || need_insert_null) columns[0]->insert(*field); - + + value_types_index += 1; continue; } @@ -87,6 +93,9 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & value.getTypeName()); const auto & tuple = value.template get(); + DataTypePtr value_type = value_types[value_types_index]; + DataTypes tuple_value_type = typeid_cast(value_type.get())->getElements(); + size_t tuple_size = tuple.size(); if (tuple_size != columns_size) @@ -101,7 +110,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & size_t i = 0; for (; i < tuple_size; ++i) { - auto converted_field = convertFieldToTypeStrict(tuple[i], *block_types[i]); + auto converted_field = convertFieldToTypeStrict(tuple[i], *tuple_value_type[i], *block_types[i]); if (!converted_field) break; tuple_values[i] = std::move(*converted_field); @@ -114,6 +123,8 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & if (i == tuple_size) for (i = 0; i < tuple_size; ++i) columns[i]->insert(tuple_values[i]); + + value_types_index += 1; } Block res; @@ -149,7 +160,8 @@ Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const /// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc. Array array{value}; - result_block = createBlockFromCollection(array, set_element_types, transform_null_in); + DataTypes value_types{value_type}; + result_block = createBlockFromCollection(array, value_types, set_element_types, transform_null_in); } else if (lhs_type_depth + 1 == rhs_type_depth) { @@ -157,10 +169,22 @@ Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const WhichDataType rhs_which_type(value_type); - if (rhs_which_type.isArray()) - result_block = createBlockFromCollection(value.get(), set_element_types, transform_null_in); - else if (rhs_which_type.isTuple()) - result_block = createBlockFromCollection(value.get(), set_element_types, transform_null_in); + if (rhs_which_type.isArray()) { + const DataTypeArray* value_array_type = typeid_cast(value_type.get()); + size_t value_array_size = value.get().size(); + DataTypes value_types; + value_types.reserve(value_array_size); + + for(size_t i = 0; i < value_array_size; ++i) { + value_types.push_back(value_array_type->getNestedType()); + } + result_block = createBlockFromCollection(value.get(), value_types, set_element_types, transform_null_in); + } + else if (rhs_which_type.isTuple()) { + const DataTypeTuple* value_tuple_type = typeid_cast(value_type.get()); + DataTypes value_types = value_tuple_type->getElements(); + result_block = createBlockFromCollection(value.get(), value_types, set_element_types, transform_null_in); + } else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Unsupported type at the right-side of IN. Expected Array or Tuple. Actual {}", diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 0bdd4c089f1d..081e7e4fc2c7 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -102,7 +102,7 @@ static size_t getTypeDepth(const DataTypePtr & type) /// 33.33 in the set is converted to 33.3, but it is not equal to 33.3 in the column, so the result should still be empty. /// We can not include values that don't represent any possible value from the type of filtered column to the set. template -static Block createBlockFromCollection(const Collection & collection, const DataTypes & types, bool transform_null_in) +static Block createBlockFromCollection(const Collection & collection, const DataTypes& value_types, const DataTypes & types, bool transform_null_in) { size_t columns_num = types.size(); MutableColumns columns(columns_num); @@ -113,14 +113,16 @@ static Block createBlockFromCollection(const Collection & collection, const Data } Row tuple_values; + size_t value_type_index = 0; for (const auto & value : collection) { if (columns_num == 1) { - auto field = convertFieldToTypeStrict(value, *types[0]); + auto field = convertFieldToTypeStrict(value, *value_types[value_type_index], *types[0]); bool need_insert_null = transform_null_in && types[0]->isNullable(); if (field && (!field->isNull() || need_insert_null)) columns[0]->insert(*field); + value_type_index += 1; } else { @@ -130,18 +132,20 @@ static Block createBlockFromCollection(const Collection & collection, const Data const auto & tuple = value.template get(); size_t tuple_size = tuple.size(); - if (tuple_size != columns_num) throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET, "Incorrect size of tuple in set: {} instead of {}", tuple_size, columns_num); if (tuple_values.empty()) tuple_values.resize(tuple_size); + + DataTypePtr value_type = value_types[value_type_index]; + DataTypes tuple_value_type = typeid_cast(value_type.get())->getElements(); size_t i = 0; for (; i < tuple_size; ++i) { - auto converted_field = convertFieldToTypeStrict(tuple[i], *types[i]); + auto converted_field = convertFieldToTypeStrict(tuple[i], *tuple_value_type[i], *types[i]); if (!converted_field) break; tuple_values[i] = std::move(*converted_field); @@ -154,6 +158,7 @@ static Block createBlockFromCollection(const Collection & collection, const Data if (i == tuple_size) for (i = 0; i < tuple_size; ++i) columns[i]->insert(tuple_values[i]); + value_type_index += 1; } } @@ -317,16 +322,27 @@ Block createBlockForSet( if (left_type_depth == right_type_depth) { Array array{right_arg_value}; - block = createBlockFromCollection(array, set_element_types, tranform_null_in); + DataTypes data_types{right_arg_type}; + block = createBlockFromCollection(array, data_types, set_element_types, tranform_null_in); } /// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc. else if (left_type_depth + 1 == right_type_depth) { auto type_index = right_arg_type->getTypeId(); - if (type_index == TypeIndex::Tuple) - block = createBlockFromCollection(right_arg_value.get(), set_element_types, tranform_null_in); - else if (type_index == TypeIndex::Array) - block = createBlockFromCollection(right_arg_value.get(), set_element_types, tranform_null_in); + if (type_index == TypeIndex::Tuple) { + DataTypes data_types = typeid_cast(right_arg_type.get())->getElements(); + block = createBlockFromCollection(right_arg_value.get(), data_types, set_element_types, tranform_null_in); + } + else if (type_index == TypeIndex::Array) { + const auto* right_arg_array_type = typeid_cast(right_arg_type.get()); + size_t right_arg_array_size = right_arg_value.get().size(); + DataTypes data_types; + data_types.reserve(right_arg_array_size); + for(size_t i = 0; i < right_arg_array_size; ++i) { + data_types.push_back(right_arg_array_type->getNestedType()); + } + block = createBlockFromCollection(right_arg_value.get(), data_types, set_element_types, tranform_null_in); + } else throw_unsupported_type(right_arg_type); } diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 9363e3d83ebd..57d4e18010bd 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -615,9 +615,9 @@ static bool decimalEqualsFloat(Field field, Float64 float_value) return decimal_to_float == float_value; } -std::optional convertFieldToTypeStrict(const Field & from_value, const IDataType & to_type) +std::optional convertFieldToTypeStrict(const Field & from_value, const IDataType& from_type, const IDataType & to_type) { - Field result_value = convertFieldToType(from_value, to_type); + Field result_value = convertFieldToType(from_value, to_type, &from_type); if (Field::isDecimal(from_value.getType()) && Field::isDecimal(result_value.getType())) { diff --git a/src/Interpreters/convertFieldToType.h b/src/Interpreters/convertFieldToType.h index 7f49ea5479de..044b0c9b1ceb 100644 --- a/src/Interpreters/convertFieldToType.h +++ b/src/Interpreters/convertFieldToType.h @@ -22,6 +22,6 @@ Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_t /// Applies stricter rules than convertFieldToType, doesn't allow loss of precision converting to Decimal. /// Returns `Field` if the conversion was successful and the result is equal to the original value, otherwise returns nullopt. -std::optional convertFieldToTypeStrict(const Field & from_value, const IDataType & to_type); +std::optional convertFieldToTypeStrict(const Field & from_value, const IDataType& from_type, const IDataType & to_type); } diff --git a/tests/queries/0_stateless/00137_in_constants.reference b/tests/queries/0_stateless/00137_in_constants.reference index 379885fb1ab0..94607ffa9249 100644 --- a/tests/queries/0_stateless/00137_in_constants.reference +++ b/tests/queries/0_stateless/00137_in_constants.reference @@ -13,6 +13,7 @@ 1 1 1 +1 0 0 0 diff --git a/tests/queries/0_stateless/00137_in_constants.sql b/tests/queries/0_stateless/00137_in_constants.sql index 297acc4ef26b..bc365523be10 100644 --- a/tests/queries/0_stateless/00137_in_constants.sql +++ b/tests/queries/0_stateless/00137_in_constants.sql @@ -13,6 +13,7 @@ SELECT 'Hello' IN (SELECT 'Hello'); SELECT materialize('Hello') IN (SELECT 'Hello'); SELECT 'Hello' IN (SELECT materialize('Hello')); SELECT materialize('Hello') IN (SELECT materialize('Hello')); +SELECT toDate('2020-01-01') IN (toDateTime('2020-01-01', 'UTC')); SELECT 2 IN (SELECT 1); SELECT materialize(2) IN (SELECT 1); From 0d7a4915aef9ce023bc586c3d7486a68f354bf04 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 15 Jun 2024 17:38:57 +0200 Subject: [PATCH 173/254] align job names, add more tests --- .github/workflows/backport_branches.yml | 6 +- .github/workflows/release_branches.yml | 6 +- tests/ci/ci_definitions.py | 10 +-- tests/ci/test_ci_config.py | 106 ++++++++++++++++++++++++ tests/ci/test_ci_options.py | 26 +++--- 5 files changed, 128 insertions(+), 26 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index c8c6ba30b0b7..e1980ec9ef2e 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -70,7 +70,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Compatibility check (amd64) + test_name: Compatibility check (release) runner_type: style-checker data: ${{ needs.RunConfig.outputs.data }} CompatibilityCheckAarch64: @@ -194,7 +194,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Install packages (amd64) + test_name: Install packages (release) runner_type: style-checker data: ${{ needs.RunConfig.outputs.data }} run_command: | @@ -204,7 +204,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Install packages (arm64) + test_name: Install packages (aarch64) runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} run_command: | diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index f9b8a4fa7649..45eb7431bb46 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -65,7 +65,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Compatibility check (amd64) + test_name: Compatibility check (release) runner_type: style-checker data: ${{ needs.RunConfig.outputs.data }} CompatibilityCheckAarch64: @@ -244,7 +244,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Install packages (amd64) + test_name: Install packages (release) runner_type: style-checker data: ${{ needs.RunConfig.outputs.data }} run_command: | @@ -254,7 +254,7 @@ jobs: if: ${{ !failure() && !cancelled() }} uses: ./.github/workflows/reusable_test.yml with: - test_name: Install packages (arm64) + test_name: Install packages (aarch64) runner_type: style-checker-aarch64 data: ${{ needs.RunConfig.outputs.data }} run_command: | diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index e427d48505e5..3d732a997f54 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -98,7 +98,7 @@ class JobNames(metaclass=WithIter): FAST_TEST = "Fast test" DOCKER_SERVER = "Docker server image" DOCKER_KEEPER = "Docker keeper image" - INSTALL_TEST_AMD = "Install packages (amd64)" + INSTALL_TEST_AMD = "Install packages (release)" INSTALL_TEST_ARM = "Install packages (aarch64)" STATELESS_TEST_DEBUG = "Stateless tests (debug)" @@ -167,8 +167,8 @@ class JobNames(metaclass=WithIter): JEPSEN_KEEPER = "ClickHouse Keeper Jepsen" JEPSEN_SERVER = "ClickHouse Server Jepsen" - PERFORMANCE_TEST_AMD64 = "Performance Comparison" - PERFORMANCE_TEST_ARM64 = "Performance Comparison Aarch64" + PERFORMANCE_TEST_AMD64 = "Performance Comparison (release)" + PERFORMANCE_TEST_ARM64 = "Performance Comparison (aarch64)" SQL_LOGIC_TEST = "Sqllogic test (release)" @@ -176,10 +176,10 @@ class JobNames(metaclass=WithIter): SQLANCER_DEBUG = "SQLancer (debug)" SQLTEST = "SQLTest" - COMPATIBILITY_TEST = "Compatibility check (amd64)" + COMPATIBILITY_TEST = "Compatibility check (release)" COMPATIBILITY_TEST_ARM = "Compatibility check (aarch64)" - CLICKBENCH_TEST = "ClickBench (amd64)" + CLICKBENCH_TEST = "ClickBench (release)" CLICKBENCH_TEST_ARM = "ClickBench (aarch64)" LIBFUZZER_TEST = "libFuzzer tests" diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 3454ea1851a8..28d417d23ec6 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -21,6 +21,112 @@ def test_runner_config(self): for job in CI.JobNames: self.assertIn(CI.JOB_CONFIGS[job].runner_type, CI.Runners) + def test_builds_configs(self): + """build name in the build config must match the job name""" + for job in CI.JobNames: + self.assertTrue(CI.JOB_CONFIGS[job].runner_type in CI.Runners) + if job in CI.BuildNames: + self.assertTrue(CI.JOB_CONFIGS[job].build_config.name == job) + self.assertTrue(CI.JOB_CONFIGS[job].required_builds is None) + else: + self.assertTrue(CI.JOB_CONFIGS[job].build_config is None) + if "asan" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_ASAN, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "msan" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_MSAN, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "tsan" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_TSAN, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "ubsan" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_UBSAN, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "debug" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_DEBUG, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "release" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + in ( + CI.BuildNames.PACKAGE_RELEASE, + CI.BuildNames.BINARY_RELEASE, + ), + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "coverage" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_RELEASE_COVERAGE, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "aarch" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_AARCH64, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "amd64" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_RELEASE, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "uzzer" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] == CI.BuildNames.FUZZERS, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "Docker" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + in ( + CI.BuildNames.PACKAGE_RELEASE, + CI.BuildNames.PACKAGE_AARCH64, + ), + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "SQLTest" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + == CI.BuildNames.PACKAGE_RELEASE, + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif "Jepsen" in job: + self.assertTrue( + CI.JOB_CONFIGS[job].required_builds[0] + in ( + CI.BuildNames.PACKAGE_RELEASE, + CI.BuildNames.BINARY_RELEASE, + ), + f"Job [{job}] probably has wrong required build [{CI.JOB_CONFIGS[job].required_builds[0]}] in JobConfig", + ) + elif job in ( + CI.JobNames.STYLE_CHECK, + CI.JobNames.FAST_TEST, + CI.JobNames.BUILD_CHECK, + CI.JobNames.DOCS_CHECK, + CI.JobNames.BUGFIX_VALIDATE, + ): + self.assertTrue(CI.JOB_CONFIGS[job].required_builds is None) + else: + print(f"Job [{job}] required build not checked") + def test_job_stage_config(self): """ check runner is provided w/o exception diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index 73499540b782..fac6662b1aab 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -64,8 +64,8 @@ "fuzzers", "Docker server image", "Docker keeper image", - "Install packages (amd64)", - "Install packages (arm64)", + "Install packages (release)", + "Install packages (aarch64)", "Stateless tests (debug)", "Stateless tests (release)", "Stateless tests (coverage)", @@ -120,15 +120,15 @@ "AST fuzzer (ubsan)", "ClickHouse Keeper Jepsen", "ClickHouse Server Jepsen", - "Performance Comparison", - "Performance Comparison Aarch64", + "Performance Comparison (release)", + "Performance Comparison (aarch64)", "Sqllogic test (release)", "SQLancer (release)", "SQLancer (debug)", "SQLTest", - "Compatibility check (amd64)", + "Compatibility check (release)", "Compatibility check (aarch64)", - "ClickBench (amd64)", + "ClickBench (release)", "ClickBench (aarch64)", "libFuzzer tests", "ClickHouse build check", @@ -170,14 +170,10 @@ def test_options_applied(self): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs[ - "fuzzers" - ].run_by_label = ( + jobs_configs["fuzzers"].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs[ - "Integration tests (asan)" - ].release_only = ( + jobs_configs["Integration tests (asan)"].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( @@ -309,9 +305,9 @@ def test_options_applied_4(self): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs[ - "fuzzers" - ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result + jobs_configs["fuzzers"].run_by_label = ( + "TEST_LABEL" # check "fuzzers" does not appears in the result + ) jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( From 2af2fc148694fbd9cd837825192fa78781875d85 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 15 Jun 2024 17:59:42 +0200 Subject: [PATCH 174/254] more tests --- tests/ci/ci_config.py | 1 + tests/ci/test_ci_config.py | 7 +++++++ 2 files changed, 8 insertions(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 3dc0040663b0..7eb4b7fdec27 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -24,6 +24,7 @@ class CI: from ci_definitions import BuildNames as BuildNames from ci_definitions import StatusNames as StatusNames from ci_definitions import CHECK_DESCRIPTIONS as CHECK_DESCRIPTIONS + from ci_definitions import REQUIRED_CHECKS as REQUIRED_CHECKS from ci_definitions import MQ_JOBS as MQ_JOBS from ci_definitions import WorkflowStages as WorkflowStages from ci_definitions import Runners as Runners diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 28d417d23ec6..c2d1a743d424 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -21,9 +21,16 @@ def test_runner_config(self): for job in CI.JobNames: self.assertIn(CI.JOB_CONFIGS[job].runner_type, CI.Runners) + def test_required_checks(self): + for job in CI.REQUIRED_CHECKS: + if job in (CI.StatusNames.PR_CHECK, CI.StatusNames.SYNC): + continue + self.assertTrue(job in CI.JOB_CONFIGS, f"Job [{job}] not in job config") + def test_builds_configs(self): """build name in the build config must match the job name""" for job in CI.JobNames: + self.assertTrue(job in CI.JOB_CONFIGS) self.assertTrue(CI.JOB_CONFIGS[job].runner_type in CI.Runners) if job in CI.BuildNames: self.assertTrue(CI.JOB_CONFIGS[job].build_config.name == job) From ed9bfbe393d217c56d8677677c023100cdb6b1ac Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 15 Jun 2024 16:06:31 +0000 Subject: [PATCH 175/254] Automatic style fix --- tests/ci/test_ci_options.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index fac6662b1aab..fc21c7dda4ec 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -170,10 +170,14 @@ def test_options_applied(self): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( + jobs_configs[ + "fuzzers" + ].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs["Integration tests (asan)"].release_only = ( + jobs_configs[ + "Integration tests (asan)" + ].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( @@ -305,9 +309,9 @@ def test_options_applied_4(self): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( - "TEST_LABEL" # check "fuzzers" does not appears in the result - ) + jobs_configs[ + "fuzzers" + ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( From 1d48bd7846d3e89809b42d55e87ce96e74c19561 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 15 Jun 2024 18:13:11 +0200 Subject: [PATCH 176/254] add SyncState class fir further adoption --- tests/ci/ci.py | 2 +- tests/ci/ci_config.py | 1 + tests/ci/ci_definitions.py | 8 ++++++++ tests/ci/test_ci_config.py | 2 +- 4 files changed, 11 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index cdf1be96b681..63d4120a210c 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -913,7 +913,7 @@ def _set_pending_statuses(pr_info: PRInfo) -> None: commit.create_status( state=PENDING, target_url="", - description="", + description=CI.SyncState.PENDING, context=CI.StatusNames.SYNC, ) except Exception as ex: diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 7eb4b7fdec27..655314cb6293 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -25,6 +25,7 @@ class CI: from ci_definitions import StatusNames as StatusNames from ci_definitions import CHECK_DESCRIPTIONS as CHECK_DESCRIPTIONS from ci_definitions import REQUIRED_CHECKS as REQUIRED_CHECKS + from ci_definitions import SyncState as SyncState from ci_definitions import MQ_JOBS as MQ_JOBS from ci_definitions import WorkflowStages as WorkflowStages from ci_definitions import Runners as Runners diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 3d732a997f54..9a4b845a61ed 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -212,6 +212,14 @@ class StatusNames(metaclass=WithIter): PR_CHECK = "PR Check" +class SyncState(metaclass=WithIter): + PENDING = "awaiting merge" + MERGE_FAILED = "merge failed" + TESTING = "awaiting test results" + TESTS_FAILED = "tests failed" + COMPLETED = "completed" + + @dataclass class DigestConfig: # all files, dirs to include into digest, glob supported diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index c2d1a743d424..d8208a5d9bc7 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -12,7 +12,7 @@ _TEST_EVENT_JSON = {"dummy": "dummy"} -# pylint:disable=protected-access +# pylint:disable=protected-access,union-attr class TestCIConfig(unittest.TestCase): From 0aa7665f048f25b275f25d2a4e2133029d9d2463 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 15 Jun 2024 18:25:49 +0200 Subject: [PATCH 177/254] do not pylint ci unittests --- utils/check-style/check-pylint | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/utils/check-style/check-pylint b/utils/check-style/check-pylint index 7959a414023f..8cfbc68ac96a 100755 --- a/utils/check-style/check-pylint +++ b/utils/check-style/check-pylint @@ -10,6 +10,7 @@ function xargs-pylint { xargs -P "$(nproc)" -n "$1" pylint --rcfile="$ROOT_PATH/pyproject.toml" --persistent=no --score=n } -find "$ROOT_PATH/tests" -maxdepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 50 +# exclude ci unittest scripts from check: test_* +find "$ROOT_PATH/tests" -maxdepth 2 -type f -exec file -F' ' --mime-type {} + | grep -v "/test_" | xargs-pylint 50 # Beware, there lambdas are checked. All of them contain `app`, and it causes brain-cucumber-zalgo find "$ROOT_PATH/tests/ci" -mindepth 2 -type f -exec file -F' ' --mime-type {} + | xargs-pylint 1 From f81cbbab2ab7963b0ed90e814e28c6ab20db0c82 Mon Sep 17 00:00:00 2001 From: Max K Date: Sat, 15 Jun 2024 19:02:15 +0200 Subject: [PATCH 178/254] do not mypy ci unittests --- utils/check-style/check-mypy | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/utils/check-style/check-mypy b/utils/check-style/check-mypy index 42cb7fbbd15a..4434377e627c 100755 --- a/utils/check-style/check-mypy +++ b/utils/check-style/check-mypy @@ -11,13 +11,15 @@ GIT_ROOT=${GIT_ROOT:-.} CONFIG="$GIT_ROOT/tests/ci/.mypy.ini" DIRS=("$GIT_ROOT/tests/ci/" "$GIT_ROOT/tests/ci/"*/) tmp=$(mktemp) + for dir in "${DIRS[@]}"; do if ! compgen -G "$dir"/*.py > /dev/null; then continue fi - if ! mypy --config-file="$CONFIG" --sqlite-cache "$dir"/*.py > "$tmp" 2>&1; then + if ! mypy --config-file="$CONFIG" --sqlite-cache $(find "$dir" -maxdepth 1 -name "*.py" | grep -v "test_") > "$tmp" 2>&1; then echo "Errors while processing $dir": cat "$tmp" fi done + rm -rf "$tmp" From 38f01bd831aa36efb76b43bf2f53357cd10ef92b Mon Sep 17 00:00:00 2001 From: chloro <13125187405@163.com> Date: Sun, 16 Jun 2024 14:31:49 +0800 Subject: [PATCH 179/254] update code style --- src/Analyzer/SetUtils.cpp | 18 +++++++++++------- src/Interpreters/ActionsVisitor.cpp | 13 ++++++++----- src/Interpreters/convertFieldToType.cpp | 2 +- src/Interpreters/convertFieldToType.h | 2 +- 4 files changed, 21 insertions(+), 14 deletions(-) diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 9a267bfa1493..71297169b5cd 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -74,7 +74,8 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes& { DataTypePtr data_type = value_types[value_types_index]; auto field = convertFieldToTypeStrict(value, *data_type, *block_types[0]); - if (!field) { + if (!field) + { value_types_index += 1; continue; } @@ -94,7 +95,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes& const auto & tuple = value.template get(); DataTypePtr value_type = value_types[value_types_index]; - DataTypes tuple_value_type = typeid_cast(value_type.get())->getElements(); + DataTypes tuple_value_type = typeid_cast(value_type.get())->getElements(); size_t tuple_size = tuple.size(); @@ -169,19 +170,22 @@ Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const WhichDataType rhs_which_type(value_type); - if (rhs_which_type.isArray()) { - const DataTypeArray* value_array_type = typeid_cast(value_type.get()); + if (rhs_which_type.isArray()) + { + const DataTypeArray * value_array_type = typeid_cast(value_type.get()); size_t value_array_size = value.get().size(); DataTypes value_types; value_types.reserve(value_array_size); - for(size_t i = 0; i < value_array_size; ++i) { + for (size_t i = 0; i < value_array_size; ++i) + { value_types.push_back(value_array_type->getNestedType()); } result_block = createBlockFromCollection(value.get(), value_types, set_element_types, transform_null_in); } - else if (rhs_which_type.isTuple()) { - const DataTypeTuple* value_tuple_type = typeid_cast(value_type.get()); + else if (rhs_which_type.isTuple()) + { + const DataTypeTuple * value_tuple_type = typeid_cast(value_type.get()); DataTypes value_types = value_tuple_type->getElements(); result_block = createBlockFromCollection(value.get(), value_types, set_element_types, transform_null_in); } diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 081e7e4fc2c7..831427180730 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -102,7 +102,7 @@ static size_t getTypeDepth(const DataTypePtr & type) /// 33.33 in the set is converted to 33.3, but it is not equal to 33.3 in the column, so the result should still be empty. /// We can not include values that don't represent any possible value from the type of filtered column to the set. template -static Block createBlockFromCollection(const Collection & collection, const DataTypes& value_types, const DataTypes & types, bool transform_null_in) +static Block createBlockFromCollection(const Collection & collection, const DataTypes & value_types, const DataTypes & types, bool transform_null_in) { size_t columns_num = types.size(); MutableColumns columns(columns_num); @@ -140,7 +140,7 @@ static Block createBlockFromCollection(const Collection & collection, const Data tuple_values.resize(tuple_size); DataTypePtr value_type = value_types[value_type_index]; - DataTypes tuple_value_type = typeid_cast(value_type.get())->getElements(); + DataTypes tuple_value_type = typeid_cast(value_type.get())->getElements(); size_t i = 0; for (; i < tuple_size; ++i) @@ -329,16 +329,19 @@ Block createBlockForSet( else if (left_type_depth + 1 == right_type_depth) { auto type_index = right_arg_type->getTypeId(); - if (type_index == TypeIndex::Tuple) { + if (type_index == TypeIndex::Tuple) + { DataTypes data_types = typeid_cast(right_arg_type.get())->getElements(); block = createBlockFromCollection(right_arg_value.get(), data_types, set_element_types, tranform_null_in); } - else if (type_index == TypeIndex::Array) { + else if (type_index == TypeIndex::Array) + { const auto* right_arg_array_type = typeid_cast(right_arg_type.get()); size_t right_arg_array_size = right_arg_value.get().size(); DataTypes data_types; data_types.reserve(right_arg_array_size); - for(size_t i = 0; i < right_arg_array_size; ++i) { + for(size_t i = 0; i < right_arg_array_size; ++i) + { data_types.push_back(right_arg_array_type->getNestedType()); } block = createBlockFromCollection(right_arg_value.get(), data_types, set_element_types, tranform_null_in); diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 57d4e18010bd..184c263dbdb7 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -615,7 +615,7 @@ static bool decimalEqualsFloat(Field field, Float64 float_value) return decimal_to_float == float_value; } -std::optional convertFieldToTypeStrict(const Field & from_value, const IDataType& from_type, const IDataType & to_type) +std::optional convertFieldToTypeStrict(const Field & from_value, const IDataType & from_type, const IDataType & to_type) { Field result_value = convertFieldToType(from_value, to_type, &from_type); diff --git a/src/Interpreters/convertFieldToType.h b/src/Interpreters/convertFieldToType.h index 044b0c9b1ceb..cb7903f587a5 100644 --- a/src/Interpreters/convertFieldToType.h +++ b/src/Interpreters/convertFieldToType.h @@ -22,6 +22,6 @@ Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_t /// Applies stricter rules than convertFieldToType, doesn't allow loss of precision converting to Decimal. /// Returns `Field` if the conversion was successful and the result is equal to the original value, otherwise returns nullopt. -std::optional convertFieldToTypeStrict(const Field & from_value, const IDataType& from_type, const IDataType & to_type); +std::optional convertFieldToTypeStrict(const Field & from_value, const IDataType & from_type, const IDataType & to_type); } From 1594f84daf18ed4c810c5b4023968194474ef618 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 16 Jun 2024 11:36:12 +0200 Subject: [PATCH 180/254] Fix test --- tests/integration/test_keeper_profiler/test.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_keeper_profiler/test.py b/tests/integration/test_keeper_profiler/test.py index 796d77987479..c0a70829107e 100644 --- a/tests/integration/test_keeper_profiler/test.py +++ b/tests/integration/test_keeper_profiler/test.py @@ -42,10 +42,8 @@ def test_profiler(started_cluster): "CREATE TABLE t (key UInt32, value String) Engine = ReplicatedMergeTree('/clickhouse-tables/test1', 'r1') ORDER BY key" ) - node.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") - node.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") - node.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") - node.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") + for _ in range(50): + node.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") node.query("system flush logs") assert int(node.query("exists system.trace_log")) From 2ecc53787ea2229dcd09c2c80b5054d952669be3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 16 Jun 2024 14:30:09 +0200 Subject: [PATCH 181/254] Ping CI From dc48eac7c8252fe96aa8c50d82bfe0f2782d3b41 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Sun, 16 Jun 2024 18:13:36 +0200 Subject: [PATCH 182/254] Update test.py --- tests/integration/test_keeper_profiler/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_profiler/test.py b/tests/integration/test_keeper_profiler/test.py index c0a70829107e..01c3680f89d8 100644 --- a/tests/integration/test_keeper_profiler/test.py +++ b/tests/integration/test_keeper_profiler/test.py @@ -42,7 +42,7 @@ def test_profiler(started_cluster): "CREATE TABLE t (key UInt32, value String) Engine = ReplicatedMergeTree('/clickhouse-tables/test1', 'r1') ORDER BY key" ) - for _ in range(50): + for _ in range(100): node.query("INSERT INTO t SELECT number, toString(number) from numbers(100)") node.query("system flush logs") From 27a0815bcc696b1aaf2c5756f56a254f2e7d7169 Mon Sep 17 00:00:00 2001 From: chloro <13125187405@163.com> Date: Sun, 16 Jun 2024 15:15:19 +0800 Subject: [PATCH 183/254] fix failed code style check --- src/Analyzer/SetUtils.cpp | 14 ++++++-------- src/Interpreters/ActionsVisitor.cpp | 10 +++++----- src/Interpreters/convertFieldToType.h | 2 +- 3 files changed, 12 insertions(+), 14 deletions(-) diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 71297169b5cd..9d898aea6ae4 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -72,7 +72,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes& { if (columns_size == 1) { - DataTypePtr data_type = value_types[value_types_index]; + const DataTypePtr & data_type = value_types[value_types_index]; auto field = convertFieldToTypeStrict(value, *data_type, *block_types[0]); if (!field) { @@ -83,7 +83,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes& bool need_insert_null = transform_null_in && block_types[0]->isNullable(); if (!field->isNull() || need_insert_null) columns[0]->insert(*field); - + value_types_index += 1; continue; } @@ -94,8 +94,8 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes& value.getTypeName()); const auto & tuple = value.template get(); - DataTypePtr value_type = value_types[value_types_index]; - DataTypes tuple_value_type = typeid_cast(value_type.get())->getElements(); + const DataTypePtr & value_type = value_types[value_types_index]; + const DataTypes & tuple_value_type = typeid_cast(value_type.get())->getElements(); size_t tuple_size = tuple.size(); @@ -124,7 +124,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes& if (i == tuple_size) for (i = 0; i < tuple_size; ++i) columns[i]->insert(tuple_values[i]); - + value_types_index += 1; } @@ -159,7 +159,6 @@ Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const if (lhs_type_depth == rhs_type_depth) { /// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc. - Array array{value}; DataTypes value_types{value_type}; result_block = createBlockFromCollection(array, value_types, set_element_types, transform_null_in); @@ -167,7 +166,6 @@ Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const else if (lhs_type_depth + 1 == rhs_type_depth) { /// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)) - WhichDataType rhs_which_type(value_type); if (rhs_which_type.isArray()) @@ -186,7 +184,7 @@ Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const else if (rhs_which_type.isTuple()) { const DataTypeTuple * value_tuple_type = typeid_cast(value_type.get()); - DataTypes value_types = value_tuple_type->getElements(); + const DataTypes & value_types = value_tuple_type->getElements(); result_block = createBlockFromCollection(value.get(), value_types, set_element_types, transform_null_in); } else diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 831427180730..e5a433b4bcdd 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -138,9 +138,9 @@ static Block createBlockFromCollection(const Collection & collection, const Data if (tuple_values.empty()) tuple_values.resize(tuple_size); - - DataTypePtr value_type = value_types[value_type_index]; - DataTypes tuple_value_type = typeid_cast(value_type.get())->getElements(); + + const DataTypePtr & value_type = value_types[value_type_index]; + const DataTypes & tuple_value_type = typeid_cast(value_type.get())->getElements(); size_t i = 0; for (; i < tuple_size; ++i) @@ -331,7 +331,7 @@ Block createBlockForSet( auto type_index = right_arg_type->getTypeId(); if (type_index == TypeIndex::Tuple) { - DataTypes data_types = typeid_cast(right_arg_type.get())->getElements(); + const DataTypes & data_types = typeid_cast(right_arg_type.get())->getElements(); block = createBlockFromCollection(right_arg_value.get(), data_types, set_element_types, tranform_null_in); } else if (type_index == TypeIndex::Array) @@ -340,7 +340,7 @@ Block createBlockForSet( size_t right_arg_array_size = right_arg_value.get().size(); DataTypes data_types; data_types.reserve(right_arg_array_size); - for(size_t i = 0; i < right_arg_array_size; ++i) + for (size_t i = 0; i < right_arg_array_size; ++i) { data_types.push_back(right_arg_array_type->getNestedType()); } diff --git a/src/Interpreters/convertFieldToType.h b/src/Interpreters/convertFieldToType.h index cb7903f587a5..4aa09f8619e2 100644 --- a/src/Interpreters/convertFieldToType.h +++ b/src/Interpreters/convertFieldToType.h @@ -22,6 +22,6 @@ Field convertFieldToTypeOrThrow(const Field & from_value, const IDataType & to_t /// Applies stricter rules than convertFieldToType, doesn't allow loss of precision converting to Decimal. /// Returns `Field` if the conversion was successful and the result is equal to the original value, otherwise returns nullopt. -std::optional convertFieldToTypeStrict(const Field & from_value, const IDataType & from_type, const IDataType & to_type); +std::optional convertFieldToTypeStrict(const Field & from_value, const IDataType & from_type, const IDataType & to_type); } From c815a3564003982d6e069eb8a816a0cad4907fcb Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Mon, 17 Jun 2024 02:28:16 +0300 Subject: [PATCH 184/254] fix corner cases --- src/Common/HilbertUtils.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/HilbertUtils.h b/src/Common/HilbertUtils.h index a6437019bd34..4f0a1d5dddfb 100644 --- a/src/Common/HilbertUtils.h +++ b/src/Common/HilbertUtils.h @@ -59,7 +59,7 @@ void segmentBinaryPartition(UInt64 start, UInt64 finish, UInt8 current_bits, F & return; const auto next_bits = current_bits - 2; - const auto history = (start >> current_bits) << current_bits; + const auto history = current_bits == 64 ? 0 : (start >> current_bits) << current_bits; const auto chunk_mask = 0b11; const auto start_chunk = (start >> next_bits) & chunk_mask; @@ -117,10 +117,10 @@ inline std::array, 2> createRangeFromCorners(UInt64 x1 UInt64 dist_x = x1 > x2 ? x1 - x2 : x2 - x1; UInt64 dist_y = y1 > y2 ? y1 - y2 : y2 - y1; UInt64 range_size = std::max(dist_x, dist_y); - bool containsMinimumVertice = x1 % (range_size + 1) == 0; - if (containsMinimumVertice) + UInt64 x_min = std::min(x1, x2); + bool contains_minimum_vertice = x_min % (range_size + 1) == 0; + if (contains_minimum_vertice) { - UInt64 x_min = std::min(x1, x2); UInt64 y_min = std::min(y1, y2); return { std::pair{x_min, x_min + range_size}, From 51ff9df7ebe9266cd2267cc1de0f05f32e1bd5b1 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Mon, 17 Jun 2024 03:41:25 +0300 Subject: [PATCH 185/254] fix --- src/Common/HilbertUtils.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/HilbertUtils.h b/src/Common/HilbertUtils.h index 4f0a1d5dddfb..f0f8360de909 100644 --- a/src/Common/HilbertUtils.h +++ b/src/Common/HilbertUtils.h @@ -117,10 +117,10 @@ inline std::array, 2> createRangeFromCorners(UInt64 x1 UInt64 dist_x = x1 > x2 ? x1 - x2 : x2 - x1; UInt64 dist_y = y1 > y2 ? y1 - y2 : y2 - y1; UInt64 range_size = std::max(dist_x, dist_y); - UInt64 x_min = std::min(x1, x2); - bool contains_minimum_vertice = x_min % (range_size + 1) == 0; + bool contains_minimum_vertice = x1 % (range_size + 1) == 0; if (contains_minimum_vertice) { + UInt64 x_min = std::min(x1, x2); UInt64 y_min = std::min(y1, y2); return { std::pair{x_min, x_min + range_size}, From d2d8006037b891b30c141a008ce968f77193c224 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 04:57:16 +0200 Subject: [PATCH 186/254] Fix error --- src/Parsers/TokenIterator.h | 14 +++++++++++++- src/Parsers/parseQuery.cpp | 10 +++++++--- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index 207ddadb8bfd..0d18ee5439ef 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -21,6 +21,7 @@ class Tokens { private: std::vector data; + size_t max_pos = 0; Lexer lexer; bool skip_insignificant; @@ -35,10 +36,16 @@ class Tokens while (true) { if (index < data.size()) + { + max_pos = std::max(max_pos, index); return data[index]; + } if (!data.empty() && data.back().isEnd()) + { + max_pos = data.size() - 1; return data.back(); + } Token token = lexer.nextToken(); @@ -51,7 +58,12 @@ class Tokens { if (data.empty()) return (*this)[0]; - return data.back(); + return data[max_pos]; + } + + void reset() + { + max_pos = 0; } }; diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index df9664060b31..a917722c92ce 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -285,6 +285,8 @@ ASTPtr tryParseQuery( return nullptr; } + Expected expected; + /** A shortcut - if Lexer found invalid tokens, fail early without full parsing. * But there are certain cases when invalid tokens are permitted: * 1. INSERT queries can have arbitrary data after the FORMAT clause, that is parsed by a different parser. @@ -293,9 +295,9 @@ ASTPtr tryParseQuery( * * This shortcut is needed to avoid complex backtracking in case of obviously erroneous queries. */ - IParser::Pos lookahead = token_iterator; - if (!ParserKeyword(Keyword::INSERT_INTO).ignore(lookahead)) + if (!ParserKeyword(Keyword::INSERT_INTO).check(token_iterator, expected)) { + IParser::Pos lookahead(token_iterator); while (lookahead->type != TokenType::Semicolon && lookahead->type != TokenType::EndOfStream) { if (lookahead->isError()) @@ -306,9 +308,11 @@ ASTPtr tryParseQuery( ++lookahead; } + + /// We should not spoil the info about maximum parsed position in the original iterator. + tokens.reset(); } - Expected expected; ASTPtr res; const bool parse_res = parser.parse(token_iterator, res, expected); const auto last_token = token_iterator.max(); From 9ecbc568903abba46fc0f874bb938882224591ac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 05:12:45 +0200 Subject: [PATCH 187/254] Fix errors --- src/Parsers/Kusto/KQL_ReleaseNote.md | 9 ++++----- src/Parsers/parseQuery.cpp | 4 ++-- .../0_stateless/02366_kql_create_table.sql | 18 +++++++++--------- 3 files changed, 15 insertions(+), 16 deletions(-) diff --git a/src/Parsers/Kusto/KQL_ReleaseNote.md b/src/Parsers/Kusto/KQL_ReleaseNote.md index bea1a627129a..440d0c73803c 100644 --- a/src/Parsers/Kusto/KQL_ReleaseNote.md +++ b/src/Parsers/Kusto/KQL_ReleaseNote.md @@ -853,7 +853,7 @@ Please note that the functions listed below only take constant parameters for no ## KQL() function - create table - `CREATE TABLE kql_table4 ENGINE = Memory AS select *, now() as new_column From kql(Customers | project LastName,Age);` + `CREATE TABLE kql_table4 ENGINE = Memory AS select *, now() as new_column From kql($$Customers | project LastName,Age$$);` verify the content of `kql_table` `select * from kql_table` @@ -867,12 +867,12 @@ Please note that the functions listed below only take constant parameters for no Age Nullable(UInt8) ) ENGINE = Memory; ``` - `INSERT INTO temp select * from kql(Customers|project FirstName,LastName,Age);` + `INSERT INTO temp select * from kql($$Customers|project FirstName,LastName,Age$$);` verify the content of `temp` `select * from temp` - - Select from kql() - `Select * from kql(Customers|project FirstName)` + - Select from kql(...) + `Select * from kql($$Customers|project FirstName$$)` ## KQL operators: - Tabular expression statements @@ -993,4 +993,3 @@ Please note that the functions listed below only take constant parameters for no - dcount() - dcountif() - bin - \ No newline at end of file diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index a917722c92ce..fab5dac8f87a 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -295,9 +295,9 @@ ASTPtr tryParseQuery( * * This shortcut is needed to avoid complex backtracking in case of obviously erroneous queries. */ - if (!ParserKeyword(Keyword::INSERT_INTO).check(token_iterator, expected)) + IParser::Pos lookahead(token_iterator); + if (!ParserKeyword(Keyword::INSERT_INTO).ignore(lookahead)) { - IParser::Pos lookahead(token_iterator); while (lookahead->type != TokenType::Semicolon && lookahead->type != TokenType::EndOfStream) { if (lookahead->isError()) diff --git a/tests/queries/0_stateless/02366_kql_create_table.sql b/tests/queries/0_stateless/02366_kql_create_table.sql index b266679b06aa..75a81c5dbd31 100644 --- a/tests/queries/0_stateless/02366_kql_create_table.sql +++ b/tests/queries/0_stateless/02366_kql_create_table.sql @@ -1,8 +1,8 @@ DROP TABLE IF EXISTS Customers; CREATE TABLE Customers -( +( FirstName Nullable(String), - LastName String, + LastName String, Occupation String, Education String, Age Nullable(UInt8) @@ -10,20 +10,20 @@ CREATE TABLE Customers INSERT INTO Customers VALUES ('Theodore','Diaz','Skilled Manual','Bachelors',28),('Stephanie','Cox','Management abcd defg','Bachelors',33),('Peter','Nara','Skilled Manual','Graduate Degree',26),('Latoya','Shen','Professional','Graduate Degree',25),('Apple','','Skilled Manual','Bachelors',28),(NULL,'why','Professional','Partial College',38); Select '-- test create table --' ; -Select * from kql(Customers|project FirstName) limit 1;; +Select * from kql($$Customers|project FirstName$$) limit 1;; DROP TABLE IF EXISTS kql_table1; -CREATE TABLE kql_table1 ENGINE = Memory AS select *, now() as new_column From kql(Customers | project LastName | filter LastName=='Diaz'); +CREATE TABLE kql_table1 ENGINE = Memory AS select *, now() as new_column From kql($$Customers | project LastName | filter LastName=='Diaz'$$); select LastName from kql_table1 limit 1; DROP TABLE IF EXISTS kql_table2; CREATE TABLE kql_table2 -( +( FirstName Nullable(String), - LastName String, + LastName String, Age Nullable(UInt8) ) ENGINE = Memory; -INSERT INTO kql_table2 select * from kql(Customers|project FirstName,LastName,Age | filter FirstName=='Theodore'); +INSERT INTO kql_table2 select * from kql($$Customers|project FirstName,LastName,Age | filter FirstName=='Theodore'$$); select * from kql_table2 limit 1; --- select * from kql(Customers | where FirstName !in ("test", "test2")); +-- select * from kql($$Customers | where FirstName !in ("test", "test2")$$); DROP TABLE IF EXISTS Customers; DROP TABLE IF EXISTS kql_table1; -DROP TABLE IF EXISTS kql_table2; \ No newline at end of file +DROP TABLE IF EXISTS kql_table2; From 92f538ae504b10f826fd01a3d91fdc7b66bb3e1c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 05:21:09 +0200 Subject: [PATCH 188/254] Disable async loading for MaterializedMySQL tests --- .../configs/no_async_load.xml | 3 +++ tests/integration/test_materialized_mysql_database/test.py | 6 +++--- 2 files changed, 6 insertions(+), 3 deletions(-) create mode 100644 tests/integration/test_materialized_mysql_database/configs/no_async_load.xml diff --git a/tests/integration/test_materialized_mysql_database/configs/no_async_load.xml b/tests/integration/test_materialized_mysql_database/configs/no_async_load.xml new file mode 100644 index 000000000000..1100874ad481 --- /dev/null +++ b/tests/integration/test_materialized_mysql_database/configs/no_async_load.xml @@ -0,0 +1,3 @@ + + false + diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 080a850a8c66..5efef3624db7 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -24,7 +24,7 @@ node_db = cluster.add_instance( "node1", - main_configs=["configs/timezone_config.xml"], + main_configs=["configs/timezone_config.xml", "configs/no_async_load.xml"], user_configs=["configs/users.xml"], with_mysql57=True, with_mysql8=True, @@ -32,7 +32,7 @@ ) node_disable_bytes_settings = cluster.add_instance( "node2", - main_configs=["configs/timezone_config.xml"], + main_configs=["configs/timezone_config.xml", "configs/no_async_load.xml"], user_configs=["configs/users_disable_bytes_settings.xml"], with_mysql57=False, with_mysql8=False, @@ -40,7 +40,7 @@ ) node_disable_rows_settings = cluster.add_instance( "node3", - main_configs=["configs/timezone_config.xml"], + main_configs=["configs/timezone_config.xml", "configs/no_async_load.xml"], user_configs=["configs/users_disable_rows_settings.xml"], with_mysql57=False, with_mysql8=False, From 279716519d88861d65ed8cbf8ef6cfc71a38de9e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 05:37:08 +0200 Subject: [PATCH 189/254] Fix tests --- .../configs/no_async_load.xml | 3 +++ tests/integration/test_disk_over_web_server/test.py | 10 +++++----- .../test_replicated_database/configs/config.xml | 1 + .../test_replicated_database/configs/config2.xml | 1 + 4 files changed, 10 insertions(+), 5 deletions(-) create mode 100644 tests/integration/test_disk_over_web_server/configs/no_async_load.xml diff --git a/tests/integration/test_disk_over_web_server/configs/no_async_load.xml b/tests/integration/test_disk_over_web_server/configs/no_async_load.xml new file mode 100644 index 000000000000..1100874ad481 --- /dev/null +++ b/tests/integration/test_disk_over_web_server/configs/no_async_load.xml @@ -0,0 +1,3 @@ + + false + diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index f4ea7d545714..891ee8f00f5f 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -11,13 +11,13 @@ def cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance( "node1", - main_configs=["configs/storage_conf.xml"], + main_configs=["configs/storage_conf.xml", "configs/no_async_load.xml"], with_nginx=True, use_old_analyzer=True, ) cluster.add_instance( "node2", - main_configs=["configs/storage_conf_web.xml"], + main_configs=["configs/storage_conf_web.xml", "configs/no_async_load.xml"], with_nginx=True, stay_alive=True, with_zookeeper=True, @@ -25,7 +25,7 @@ def cluster(): ) cluster.add_instance( "node3", - main_configs=["configs/storage_conf_web.xml"], + main_configs=["configs/storage_conf_web.xml", "configs/no_async_load.xml"], with_nginx=True, with_zookeeper=True, use_old_analyzer=True, @@ -33,7 +33,7 @@ def cluster(): cluster.add_instance( "node4", - main_configs=["configs/storage_conf.xml"], + main_configs=["configs/storage_conf.xml", "configs/no_async_load.xml"], with_nginx=True, stay_alive=True, with_installed_binary=True, @@ -42,7 +42,7 @@ def cluster(): ) cluster.add_instance( "node5", - main_configs=["configs/storage_conf.xml"], + main_configs=["configs/storage_conf.xml", "configs/no_async_load.xml"], with_nginx=True, use_old_analyzer=True, ) diff --git a/tests/integration/test_replicated_database/configs/config.xml b/tests/integration/test_replicated_database/configs/config.xml index 5150e9096de6..706628cf93be 100644 --- a/tests/integration/test_replicated_database/configs/config.xml +++ b/tests/integration/test_replicated_database/configs/config.xml @@ -6,4 +6,5 @@ 50 42 + false diff --git a/tests/integration/test_replicated_database/configs/config2.xml b/tests/integration/test_replicated_database/configs/config2.xml index 727461697ca7..8192c191952e 100644 --- a/tests/integration/test_replicated_database/configs/config2.xml +++ b/tests/integration/test_replicated_database/configs/config2.xml @@ -7,4 +7,5 @@ 50 42 group + false From 188f8a3df74caf830ad1ced3c4cf6dfb0aa90093 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 07:06:30 +0200 Subject: [PATCH 190/254] Fix test --- .../queries/0_stateless/01676_dictget_in_default_expression.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01676_dictget_in_default_expression.sql b/tests/queries/0_stateless/01676_dictget_in_default_expression.sql index 1785979f60b7..db23ae1919ce 100644 --- a/tests/queries/0_stateless/01676_dictget_in_default_expression.sql +++ b/tests/queries/0_stateless/01676_dictget_in_default_expression.sql @@ -23,7 +23,7 @@ ATTACH DATABASE test_01676; SELECT 'status_after_detach_and_attach:'; -- It can be not loaded, or not even finish attaching in case of asynchronous tables loading. -SELECT COALESCE((SELECT status FROM system.dictionaries WHERE database='test_01676' AND name='dict')::String, 'NOT_LOADED'); +SELECT COALESCE((SELECT status FROM system.dictionaries WHERE database='test_01676' AND name='dict')::Nullable(String), 'NOT_LOADED'); INSERT INTO test_01676.table (x) VALUES (toInt64(4)); SELECT * FROM test_01676.table ORDER BY x; From 7bed33012db0047db31e302c42193887138281f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 07:51:58 +0200 Subject: [PATCH 191/254] Fix bad code, but not error in system.session_log --- src/Access/SettingsProfilesInfo.cpp | 32 +++++++++------------------- src/Access/SettingsProfilesInfo.h | 6 +++++- src/Interpreters/Session.cpp | 8 +++---- src/Interpreters/Session.h | 3 +-- src/Interpreters/SessionLog.cpp | 2 +- src/Parsers/Kusto/KQL_ReleaseNote.md | 1 - 6 files changed, 21 insertions(+), 31 deletions(-) diff --git a/src/Access/SettingsProfilesInfo.cpp b/src/Access/SettingsProfilesInfo.cpp index d8b52ecf5e4d..a5eacbe1b6e5 100644 --- a/src/Access/SettingsProfilesInfo.cpp +++ b/src/Access/SettingsProfilesInfo.cpp @@ -15,22 +15,8 @@ namespace ErrorCodes bool operator==(const SettingsProfilesInfo & lhs, const SettingsProfilesInfo & rhs) { - if (lhs.settings != rhs.settings) - return false; - - if (lhs.constraints != rhs.constraints) - return false; - - if (lhs.profiles != rhs.profiles) - return false; - - if (lhs.profiles_with_implicit != rhs.profiles_with_implicit) - return false; - - if (lhs.names_of_profiles != rhs.names_of_profiles) - return false; - - return true; + return std::tie(lhs.settings, lhs.constraints, lhs.profiles, lhs.profiles_with_implicit, lhs.names_of_profiles) + == std::tie(rhs.settings, rhs.constraints, rhs.profiles, rhs.profiles_with_implicit, rhs.names_of_profiles); } std::shared_ptr @@ -66,18 +52,20 @@ Strings SettingsProfilesInfo::getProfileNames() const { Strings result; result.reserve(profiles.size()); - for (const auto & profile_id : profiles) + for (const UUID & profile_uuid : profiles) { - const auto p = names_of_profiles.find(profile_id); - if (p != names_of_profiles.end()) - result.push_back(p->second); + const auto names_it = names_of_profiles.find(profile_uuid); + if (names_it != names_of_profiles.end()) + { + result.push_back(names_it->second); + } else { - if (const auto name = access_control.tryReadName(profile_id)) + if (const auto name = access_control.tryReadName(profile_uuid)) // We could've updated cache here, but it is a very rare case, so don't bother. result.push_back(*name); else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to get profile name for {}", toString(profile_id)); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unable to get profile name for {}", toString(profile_uuid)); } } diff --git a/src/Access/SettingsProfilesInfo.h b/src/Access/SettingsProfilesInfo.h index ec289a5ec0a5..bc1b01f47d03 100644 --- a/src/Access/SettingsProfilesInfo.h +++ b/src/Access/SettingsProfilesInfo.h @@ -29,7 +29,11 @@ struct SettingsProfilesInfo /// Names of all the profiles in `profiles`. std::unordered_map names_of_profiles; - explicit SettingsProfilesInfo(const AccessControl & access_control_) : constraints(access_control_), access_control(access_control_) {} + explicit SettingsProfilesInfo(const AccessControl & access_control_) + : constraints(access_control_), access_control(access_control_) + { + } + std::shared_ptr getConstraintsAndProfileIDs( const std::shared_ptr & previous = nullptr) const; diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 396562189e07..9dd686290db4 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -532,7 +532,7 @@ ContextMutablePtr Session::makeSessionContext() session_context->checkSettingsConstraints(settings_from_auth_server, SettingSource::QUERY); session_context->applySettingsChanges(settings_from_auth_server); - recordLoginSucess(session_context); + recordLoginSuccess(session_context); return session_context; } @@ -596,7 +596,7 @@ ContextMutablePtr Session::makeSessionContext(const String & session_name_, std: { session_name_ }, max_sessions_for_user); - recordLoginSucess(session_context); + recordLoginSuccess(session_context); return session_context; } @@ -672,13 +672,13 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t user = query_context->getUser(); /// Interserver does not create session context - recordLoginSucess(query_context); + recordLoginSuccess(query_context); return query_context; } -void Session::recordLoginSucess(ContextPtr login_context) const +void Session::recordLoginSuccess(ContextPtr login_context) const { if (notified_session_log_about_login) return; diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 14f6f806acd4..fc41c78e666d 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -102,8 +102,7 @@ class Session private: std::shared_ptr getSessionLog() const; ContextMutablePtr makeQueryContextImpl(const ClientInfo * client_info_to_copy, ClientInfo * client_info_to_move) const; - void recordLoginSucess(ContextPtr login_context) const; - + void recordLoginSuccess(ContextPtr login_context) const; mutable bool notified_session_log_about_login = false; const UUID auth_id; diff --git a/src/Interpreters/SessionLog.cpp b/src/Interpreters/SessionLog.cpp index adb94cae0c28..dd6af8b2a19c 100644 --- a/src/Interpreters/SessionLog.cpp +++ b/src/Interpreters/SessionLog.cpp @@ -214,7 +214,7 @@ void SessionLog::addLoginSuccess(const UUID & auth_id, const ClientInfo & client_info, const UserPtr & login_user) { - DB::SessionLogElement log_entry(auth_id, SESSION_LOGIN_SUCCESS); + SessionLogElement log_entry(auth_id, SESSION_LOGIN_SUCCESS); log_entry.client_info = client_info; if (login_user) diff --git a/src/Parsers/Kusto/KQL_ReleaseNote.md b/src/Parsers/Kusto/KQL_ReleaseNote.md index bea1a627129a..fa60ce778359 100644 --- a/src/Parsers/Kusto/KQL_ReleaseNote.md +++ b/src/Parsers/Kusto/KQL_ReleaseNote.md @@ -993,4 +993,3 @@ Please note that the functions listed below only take constant parameters for no - dcount() - dcountif() - bin - \ No newline at end of file From f00750a152babd8bab7141abfac47b626e8a12e2 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 17 Jun 2024 08:33:31 +0200 Subject: [PATCH 192/254] do not overwrite Sync status if set --- tests/ci/ci.py | 25 ++++++++++++++++++------- 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 63d4120a210c..135a4c91c567 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -31,6 +31,7 @@ get_commit, post_commit_status, set_status_comment, + get_commit_filtered_statuses, ) from digest_helper import DockerDigester from env_helper import ( @@ -909,13 +910,23 @@ def _cancel_pr_wf(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> No def _set_pending_statuses(pr_info: PRInfo) -> None: commit = get_commit(GitHub(get_best_robot_token(), per_page=100), pr_info.sha) try: - print("Set SYNC status to pending") - commit.create_status( - state=PENDING, - target_url="", - description=CI.SyncState.PENDING, - context=CI.StatusNames.SYNC, - ) + found = False + statuses = get_commit_filtered_statuses(commit) + for commit_status in statuses: + if commit_status.context == CI.StatusNames.SYNC: + print( + f"Sync status found [{commit_status.state}], [{commit_status.description}] - won't be overwritten" + ) + found = True + break + if not found: + print("Set Sync status to pending") + commit.create_status( + state=PENDING, + target_url="", + description=CI.SyncState.PENDING, + context=CI.StatusNames.SYNC, + ) except Exception as ex: print(f"ERROR: failed to set GH commit status, ex: {ex}") From 016a680d309d6bace02712325f0fa765926444c5 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Mon, 17 Jun 2024 12:07:03 +0300 Subject: [PATCH 193/254] reload-ci From 59b871663fa0296d5f91d862bedf18915db233d0 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 17 Jun 2024 08:33:31 +0200 Subject: [PATCH 194/254] More ci unit tests --- tests/ci/ci_config.py | 18 +++++++++++++----- tests/ci/ci_definitions.py | 25 ++++++++++++++++++++++--- tests/ci/test_ci_config.py | 36 ++++++++++++++++++++++++++++++++++++ tests/ci/test_ci_options.py | 14 +++++--------- 4 files changed, 76 insertions(+), 17 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 655314cb6293..60c6a60af1ab 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -260,7 +260,8 @@ class CI: required_builds=[BuildNames.PACKAGE_RELEASE] ), JobNames.INSTALL_TEST_ARM: CommonJobConfigs.INSTALL_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_AARCH64] + required_builds=[BuildNames.PACKAGE_AARCH64], + runner_type=Runners.STYLE_CHECKER_ARM, ), JobNames.STATEFUL_TEST_ASAN: CommonJobConfigs.STATEFUL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN] @@ -284,7 +285,8 @@ class CI: required_builds=[BuildNames.PACKAGE_RELEASE_COVERAGE] ), JobNames.STATEFUL_TEST_AARCH64: CommonJobConfigs.STATEFUL_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_AARCH64] + required_builds=[BuildNames.PACKAGE_AARCH64], + runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.STATEFUL_TEST_PARALLEL_REPL_RELEASE: CommonJobConfigs.STATEFUL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE] @@ -331,6 +333,7 @@ class CI: ), JobNames.STATELESS_TEST_AARCH64: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], + runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.STATELESS_TEST_OLD_ANALYZER_S3_REPLICATED_RELEASE: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], num_batches=4 @@ -397,7 +400,9 @@ class CI: required_builds=[BuildNames.PACKAGE_TSAN], num_batches=6 ), JobNames.INTEGRATION_TEST_ARM: CommonJobConfigs.INTEGRATION_TEST.with_properties( - required_builds=[BuildNames.PACKAGE_AARCH64], num_batches=6 + required_builds=[BuildNames.PACKAGE_AARCH64], + num_batches=6, + runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.INTEGRATION_TEST: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], @@ -414,6 +419,7 @@ class CI: JobNames.COMPATIBILITY_TEST_ARM: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], required_on_release_branch=True, + runner_type=Runners.STYLE_CHECKER_ARM, ), JobNames.UNIT_TEST: CommonJobConfigs.UNIT_TEST.with_properties( required_builds=[BuildNames.BINARY_RELEASE], @@ -467,6 +473,7 @@ class CI: required_builds=[BuildNames.PACKAGE_AARCH64], num_batches=4, run_by_label="pr-performance", + runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.SQLANCER: CommonJobConfigs.SQLLANCER_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], @@ -480,11 +487,12 @@ class CI: JobNames.SQLTEST: CommonJobConfigs.SQL_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], ), - JobNames.CLICKBENCH_TEST: CommonJobConfigs.SQL_TEST.with_properties( + JobNames.CLICKBENCH_TEST: CommonJobConfigs.CLICKBENCH_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], ), - JobNames.CLICKBENCH_TEST_ARM: CommonJobConfigs.SQL_TEST.with_properties( + JobNames.CLICKBENCH_TEST_ARM: CommonJobConfigs.CLICKBENCH_TEST.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], + runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.LIBFUZZER_TEST: JobConfig( required_builds=[BuildNames.FUZZERS], diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 9a4b845a61ed..fdd5dc7a6711 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -280,6 +280,8 @@ class JobConfig: # GH Runner type (tag from @Runners) runner_type: str + # used for config validation in ci unittests + job_name_keyword: str = "" # builds required for the job (applicable for test jobs) required_builds: Optional[List[str]] = None # build config for the build job (applicable for builds) @@ -328,6 +330,7 @@ class CommonJobConfigs: """ BUILD_REPORT = JobConfig( + job_name_keyword="build_check", run_command="build_report_check.py", digest=DigestConfig( include_paths=[ @@ -338,23 +341,26 @@ class CommonJobConfigs: runner_type=Runners.STYLE_CHECKER_ARM, ) COMPATIBILITY_TEST = JobConfig( + job_name_keyword="compatibility", digest=DigestConfig( include_paths=["./tests/ci/compatibility_check.py"], docker=["clickhouse/test-old-ubuntu", "clickhouse/test-old-centos"], ), run_command="compatibility_check.py", - runner_type=Runners.STYLE_CHECKER_ARM, + runner_type=Runners.STYLE_CHECKER, ) INSTALL_TEST = JobConfig( + job_name_keyword="install", digest=DigestConfig( include_paths=["./tests/ci/install_check.py"], docker=["clickhouse/install-deb-test", "clickhouse/install-rpm-test"], ), run_command='install_check.py "$CHECK_NAME"', - runner_type=Runners.STYLE_CHECKER_ARM, + runner_type=Runners.STYLE_CHECKER, timeout=900, ) STATELESS_TEST = JobConfig( + job_name_keyword="stateless", digest=DigestConfig( include_paths=[ "./tests/ci/functional_test_check.py", @@ -371,6 +377,7 @@ class CommonJobConfigs: timeout=10800, ) STATEFUL_TEST = JobConfig( + job_name_keyword="stateful", digest=DigestConfig( include_paths=[ "./tests/ci/functional_test_check.py", @@ -387,6 +394,7 @@ class CommonJobConfigs: timeout=3600, ) STRESS_TEST = JobConfig( + job_name_keyword="stress", digest=DigestConfig( include_paths=[ "./tests/queries/0_stateless/", @@ -403,6 +411,7 @@ class CommonJobConfigs: timeout=9000, ) UPGRADE_TEST = JobConfig( + job_name_keyword="upgrade", digest=DigestConfig( include_paths=["./tests/ci/upgrade_check.py"], exclude_files=[".md"], @@ -412,6 +421,7 @@ class CommonJobConfigs: runner_type=Runners.STRESS_TESTER, ) INTEGRATION_TEST = JobConfig( + job_name_keyword="integration", digest=DigestConfig( include_paths=[ "./tests/ci/integration_test_check.py", @@ -425,12 +435,14 @@ class CommonJobConfigs: runner_type=Runners.STRESS_TESTER, ) ASTFUZZER_TEST = JobConfig( + job_name_keyword="ast", digest=DigestConfig(), run_command="ast_fuzzer_check.py", run_always=True, runner_type=Runners.FUZZER_UNIT_TESTER, ) UNIT_TEST = JobConfig( + job_name_keyword="unit", digest=DigestConfig( include_paths=["./tests/ci/unit_tests_check.py"], exclude_files=[".md"], @@ -440,6 +452,7 @@ class CommonJobConfigs: runner_type=Runners.FUZZER_UNIT_TESTER, ) PERF_TESTS = JobConfig( + job_name_keyword="performance", digest=DigestConfig( include_paths=[ "./tests/ci/performance_comparison_check.py", @@ -452,6 +465,7 @@ class CommonJobConfigs: runner_type=Runners.STRESS_TESTER, ) SQLLANCER_TEST = JobConfig( + job_name_keyword="lancer", digest=DigestConfig(), run_command="sqlancer_check.py", release_only=True, @@ -459,6 +473,7 @@ class CommonJobConfigs: runner_type=Runners.FUZZER_UNIT_TESTER, ) SQLLOGIC_TEST = JobConfig( + job_name_keyword="logic", digest=DigestConfig( include_paths=["./tests/ci/sqllogic_test.py"], exclude_files=[".md"], @@ -467,9 +482,10 @@ class CommonJobConfigs: run_command="sqllogic_test.py", timeout=10800, release_only=True, - runner_type=Runners.STYLE_CHECKER_ARM, + runner_type=Runners.STYLE_CHECKER, ) SQL_TEST = JobConfig( + job_name_keyword="sqltest", digest=DigestConfig( include_paths=["./tests/ci/sqltest.py"], exclude_files=[".md"], @@ -481,12 +497,14 @@ class CommonJobConfigs: runner_type=Runners.FUZZER_UNIT_TESTER, ) BUGFIX_TEST = JobConfig( + job_name_keyword="bugfix", digest=DigestConfig(), run_command="bugfix_validate_check.py", timeout=900, runner_type=Runners.FUNC_TESTER, ) DOCKER_SERVER = JobConfig( + job_name_keyword="docker", required_on_release_branch=True, run_command='docker_server.py --check-name "$CHECK_NAME" --release-type head --allow-build-reuse', digest=DigestConfig( @@ -498,6 +516,7 @@ class CommonJobConfigs: runner_type=Runners.STYLE_CHECKER, ) CLICKBENCH_TEST = JobConfig( + job_name_keyword="clickbench", digest=DigestConfig( include_paths=[ "tests/ci/clickbench.py", diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index d8208a5d9bc7..7a51a65b5d59 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -20,6 +20,42 @@ def test_runner_config(self): """check runner is provided w/o exception""" for job in CI.JobNames: self.assertIn(CI.JOB_CONFIGS[job].runner_type, CI.Runners) + if ( + job + in ( + CI.JobNames.STYLE_CHECK, + CI.JobNames.BUILD_CHECK, + ) + or "jepsen" in job.lower() + ): + self.assertTrue( + "style" in CI.JOB_CONFIGS[job].runner_type, + f"Job [{job}] must have style-checker(-aarch64) runner", + ) + elif "binary_" in job.lower() or "package_" in job.lower(): + self.assertTrue( + CI.JOB_CONFIGS[job].runner_type == CI.Runners.BUILDER, + f"Job [{job}] must have [{CI.Runners.BUILDER}] runner", + ) + elif "aarch64" in job.lower(): + self.assertTrue( + "aarch" in CI.JOB_CONFIGS[job].runner_type, + f"Job [{job}] does not match runner [{CI.JOB_CONFIGS[job].runner_type}]", + ) + else: + self.assertTrue( + "aarch" not in CI.JOB_CONFIGS[job].runner_type, + f"Job [{job}] does not match runner [{CI.JOB_CONFIGS[job].runner_type}]", + ) + + def test_common_configs_applied_properly(self): + for job in CI.JobNames: + if CI.JOB_CONFIGS[job].job_name_keyword: + self.assertTrue( + CI.JOB_CONFIGS[job].job_name_keyword.lower() + in normalize_string(job), + f"Job [{job}] apparently uses wrong common config with job keyword [{CI.JOB_CONFIGS[job].job_name_keyword}]", + ) def test_required_checks(self): for job in CI.REQUIRED_CHECKS: diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index fc21c7dda4ec..fac6662b1aab 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -170,14 +170,10 @@ def test_options_applied(self): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs[ - "fuzzers" - ].run_by_label = ( + jobs_configs["fuzzers"].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs[ - "Integration tests (asan)" - ].release_only = ( + jobs_configs["Integration tests (asan)"].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( @@ -309,9 +305,9 @@ def test_options_applied_4(self): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs[ - "fuzzers" - ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result + jobs_configs["fuzzers"].run_by_label = ( + "TEST_LABEL" # check "fuzzers" does not appears in the result + ) jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( From 9f64e46f5e5e9cb4b14ab94fa5badbed8850d8bf Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 17 Jun 2024 10:55:34 +0000 Subject: [PATCH 195/254] Automatic style fix --- tests/ci/test_ci_options.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index fac6662b1aab..fc21c7dda4ec 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -170,10 +170,14 @@ def test_options_applied(self): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( + jobs_configs[ + "fuzzers" + ].run_by_label = ( "TEST_LABEL" # check "fuzzers" appears in the result due to the label ) - jobs_configs["Integration tests (asan)"].release_only = ( + jobs_configs[ + "Integration tests (asan)" + ].release_only = ( True # still must be included as it's set with include keywords ) filtered_jobs = list( @@ -305,9 +309,9 @@ def test_options_applied_4(self): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( - "TEST_LABEL" # check "fuzzers" does not appears in the result - ) + jobs_configs[ + "fuzzers" + ].run_by_label = "TEST_LABEL" # check "fuzzers" does not appears in the result jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( From b845a242cc1f616eca3919a9d509afabc63b4f7f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 17 Jun 2024 11:06:22 +0000 Subject: [PATCH 196/254] Fixing build. --- src/Interpreters/tests/gtest_actions_visitor.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/tests/gtest_actions_visitor.cpp b/src/Interpreters/tests/gtest_actions_visitor.cpp index 3de39ae6bfaa..28e83306c536 100644 --- a/src/Interpreters/tests/gtest_actions_visitor.cpp +++ b/src/Interpreters/tests/gtest_actions_visitor.cpp @@ -31,7 +31,7 @@ TEST(ActionsVisitor, VisitLiteral) size_limits_for_set, size_t(0), name_and_types, - std::make_shared(name_and_types), + ActionsDAG(name_and_types), std::make_shared(), false /* no_subqueries */, false /* no_makeset */, @@ -39,7 +39,7 @@ TEST(ActionsVisitor, VisitLiteral) info); ActionsVisitor(visitor_data).visit(ast); auto actions = visitor_data.getActions(); - ASSERT_EQ(actions->getResultColumns().back().type->getTypeId(), expect_type->getTypeId()); + ASSERT_EQ(actions.getResultColumns().back().type->getTypeId(), expect_type->getTypeId()); } TEST(ActionsVisitor, VisitLiteralWithType) @@ -61,7 +61,7 @@ TEST(ActionsVisitor, VisitLiteralWithType) size_limits_for_set, size_t(0), name_and_types, - std::make_shared(name_and_types), + ActionsDAG(name_and_types), std::make_shared(), false /* no_subqueries */, false /* no_makeset */, @@ -69,5 +69,5 @@ TEST(ActionsVisitor, VisitLiteralWithType) info); ActionsVisitor(visitor_data).visit(ast); auto actions = visitor_data.getActions(); - ASSERT_EQ(actions->getResultColumns().back().type->getTypeId(), date_type->getTypeId()); + ASSERT_EQ(actions.getResultColumns().back().type->getTypeId(), date_type->getTypeId()); } From bab5359448c7c4ccf81727304713e8e8fd9cf1a4 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 17 Jun 2024 13:24:07 +0200 Subject: [PATCH 197/254] Update test.py --- tests/integration/test_keeper_profiler/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_keeper_profiler/test.py b/tests/integration/test_keeper_profiler/test.py index 01c3680f89d8..f9a90b9033e2 100644 --- a/tests/integration/test_keeper_profiler/test.py +++ b/tests/integration/test_keeper_profiler/test.py @@ -37,6 +37,8 @@ def started_cluster(): def test_profiler(started_cluster): node = cluster.instances["node1"] + if node1.is_built_with_sanitizer(): + return node.query( "CREATE TABLE t (key UInt32, value String) Engine = ReplicatedMergeTree('/clickhouse-tables/test1', 'r1') ORDER BY key" From 263b22d89cb8d74bf874c235c122dd1f9e91089a Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 17 Jun 2024 13:24:44 +0200 Subject: [PATCH 198/254] Update test.py --- tests/integration/test_keeper_profiler/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_profiler/test.py b/tests/integration/test_keeper_profiler/test.py index f9a90b9033e2..28dae69bd5fb 100644 --- a/tests/integration/test_keeper_profiler/test.py +++ b/tests/integration/test_keeper_profiler/test.py @@ -37,7 +37,7 @@ def started_cluster(): def test_profiler(started_cluster): node = cluster.instances["node1"] - if node1.is_built_with_sanitizer(): + if node.is_built_with_sanitizer(): return node.query( From e13ff4ec43ab8f8d9738a11ff4372232874b7d19 Mon Sep 17 00:00:00 2001 From: Max K Date: Mon, 17 Jun 2024 14:00:58 +0200 Subject: [PATCH 199/254] add Builds_2 dependency for Build_Report --- .github/workflows/master.yml | 2 +- .github/workflows/pull_request.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 837dbba61743..88bc50a729d6 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -119,7 +119,7 @@ jobs: Builds_Report: # run report check for failed builds to indicate the CI error if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} - needs: [RunConfig, Builds_1] + needs: [RunConfig, Builds_1, Builds_2] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickHouse build check diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 422bcf551ae4..70b71da8fa5e 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -147,7 +147,7 @@ jobs: Builds_Report: # run report check for failed builds to indicate the CI error if: ${{ !cancelled() && needs.RunConfig.result == 'success' && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse build check') }} - needs: [RunConfig, StyleCheck, Builds_1] + needs: [RunConfig, StyleCheck, Builds_1, Builds_2] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickHouse build check From e3818a97944bb10d56646b4145696a60a7aa0edd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Jun 2024 15:55:42 +0000 Subject: [PATCH 200/254] Re-enable session caching --- base/poco/NetSSL_OpenSSL/src/SSLManager.cpp | 41 +++++++++---------- .../01393_benchmark_secure_port.sh | 2 +- .../0_stateless/01683_text_log_deadlock.sh | 2 +- 3 files changed, 22 insertions(+), 23 deletions(-) diff --git a/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp b/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp index 7f6cc9abcb28..d404aed4d136 100644 --- a/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp @@ -330,27 +330,26 @@ void SSLManager::initDefaultContext(bool server) else _ptrDefaultClientContext->disableProtocols(disabledProtocols); - /// Temporarily disabled during the transition from boringssl to OpenSSL due to tsan issues. - /// bool cacheSessions = config.getBool(prefix + CFG_CACHE_SESSIONS, false); - /// if (server) - /// { - /// std::string sessionIdContext = config.getString(prefix + CFG_SESSION_ID_CONTEXT, config.getString("application.name", "")); - /// _ptrDefaultServerContext->enableSessionCache(cacheSessions, sessionIdContext); - /// if (config.hasProperty(prefix + CFG_SESSION_CACHE_SIZE)) - /// { - /// int cacheSize = config.getInt(prefix + CFG_SESSION_CACHE_SIZE); - /// _ptrDefaultServerContext->setSessionCacheSize(cacheSize); - /// } - /// if (config.hasProperty(prefix + CFG_SESSION_TIMEOUT)) - /// { - /// int timeout = config.getInt(prefix + CFG_SESSION_TIMEOUT); - /// _ptrDefaultServerContext->setSessionTimeout(timeout); - /// } - /// } - /// else - /// { - /// _ptrDefaultClientContext->enableSessionCache(cacheSessions); - /// } + bool cacheSessions = config.getBool(prefix + CFG_CACHE_SESSIONS, false); + if (server) + { + std::string sessionIdContext = config.getString(prefix + CFG_SESSION_ID_CONTEXT, config.getString("application.name", "")); + _ptrDefaultServerContext->enableSessionCache(cacheSessions, sessionIdContext); + if (config.hasProperty(prefix + CFG_SESSION_CACHE_SIZE)) + { + int cacheSize = config.getInt(prefix + CFG_SESSION_CACHE_SIZE); + _ptrDefaultServerContext->setSessionCacheSize(cacheSize); + } + if (config.hasProperty(prefix + CFG_SESSION_TIMEOUT)) + { + int timeout = config.getInt(prefix + CFG_SESSION_TIMEOUT); + _ptrDefaultServerContext->setSessionTimeout(timeout); + } + } + else + { + _ptrDefaultClientContext->enableSessionCache(cacheSessions); + } bool extendedVerification = config.getBool(prefix + CFG_EXTENDED_VERIFICATION, false); if (server) _ptrDefaultServerContext->enableExtendedCertificateVerification(extendedVerification); diff --git a/tests/queries/0_stateless/01393_benchmark_secure_port.sh b/tests/queries/0_stateless/01393_benchmark_secure_port.sh index 7954e439977f..c1874a079770 100755 --- a/tests/queries/0_stateless/01393_benchmark_secure_port.sh +++ b/tests/queries/0_stateless/01393_benchmark_secure_port.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-tsan, no-asan +# Tags: no-fasttest, no-asan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01683_text_log_deadlock.sh b/tests/queries/0_stateless/01683_text_log_deadlock.sh index 1aced61cb422..e838ab87c1da 100755 --- a/tests/queries/0_stateless/01683_text_log_deadlock.sh +++ b/tests/queries/0_stateless/01683_text_log_deadlock.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: deadlock, no-tsan, no-asan +# Tags: deadlock, no-asan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From e484390a0e27528c0baa2e0f99a3bd5143184117 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Jun 2024 23:24:21 +0200 Subject: [PATCH 201/254] Remove no-asan tag from tests --- tests/queries/0_stateless/01393_benchmark_secure_port.sh | 2 +- tests/queries/0_stateless/01683_text_log_deadlock.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01393_benchmark_secure_port.sh b/tests/queries/0_stateless/01393_benchmark_secure_port.sh index c1874a079770..f75577e6ddf0 100755 --- a/tests/queries/0_stateless/01393_benchmark_secure_port.sh +++ b/tests/queries/0_stateless/01393_benchmark_secure_port.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-asan +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01683_text_log_deadlock.sh b/tests/queries/0_stateless/01683_text_log_deadlock.sh index e838ab87c1da..af7f348a6a27 100755 --- a/tests/queries/0_stateless/01683_text_log_deadlock.sh +++ b/tests/queries/0_stateless/01683_text_log_deadlock.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: deadlock, no-asan +# Tags: deadlock CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From fc4249b0e1a365a17c497179c2c412fdf7798733 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 12 Jun 2024 09:54:07 +0000 Subject: [PATCH 202/254] Reduce runtime of 01683_text_log_deadlock - some runs of this test in CI timed out, others succeeded - reducing the test size is not a greate strategy compared to reproducing the slowness locally. I could not make the test run on my system (clickhouse-benchmark did not connect to the server via a secure connecstion, interestingly a secure connection did work for clickhouse-client). - this commit should unblock CI and uncover more interesting bugs caused by re-enabling session caching --- tests/queries/0_stateless/01683_text_log_deadlock.reference | 2 +- tests/queries/0_stateless/01683_text_log_deadlock.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01683_text_log_deadlock.reference b/tests/queries/0_stateless/01683_text_log_deadlock.reference index 4cf614602521..3805f2a95e9d 100644 --- a/tests/queries/0_stateless/01683_text_log_deadlock.reference +++ b/tests/queries/0_stateless/01683_text_log_deadlock.reference @@ -1 +1 @@ -queries: 25000 +queries: 5000 diff --git a/tests/queries/0_stateless/01683_text_log_deadlock.sh b/tests/queries/0_stateless/01683_text_log_deadlock.sh index af7f348a6a27..6b3bcc588685 100755 --- a/tests/queries/0_stateless/01683_text_log_deadlock.sh +++ b/tests/queries/0_stateless/01683_text_log_deadlock.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_BENCHMARK --secure -i 25000 -c 32 --query 'SELECT 1' 2>&1 | grep -oF 'queries: 25000' +$CLICKHOUSE_BENCHMARK --secure -i 5000 -c 32 --query 'SELECT 1' 2>&1 | grep -oF 'queries: 5000' From 186bd0cc3d21f124e12a19b7d4874111d2597a8e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jun 2024 12:20:54 +0000 Subject: [PATCH 203/254] Temporary fix for tsan issue openssl#24629 --- contrib/openssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/openssl b/contrib/openssl index 67c0b63e578e..e0d6ae2bf93c 160000 --- a/contrib/openssl +++ b/contrib/openssl @@ -1 +1 @@ -Subproject commit 67c0b63e578e4c751ac9edf490f5a96124fff8dc +Subproject commit e0d6ae2bf93cf6dc26bb86aa39992bc6a410869a From b0b2c3fea99007b9f22ff34c3b1d5b9b51cad4b4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jun 2024 13:19:59 +0000 Subject: [PATCH 204/254] Minor clarification of setting docs --- docs/en/operations/settings/settings.md | 2 +- src/Core/Settings.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 74d81548e98d..59dd92f0fcda 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5420,7 +5420,7 @@ Default value: `true`. ## uniform_snowflake_conversion_functions {#uniform_snowflake_conversion_functions} -Controls if functions `snowflakeIDToDateTime`, `snowflakeIDToDateTime64`, `dateTimeToSnowflakeID`, and `dateTime64ToSnowflakeID` are enabled (if `true`), or functions `snowflakeToDateTime`, `snowflakeToDateTime64`, `dateTimeToSnowflake`, and `dateTime64ToSnowflake` (if `false`). +If set to `true`, then functions `snowflakeIDToDateTime`, `snowflakeIDToDateTime64`, `dateTimeToSnowflakeID`, and `dateTime64ToSnowflakeID` are enabled, and functions `snowflakeToDateTime`, `snowflakeToDateTime64`, `dateTimeToSnowflake`, and `dateTime64ToSnowflake` are disabled (and vice versa if set to `false`). Default value: `true` diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 064faa228aec..182fe67f135b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -930,7 +930,7 @@ class IColumn; M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \ M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ M(Bool, allow_deprecated_error_prone_window_functions, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)", 0) \ - M(Bool, uniform_snowflake_conversion_functions, true, "Enable functions snowflakeIDToDateTime[64] and dateTime[64]ToSnowflakeID.", 0) \ + M(Bool, uniform_snowflake_conversion_functions, true, "Enables functions snowflakeIDToDateTime[64] and dateTime[64]ToSnowflakeID while disabling functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. From c9178418ff2e3f36ab9801791f1fd9a3cf2c28e3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 17 Jun 2024 15:56:51 +0000 Subject: [PATCH 205/254] Another attempt. --- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageMerge.cpp | 10 ---------- 2 files changed, 1 insertion(+), 11 deletions(-) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 5048ef4788e5..1e013ff9a0c4 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -849,7 +849,7 @@ void StorageDistributed::read( storage_snapshot, remote_storage_id, remote_table_function_ptr); - header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_distributed, local_context, SelectQueryOptions(processed_stage).analyze()); + header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_distributed->toAST(), local_context, SelectQueryOptions(processed_stage).analyze()); /** For distributed tables we do not need constants in header, since we don't send them to remote servers. * Moreover, constants can break some functions like `hostName` that are constants only for local queries. */ diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 419e944f456e..0e75282f4991 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1172,16 +1172,6 @@ ReadFromMerge::ChildPlan ReadFromMerge::createPlanForTable( if (real_column_names.empty()) real_column_names.push_back(ExpressionActions::getSmallestColumn(storage_snapshot_->metadata->getColumns().getAllPhysical()).name); - if (allow_experimental_analyzer) - { - auto ast = modified_query_info.query_tree->toAST(); - InterpreterSelectQueryAnalyzer interpreter(ast, - modified_context, - SelectQueryOptions(processed_stage)); - - modified_query_info.query_tree = interpreter.getQueryTree(); - } - storage->read(plan, real_column_names, storage_snapshot_, From cd1475a5dfcf5f45d222b17942c2fe95b80606f0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jun 2024 17:59:17 +0200 Subject: [PATCH 206/254] Debug why test failed on aarch64 --- .../integration/test_keeper_profiler/test.py | 38 +++++++++++++++++-- 1 file changed, 34 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_keeper_profiler/test.py b/tests/integration/test_keeper_profiler/test.py index 28dae69bd5fb..f5095221286b 100644 --- a/tests/integration/test_keeper_profiler/test.py +++ b/tests/integration/test_keeper_profiler/test.py @@ -50,9 +50,8 @@ def test_profiler(started_cluster): node.query("system flush logs") assert int(node.query("exists system.trace_log")) - assert 1 < int( - node.query( - """ + result = node.query( + """ set allow_introspection_functions=1; system flush logs; select cnt from ( @@ -62,5 +61,36 @@ def test_profiler(started_cluster): '\n') as trace from system.trace_log where trace_type = ‘Real’ and trace ilike '%KeeperTCPHandler%' group by trace); """ - ) ) + + if len(result) == 0: + assert 0 < int( + node.query( + """ + set allow_introspection_functions=1; + system flush logs; + select cnt from ( + select count() as cnt, formatReadableSize(sum(size)), + arrayStringConcat( + arrayMap(x, y -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), arrayMap(x -> demangle(addressToSymbol(x)), trace)), + '\n') as trace + from system.trace_log where trace_type = ‘Real’ group by trace); + """ + ) + ) + result = node.query( + """ + set allow_introspection_functions=1; + system flush logs; + select * from ( + select count() as cnt, formatReadableSize(sum(size)), + arrayStringConcat( + arrayMap(x, y -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), arrayMap(x -> demangle(addressToSymbol(x)), trace)), + '\n') as trace + from system.trace_log where trace_type = ‘Real’ group by trace); + """ + ) + print(result) + assert False + + assert 1 < int(result) From 84f70ba664c6d9562e0b1bbfa62ffea432f505a6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 17 Jun 2024 16:09:55 +0000 Subject: [PATCH 207/254] Trying to fix more tests. --- src/Interpreters/ActionsVisitor.cpp | 2 +- src/Interpreters/ActionsVisitor.h | 2 +- src/Interpreters/ExpressionActions.h | 7 ++++++- src/Interpreters/ExpressionAnalyzer.cpp | 4 +++- 4 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 89de76d28153..9e56d740e5e7 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -394,7 +394,7 @@ Block createBlockForSet( ScopeStack::Level::Level() = default; ScopeStack::Level::~Level() = default; -ScopeStack::Level::Level(Level &&) = default; +ScopeStack::Level::Level(Level &&) noexcept = default; FutureSetPtr makeExplicitSet( const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets) diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 7e3b7fcb6bbe..46d2d60e461f 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -51,7 +51,7 @@ struct ScopeStack : WithContext ~Level(); Level(); - Level(Level &&); + Level(Level &&) noexcept; }; using Levels = std::deque; diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index 7c6af41c04db..c762abc1149b 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -288,10 +288,15 @@ struct ExpressionActionsChain : WithContext Step & lastStep(const NamesAndTypesList & columns) { if (steps.empty()) - steps.emplace_back(std::make_unique(std::make_shared(ActionsDAG(columns), false, false))); + return addStep(columns); return *steps.back(); } + Step & addStep(const NamesAndTypesList & columns) + { + return *steps.emplace_back(std::make_unique(std::make_shared(ActionsDAG(columns), false, false))); + } + std::string dumpChain() const; }; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index d0f037ad9e1f..ff65475d9af3 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1466,7 +1466,9 @@ void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAG & before_a else result_columns.push_back(source_column); } - ExpressionActionsChain::Step & step = chain.lastStep(before_aggregation.getNamesAndTypesList()); + auto required_output = chain.getLastStep().required_output; + ExpressionActionsChain::Step & step = chain.addStep(before_aggregation.getNamesAndTypesList()); + step.required_output = std::move(required_output); step.actions()->actions = std::move(*ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position)); } From e310533930c6857e6ed08bca1ba4cc7e56240cd8 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 17 Jun 2024 18:16:05 +0200 Subject: [PATCH 208/254] Forbid QUALIFY clause in the old analyzer --- src/Interpreters/InterpreterSelectQuery.cpp | 3 +++ .../0_stateless/03173_forbid_qualify.reference | 3 +++ tests/queries/0_stateless/03173_forbid_qualify.sql | 11 +++++++++++ 3 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/03173_forbid_qualify.reference create mode 100644 tests/queries/0_stateless/03173_forbid_qualify.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8e072779b53a..38ffd40b6cd7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1474,6 +1474,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

(source_header); diff --git a/tests/queries/0_stateless/03173_forbid_qualify.reference b/tests/queries/0_stateless/03173_forbid_qualify.reference new file mode 100644 index 000000000000..c2f595d8c4bd --- /dev/null +++ b/tests/queries/0_stateless/03173_forbid_qualify.reference @@ -0,0 +1,3 @@ +100 +49 +100 diff --git a/tests/queries/0_stateless/03173_forbid_qualify.sql b/tests/queries/0_stateless/03173_forbid_qualify.sql new file mode 100644 index 000000000000..59f0153cd366 --- /dev/null +++ b/tests/queries/0_stateless/03173_forbid_qualify.sql @@ -0,0 +1,11 @@ +drop table if exists default.test_qualify; +create table default.test_qualify (number Int64) ENGINE = MergeTree ORDER BY (number); + +insert into default.test_qualify SELECT * FROM numbers(100); + +select count() from default.test_qualify; -- 100 +select * from default.test_qualify qualify row_number() over (order by number) = 50 SETTINGS allow_experimental_analyzer = 1; -- 49 +select * from default.test_qualify qualify row_number() over (order by number) = 50 SETTINGS allow_experimental_analyzer = 0; -- { serverError NOT_IMPLEMENTED } + +delete from default.test_qualify where number in (select number from default.test_qualify qualify row_number() over (order by number) = 50); -- { serverError UNFINISHED } +select count() from default.test_qualify; -- 100 From 6b889bf07f07d5e7fd545c921ef99a4f2b89c9b5 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Mon, 17 Jun 2024 22:41:36 +0300 Subject: [PATCH 209/254] reload-ci From 8d072ade18b61b6eafef0899f5ce551030b6662d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jun 2024 22:35:49 +0200 Subject: [PATCH 210/254] Update test --- tests/integration/test_keeper_profiler/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_profiler/test.py b/tests/integration/test_keeper_profiler/test.py index f5095221286b..98738890ad84 100644 --- a/tests/integration/test_keeper_profiler/test.py +++ b/tests/integration/test_keeper_profiler/test.py @@ -69,7 +69,7 @@ def test_profiler(started_cluster): """ set allow_introspection_functions=1; system flush logs; - select cnt from ( + select sum(cnt) from ( select count() as cnt, formatReadableSize(sum(size)), arrayStringConcat( arrayMap(x, y -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), arrayMap(x -> demangle(addressToSymbol(x)), trace)), From fb110827f82746964fc91ef73d45719244bbcad8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 14 Jun 2024 19:16:37 +0200 Subject: [PATCH 211/254] save server data for failed stateless tests --- docker/test/stateless/run.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index f94621ba0927..4434a5338a78 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -254,7 +254,7 @@ function run_tests() set +e clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --hung-check --print-time \ - --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ + --no-drop-if-fail --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt set -e @@ -379,6 +379,10 @@ fi tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||: +rm -rf /var/lib/clickhouse/data/system/*/ +tar -chf /test_output/store.tar /var/lib/clickhouse/store ||: +tar -chf /test_output/metadata.tar /var/lib/clickhouse/metadata/*.sql ||: + if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then rg -Fa "" /var/log/clickhouse-server/clickhouse-server1.log ||: rg -Fa "" /var/log/clickhouse-server/clickhouse-server2.log ||: From 42dd981fe41dc22857f65e2f21b6d03893b04b4a Mon Sep 17 00:00:00 2001 From: chloro <13125187405@163.com> Date: Tue, 18 Jun 2024 08:14:14 +0800 Subject: [PATCH 212/254] fix code review --- src/Analyzer/SetUtils.cpp | 26 +++++++++---------------- src/Interpreters/ActionsVisitor.cpp | 30 ++++++++++++----------------- 2 files changed, 21 insertions(+), 35 deletions(-) diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 9d898aea6ae4..0ecb3545225f 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -9,6 +9,8 @@ #include #include +#include + namespace DB { @@ -66,17 +68,16 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes& } Row tuple_values; - size_t value_types_index = 0; - for (const auto & value : collection) + for (size_t collection_index = 0; collection_index < collection.size(); ++collection_index) { + const auto & value = collection[collection_index]; if (columns_size == 1) { - const DataTypePtr & data_type = value_types[value_types_index]; + const DataTypePtr & data_type = value_types[collection_index]; auto field = convertFieldToTypeStrict(value, *data_type, *block_types[0]); if (!field) { - value_types_index += 1; continue; } @@ -84,7 +85,6 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes& if (!field->isNull() || need_insert_null) columns[0]->insert(*field); - value_types_index += 1; continue; } @@ -94,7 +94,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes& value.getTypeName()); const auto & tuple = value.template get(); - const DataTypePtr & value_type = value_types[value_types_index]; + const DataTypePtr & value_type = value_types[collection_index]; const DataTypes & tuple_value_type = typeid_cast(value_type.get())->getElements(); size_t tuple_size = tuple.size(); @@ -124,8 +124,6 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes& if (i == tuple_size) for (i = 0; i < tuple_size; ++i) columns[i]->insert(tuple_values[i]); - - value_types_index += 1; } Block res; @@ -170,20 +168,14 @@ Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const if (rhs_which_type.isArray()) { - const DataTypeArray * value_array_type = typeid_cast(value_type.get()); + const DataTypeArray * value_array_type = assert_cast(value_type.get()); size_t value_array_size = value.get().size(); - DataTypes value_types; - value_types.reserve(value_array_size); - - for (size_t i = 0; i < value_array_size; ++i) - { - value_types.push_back(value_array_type->getNestedType()); - } + DataTypes value_types(value_array_size, value_array_type->getNestedType()); result_block = createBlockFromCollection(value.get(), value_types, set_element_types, transform_null_in); } else if (rhs_which_type.isTuple()) { - const DataTypeTuple * value_tuple_type = typeid_cast(value_type.get()); + const DataTypeTuple * value_tuple_type = assert_cast(value_type.get()); const DataTypes & value_types = value_tuple_type->getElements(); result_block = createBlockFromCollection(value.get(), value_types, set_element_types, transform_null_in); } diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index e5a433b4bcdd..3f4afff56e8c 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -113,16 +114,15 @@ static Block createBlockFromCollection(const Collection & collection, const Data } Row tuple_values; - size_t value_type_index = 0; - for (const auto & value : collection) + for (size_t collection_index = 0; collection_index < collection.size(); ++collection_index) { + const auto& value = collection[collection_index]; if (columns_num == 1) { - auto field = convertFieldToTypeStrict(value, *value_types[value_type_index], *types[0]); + auto field = convertFieldToTypeStrict(value, *value_types[collection_index], *types[0]); bool need_insert_null = transform_null_in && types[0]->isNullable(); if (field && (!field->isNull() || need_insert_null)) columns[0]->insert(*field); - value_type_index += 1; } else { @@ -139,7 +139,7 @@ static Block createBlockFromCollection(const Collection & collection, const Data if (tuple_values.empty()) tuple_values.resize(tuple_size); - const DataTypePtr & value_type = value_types[value_type_index]; + const DataTypePtr & value_type = value_types[collection_index]; const DataTypes & tuple_value_type = typeid_cast(value_type.get())->getElements(); size_t i = 0; @@ -158,7 +158,6 @@ static Block createBlockFromCollection(const Collection & collection, const Data if (i == tuple_size) for (i = 0; i < tuple_size; ++i) columns[i]->insert(tuple_values[i]); - value_type_index += 1; } } @@ -322,8 +321,8 @@ Block createBlockForSet( if (left_type_depth == right_type_depth) { Array array{right_arg_value}; - DataTypes data_types{right_arg_type}; - block = createBlockFromCollection(array, data_types, set_element_types, tranform_null_in); + DataTypes value_types{right_arg_type}; + block = createBlockFromCollection(array, value_types, set_element_types, tranform_null_in); } /// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc. else if (left_type_depth + 1 == right_type_depth) @@ -331,20 +330,15 @@ Block createBlockForSet( auto type_index = right_arg_type->getTypeId(); if (type_index == TypeIndex::Tuple) { - const DataTypes & data_types = typeid_cast(right_arg_type.get())->getElements(); - block = createBlockFromCollection(right_arg_value.get(), data_types, set_element_types, tranform_null_in); + const DataTypes & value_types = assert_cast(right_arg_type.get())->getElements(); + block = createBlockFromCollection(right_arg_value.get(), value_types, set_element_types, tranform_null_in); } else if (type_index == TypeIndex::Array) { - const auto* right_arg_array_type = typeid_cast(right_arg_type.get()); + const auto* right_arg_array_type = assert_cast(right_arg_type.get()); size_t right_arg_array_size = right_arg_value.get().size(); - DataTypes data_types; - data_types.reserve(right_arg_array_size); - for (size_t i = 0; i < right_arg_array_size; ++i) - { - data_types.push_back(right_arg_array_type->getNestedType()); - } - block = createBlockFromCollection(right_arg_value.get(), data_types, set_element_types, tranform_null_in); + DataTypes value_types(right_arg_array_size, right_arg_array_type->getNestedType()); + block = createBlockFromCollection(right_arg_value.get(), value_types, set_element_types, tranform_null_in); } else throw_unsupported_type(right_arg_type); From 579e94319c8e93e1e58dac1942691be195ca97a3 Mon Sep 17 00:00:00 2001 From: haohang Date: Tue, 18 Jun 2024 10:58:44 +0800 Subject: [PATCH 213/254] [docs] pin the gimli-rs/addr2line version in Update allocation-profiling.md after upstream changed the folder, the build command is broken: https://github.com/gimli-rs/addr2line/pull/291/files#diff-2e9d962a08321605940b5a657135052fbcef87b5e360662bb527c96d9a615542R70-R72 for a more stable way, I intend to pin the version --- docs/en/operations/allocation-profiling.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/allocation-profiling.md b/docs/en/operations/allocation-profiling.md index 64b4106a7e16..574e1ae2ff3c 100644 --- a/docs/en/operations/allocation-profiling.md +++ b/docs/en/operations/allocation-profiling.md @@ -59,10 +59,10 @@ For that, we need to use `jemalloc`'s tool called [jeprof](https://github.com/je If that’s the case, we recommend installing an [alternative implementation](https://github.com/gimli-rs/addr2line) of the tool. ``` -git clone https://github.com/gimli-rs/addr2line +git clone https://github.com/gimli-rs/addr2line.git --depth=1 --branch=0.23.0 cd addr2line -cargo b --examples -r -cp ./target/release/examples/addr2line path/to/current/addr2line +cargo build --features bin --release +cp ./target/release/addr2line path/to/current/addr2line ``` ::: From f2a162a4a784af00fc788084eab8bf6763f06f73 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 18 Jun 2024 11:09:13 +0800 Subject: [PATCH 214/254] fix style error --- tests/integration/test_table_db_num_limit/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_table_db_num_limit/test.py b/tests/integration/test_table_db_num_limit/test.py index f2080ec4738a..aa8030b077cc 100644 --- a/tests/integration/test_table_db_num_limit/test.py +++ b/tests/integration/test_table_db_num_limit/test.py @@ -32,7 +32,7 @@ def test_table_db_limit(started_cluster): for i in range(10): node1.query("create table t{} (a Int32) Engine = Log".format(i)) - node1.query("system flush logs"); + node1.query("system flush logs") for i in range(10): node1.query("drop table t{}".format(i)) for i in range(10): From d8379bfba2aeac91f3e578c182daddee9ea70353 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 18 Jun 2024 11:48:04 +0800 Subject: [PATCH 215/254] fix the error system flush logs hang --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index c33f97dc80db..9230cba2a29c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1577,7 +1577,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, } UInt64 table_num_limit = getContext()->getGlobalContext()->getServerSettings().max_table_num_to_throw; - if (table_num_limit > 0) + if (table_num_limit > 0 && create.getDatabase() != DatabaseCatalog::SYSTEM_DATABASE) { UInt64 table_count = CurrentMetrics::get(CurrentMetrics::AttachedTable); if (table_count >= table_num_limit) From 966a32a4d4b2842633b8ae27c1b95984f5259b0b Mon Sep 17 00:00:00 2001 From: Julian Maicher Date: Tue, 11 Jun 2024 18:36:20 +0200 Subject: [PATCH 216/254] Fix HTTP exception handling in case of receive timeouts Closes #65117 --- src/Server/HTTPHandler.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index f6ca69813aea..bbb596d0a50f 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -1046,12 +1046,21 @@ void HTTPHandler::formatExceptionForClient(int exception_code, HTTPServerRequest /// FIXME: make sure that no one else is reading from the same stream at the moment. - /// If HTTP method is POST and Keep-Alive is turned on, we should read the whole request body + /// If HTTP method is POST and Keep-Alive is turned on, we should try to read the whole request body /// to avoid reading part of the current request body in the next request. if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive() - && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED && !request.getStream().eof()) + && exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED) { - request.getStream().ignoreAll(); + try + { + if (!request.getStream().eof()) + request.getStream().ignoreAll(); + } + catch (...) + { + tryLogCurrentException(log, "Cannot read remaining request body during exception handling"); + response.setKeepAlive(false); + } } if (exception_code == ErrorCodes::REQUIRED_PASSWORD) From c844724e4ce9b344ae3de7d2c5e99cf9b2c6fe07 Mon Sep 17 00:00:00 2001 From: Julian Maicher Date: Wed, 12 Jun 2024 11:18:32 +0200 Subject: [PATCH 217/254] Fix test checking max_chunk_size The `boundary` parameter is required, and if it's missing, we're running into a different exception: `Poco::Exception. Code: 1000, Not found: boundary`. With the `boundary` parameter, the test correctly yields `Code: 69. DB::Exception: Chunk size exceeded the limit (max size: 107374182400)`. --- tests/queries/0_stateless/02403_big_http_chunk_size.python | 2 +- tests/queries/0_stateless/02403_big_http_chunk_size.reference | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02403_big_http_chunk_size.python b/tests/queries/0_stateless/02403_big_http_chunk_size.python index 3213b8cd387e..f74459489a5d 100644 --- a/tests/queries/0_stateless/02403_big_http_chunk_size.python +++ b/tests/queries/0_stateless/02403_big_http_chunk_size.python @@ -16,7 +16,7 @@ def main(): sock.settimeout(60) s = "POST / HTTP/1.1\r\n" s += "Host: %s\r\n" % host - s += "Content-type: multipart/form-data\r\n" + s += "Content-type: multipart/form-data; boundary=--b3f1zid8kqwy\r\n" s += "Transfer-encoding: chunked\r\n" s += "\r\n" s += "ffffffffffffffff" diff --git a/tests/queries/0_stateless/02403_big_http_chunk_size.reference b/tests/queries/0_stateless/02403_big_http_chunk_size.reference index d7970bd2eb1c..466ff9002e95 100644 --- a/tests/queries/0_stateless/02403_big_http_chunk_size.reference +++ b/tests/queries/0_stateless/02403_big_http_chunk_size.reference @@ -1,3 +1,3 @@ -HTTP/1.1 200 OK +HTTP/1.1 500 Internal Server Error encoding type chunked -error code 1000 +error code 69 From a12ebf05b8d8ce3ff1fa5fe913c4ff26d0ff2bf2 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 14 Jun 2024 15:12:31 +0800 Subject: [PATCH 218/254] add function edit distance utf8 --- .../functions/string-functions.md | 26 ++++ src/Functions/FunctionsStringDistance.cpp | 120 +++++++++++------- .../02884_string_distance_function.reference | 31 +++++ .../02884_string_distance_function.sql | 3 +- .../aspell-ignore/en/aspell-dict.txt | 2 + 5 files changed, 135 insertions(+), 47 deletions(-) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index c535b82d710a..a258456345eb 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -2178,6 +2178,32 @@ Result: Alias: levenshteinDistance +## editDistanceUTF8 + +Calculates the [edit distance](https://en.wikipedia.org/wiki/Edit_distance) between two UTF8 strings. + +**Syntax** + +```sql +editDistanceUTF8(string1, string2) +``` + +**Examples** + +``` sql +SELECT editDistanceUTF8('我是谁', '我是我'); +``` + +Result: + +``` text +┌─editDistanceUTF8('我是谁', '我是我')──┐ +│ 1 │ +└─────────────────────────────────────┘ +``` + +Alias: levenshteinDistanceUTF8 + ## damerauLevenshteinDistance Calculates the [Damerau-Levenshtein distance](https://en.wikipedia.org/wiki/Damerau%E2%80%93Levenshtein_distance) between two byte strings. diff --git a/src/Functions/FunctionsStringDistance.cpp b/src/Functions/FunctionsStringDistance.cpp index 6cb23bbea9f8..48f4aaf4e095 100644 --- a/src/Functions/FunctionsStringDistance.cpp +++ b/src/Functions/FunctionsStringDistance.cpp @@ -113,6 +113,36 @@ struct ByteHammingDistanceImpl } }; +void parseUTF8String(const char * __restrict data, size_t size, std::function utf8_consumer, std::function ascii_consumer = nullptr) +{ + const char * end = data + size; + while (data < end) + { + size_t len = UTF8::seqLength(*data); + if (len == 1) + { + if (ascii_consumer) + ascii_consumer(static_cast(*data)); + else + utf8_consumer(static_cast(*data)); + ++data; + } + else + { + auto code_point = UTF8::convertUTF8ToCodePoint(data, end - data); + if (code_point.has_value()) + { + utf8_consumer(code_point.value()); + data += len; + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", StringRef(data, end - data)); + } + } + } +} + template struct ByteJaccardIndexImpl { @@ -138,57 +168,28 @@ struct ByteJaccardIndexImpl haystack_set.fill(0); needle_set.fill(0); - while (haystack < haystack_end) + if constexpr (is_utf8) { - size_t len = 1; - if constexpr (is_utf8) - len = UTF8::seqLength(*haystack); - - if (len == 1) + parseUTF8String( + haystack, + haystack_size, + [&](UInt32 data) { haystack_utf8_set.insert(data); }, + [&](unsigned char data) { haystack_set[data] = 1; }); + parseUTF8String( + needle, needle_size, [&](UInt32 data) { needle_utf8_set.insert(data); }, [&](unsigned char data) { needle_set[data] = 1; }); + } + else + { + while (haystack < haystack_end) { haystack_set[static_cast(*haystack)] = 1; ++haystack; } - else - { - auto code_point = UTF8::convertUTF8ToCodePoint(haystack, haystack_end - haystack); - if (code_point.has_value()) - { - haystack_utf8_set.insert(code_point.value()); - haystack += len; - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", StringRef(haystack, haystack_end - haystack)); - } - } - } - - while (needle < needle_end) - { - - size_t len = 1; - if constexpr (is_utf8) - len = UTF8::seqLength(*needle); - - if (len == 1) + while (needle < needle_end) { needle_set[static_cast(*needle)] = 1; ++needle; } - else - { - auto code_point = UTF8::convertUTF8ToCodePoint(needle, needle_end - needle); - if (code_point.has_value()) - { - needle_utf8_set.insert(code_point.value()); - needle += len; - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", StringRef(needle, needle_end - needle)); - } - } } UInt8 intersection = 0; @@ -226,6 +227,7 @@ struct ByteJaccardIndexImpl static constexpr size_t max_string_size = 1u << 16; +template struct ByteEditDistanceImpl { using ResultType = UInt64; @@ -242,6 +244,16 @@ struct ByteEditDistanceImpl ErrorCodes::TOO_LARGE_STRING_SIZE, "The string size is too big for function editDistance, should be at most {}", max_string_size); + PaddedPODArray haystack_utf8; + PaddedPODArray needle_utf8; + if constexpr (is_utf8) + { + parseUTF8String(haystack, haystack_size, [&](UInt32 data) { haystack_utf8.push_back(data); }); + parseUTF8String(needle, needle_size, [&](UInt32 data) { needle_utf8.push_back(data); }); + haystack_size = haystack_utf8.size(); + needle_size = needle_utf8.size(); + } + PaddedPODArray distances0(haystack_size + 1, 0); PaddedPODArray distances1(haystack_size + 1, 0); @@ -261,9 +273,16 @@ struct ByteEditDistanceImpl insertion = distances1[pos_haystack] + 1; substitution = distances0[pos_haystack]; - if (*(needle + pos_needle) != *(haystack + pos_haystack)) - substitution += 1; - + if constexpr (is_utf8) + { + if (needle_utf8[pos_needle] != haystack_utf8[pos_haystack]) + substitution += 1; + } + else + { + if (*(needle + pos_needle) != *(haystack + pos_haystack)) + substitution += 1; + } distances1[pos_haystack + 1] = std::min(deletion, std::min(substitution, insertion)); } distances0.swap(distances1); @@ -457,7 +476,12 @@ struct NameEditDistance { static constexpr auto name = "editDistance"; }; -using FunctionEditDistance = FunctionsStringSimilarity, NameEditDistance>; +using FunctionEditDistance = FunctionsStringSimilarity>, NameEditDistance>; +struct NameEditDistanceUTF8 +{ + static constexpr auto name = "editDistanceUTF8"; +}; +using FunctionEditDistanceUTF8 = FunctionsStringSimilarity>, NameEditDistanceUTF8>; struct NameDamerauLevenshteinDistance { @@ -499,6 +523,10 @@ REGISTER_FUNCTION(StringDistance) FunctionDocumentation{.description = R"(Calculates the edit distance between two byte-strings.)"}); factory.registerAlias("levenshteinDistance", NameEditDistance::name); + factory.registerFunction( + FunctionDocumentation{.description = R"(Calculates the edit distance between two UTF8 strings.)"}); + factory.registerAlias("levenshteinDistanceUTF8", NameEditDistanceUTF8::name); + factory.registerFunction( FunctionDocumentation{.description = R"(Calculates the Damerau-Levenshtein distance two between two byte-string.)"}); diff --git a/tests/queries/0_stateless/02884_string_distance_function.reference b/tests/queries/0_stateless/02884_string_distance_function.reference index e8cce2017d93..71b15bc87539 100644 --- a/tests/queries/0_stateless/02884_string_distance_function.reference +++ b/tests/queries/0_stateless/02884_string_distance_function.reference @@ -13,53 +13,84 @@ clickhouse mouse 6 -- non-const arguments byteHammingDistance 0 byteHammingDistance abc 3 +byteHammingDistance Jerry 我是谁 9 byteHammingDistance abc 3 byteHammingDistance abc ab 1 byteHammingDistance abc abc 0 byteHammingDistance abc bc 3 byteHammingDistance clickhouse mouse 10 +byteHammingDistance 我是谁 Tom 9 +byteHammingDistance 我是谁 我是我 3 editDistance 0 editDistance abc 3 +editDistance Jerry 我是谁 9 editDistance abc 3 editDistance abc ab 1 editDistance abc abc 0 editDistance abc bc 1 editDistance clickhouse mouse 6 +editDistance 我是谁 Tom 9 +editDistance 我是谁 我是我 3 +editDistanceUTF8 0 +editDistanceUTF8 abc 3 +editDistanceUTF8 Jerry 我是谁 5 +editDistanceUTF8 abc 3 +editDistanceUTF8 abc ab 1 +editDistanceUTF8 abc abc 0 +editDistanceUTF8 abc bc 1 +editDistanceUTF8 clickhouse mouse 6 +editDistanceUTF8 我是谁 Tom 3 +editDistanceUTF8 我是谁 我是我 1 damerauLevenshteinDistance 0 damerauLevenshteinDistance abc 3 +damerauLevenshteinDistance Jerry 我是谁 9 damerauLevenshteinDistance abc 3 damerauLevenshteinDistance abc ab 1 damerauLevenshteinDistance abc abc 0 damerauLevenshteinDistance abc bc 1 damerauLevenshteinDistance clickhouse mouse 6 +damerauLevenshteinDistance 我是谁 Tom 9 +damerauLevenshteinDistance 我是谁 我是我 3 stringJaccardIndex 0 stringJaccardIndex abc 0 +stringJaccardIndex Jerry 我是谁 0 stringJaccardIndex abc 0 stringJaccardIndex abc ab 0.6666666666666666 stringJaccardIndex abc abc 1 stringJaccardIndex abc bc 0.6666666666666666 stringJaccardIndex clickhouse mouse 0.4 +stringJaccardIndex 我是谁 Tom 0 +stringJaccardIndex 我是谁 我是我 0.625 stringJaccardIndexUTF8 0 stringJaccardIndexUTF8 abc 0 +stringJaccardIndexUTF8 Jerry 我是谁 0 stringJaccardIndexUTF8 abc 0 stringJaccardIndexUTF8 abc ab 0.6666666666666666 stringJaccardIndexUTF8 abc abc 1 stringJaccardIndexUTF8 abc bc 0.6666666666666666 stringJaccardIndexUTF8 clickhouse mouse 0.4 +stringJaccardIndexUTF8 我是谁 Tom 0 +stringJaccardIndexUTF8 我是谁 我是我 0.6666666666666666 jaroSimilarity 0 jaroSimilarity abc 3 +jaroSimilarity Jerry 我是谁 0 jaroSimilarity abc 3 jaroSimilarity abc ab 0.8888888888888888 jaroSimilarity abc abc 1 jaroSimilarity abc bc 0 jaroSimilarity clickhouse mouse 0 +jaroSimilarity 我是谁 Tom 0 +jaroSimilarity 我是谁 我是我 0.7777777777777777 jaroWinklerSimilarity 0 jaroWinklerSimilarity abc 3 +jaroWinklerSimilarity Jerry 我是谁 0 jaroWinklerSimilarity abc 3 jaroWinklerSimilarity abc ab 0.9111111111111111 jaroWinklerSimilarity abc abc 1 jaroWinklerSimilarity abc bc 0 jaroWinklerSimilarity clickhouse mouse 0 +jaroWinklerSimilarity 我是谁 Tom 0 +jaroWinklerSimilarity 我是谁 我是我 0.8666666666666666 -- Special UTF-8 tests 0.4 0 diff --git a/tests/queries/0_stateless/02884_string_distance_function.sql b/tests/queries/0_stateless/02884_string_distance_function.sql index fddbf41f0e56..482996e1448b 100644 --- a/tests/queries/0_stateless/02884_string_distance_function.sql +++ b/tests/queries/0_stateless/02884_string_distance_function.sql @@ -26,11 +26,12 @@ CREATE TABLE t ) ENGINE = MergeTree ORDER BY s1; -- actual test cases -INSERT INTO t VALUES ('', '') ('abc', '') ('', 'abc') ('abc', 'abc') ('abc', 'ab') ('abc', 'bc') ('clickhouse', 'mouse'); +INSERT INTO t VALUES ('', '') ('abc', '') ('', 'abc') ('abc', 'abc') ('abc', 'ab') ('abc', 'bc') ('clickhouse', 'mouse') ('我是谁', 'Tom') ('Jerry', '我是谁') ('我是谁', '我是我'); SELECT '-- non-const arguments'; SELECT 'byteHammingDistance', s1, s2, byteHammingDistance(s1, s2) FROM t ORDER BY ALL; SELECT 'editDistance', s1, s2, editDistance(s1, s2) FROM t ORDER BY ALL; +SELECT 'editDistanceUTF8', s1, s2, editDistanceUTF8(s1, s2) FROM t ORDER BY ALL; SELECT 'damerauLevenshteinDistance', s1, s2, damerauLevenshteinDistance(s1, s2) FROM t ORDER BY ALL; SELECT 'stringJaccardIndex', s1, s2, stringJaccardIndex(s1, s2) FROM t ORDER BY ALL; SELECT 'stringJaccardIndexUTF8', s1, s2, stringJaccardIndexUTF8(s1, s2) FROM t ORDER BY ALL; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 64ff3e8e2cb8..c8fc67545029 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1541,6 +1541,7 @@ dumpColumnStructure durations ecto editDistance +editDistanceUTF embeddings emptyArray emptyArrayDate @@ -1898,6 +1899,7 @@ lessOrEquals lessorequals levenshtein levenshteinDistance +levenshteinDistanceUTF lexicographically lgamma libFuzzer From 72e3fdc8cae727e925d0628c5eb5e1f25f9bf578 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 18 Jun 2024 09:53:37 +0200 Subject: [PATCH 219/254] Use test database --- .../queries/0_stateless/03173_forbid_qualify.sql | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/03173_forbid_qualify.sql b/tests/queries/0_stateless/03173_forbid_qualify.sql index 59f0153cd366..d8cb2bad2eaa 100644 --- a/tests/queries/0_stateless/03173_forbid_qualify.sql +++ b/tests/queries/0_stateless/03173_forbid_qualify.sql @@ -1,11 +1,11 @@ -drop table if exists default.test_qualify; -create table default.test_qualify (number Int64) ENGINE = MergeTree ORDER BY (number); +drop table if exists test_qualify; +create table test_qualify (number Int64) ENGINE = MergeTree ORDER BY (number); -insert into default.test_qualify SELECT * FROM numbers(100); +insert into test_qualify SELECT * FROM numbers(100); -select count() from default.test_qualify; -- 100 -select * from default.test_qualify qualify row_number() over (order by number) = 50 SETTINGS allow_experimental_analyzer = 1; -- 49 -select * from default.test_qualify qualify row_number() over (order by number) = 50 SETTINGS allow_experimental_analyzer = 0; -- { serverError NOT_IMPLEMENTED } +select count() from test_qualify; -- 100 +select * from test_qualify qualify row_number() over (order by number) = 50 SETTINGS allow_experimental_analyzer = 1; -- 49 +select * from test_qualify qualify row_number() over (order by number) = 50 SETTINGS allow_experimental_analyzer = 0; -- { serverError NOT_IMPLEMENTED } -delete from default.test_qualify where number in (select number from default.test_qualify qualify row_number() over (order by number) = 50); -- { serverError UNFINISHED } -select count() from default.test_qualify; -- 100 +delete from test_qualify where number in (select number from test_qualify qualify row_number() over (order by number) = 50); -- { serverError UNFINISHED } +select count() from test_qualify; -- 100 From 5701e3e48bb00d9b3f13ee064fe6401ac1e22833 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 18 Jun 2024 10:43:41 +0200 Subject: [PATCH 220/254] Fix test --- tests/integration/test_keeper_profiler/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_profiler/test.py b/tests/integration/test_keeper_profiler/test.py index 98738890ad84..69aaaeb61c8a 100644 --- a/tests/integration/test_keeper_profiler/test.py +++ b/tests/integration/test_keeper_profiler/test.py @@ -59,7 +59,7 @@ def test_profiler(started_cluster): arrayStringConcat( arrayMap(x, y -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace -from system.trace_log where trace_type = ‘Real’ and trace ilike '%KeeperTCPHandler%' group by trace); +from system.trace_log where trace_type = ‘Real’ and (trace ilike '%KeeperTCPHandler%' or trace ilike '%KeeperDispatcher%') group by trace); """ ) From 446e28d51b124d6652b5502ba3728668a2f8dde9 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 18 Jun 2024 09:29:52 +0000 Subject: [PATCH 221/254] fix test --- tests/integration/test_checking_s3_blobs_paranoid/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 476f7c61b28d..2471c93458b3 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -300,7 +300,7 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): LIMIT 1000000 SETTINGS s3_max_single_part_upload_size=100, - s3_min_upload_part_size=1000000, + s3_min_upload_part_size=100000, s3_check_objects_after_upload=0 """, query_id=insert_query_id, @@ -311,7 +311,7 @@ def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): ) assert create_multipart == 1 - assert upload_parts == 7 + assert upload_parts == 69 assert s3_errors == 3 broken_s3.setup_at_part_upload( From aeee0b2c180ad13045a46aab41f89088ae734d1d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 18 Jun 2024 09:33:32 +0000 Subject: [PATCH 222/254] Use ActionsDAG ref in PlannerActionsVisitor --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Planner/PlannerActionsVisitor.cpp | 35 +++++++++++-------------- 2 files changed, 17 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index ff65475d9af3..c28b49f4eb86 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -824,7 +824,7 @@ void ExpressionAnalyzer::makeWindowDescriptions(ActionsDAG & actions) desc, &definition); auto [it, inserted] = window_descriptions.insert( - {default_window_name, std::move(desc)}); + {default_window_name, desc}); if (!inserted) { diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index f5e71acee387..7a12d5d690d2 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -413,11 +413,11 @@ class ActionNodeNameHelper class ActionsScopeNode { public: - explicit ActionsScopeNode(ActionsDAGPtr actions_dag_, QueryTreeNodePtr scope_node_) - : actions_dag(std::move(actions_dag_)) + explicit ActionsScopeNode(ActionsDAG & actions_dag_, QueryTreeNodePtr scope_node_) + : actions_dag(actions_dag_) , scope_node(std::move(scope_node_)) { - for (const auto & node : actions_dag->getNodes()) + for (const auto & node : actions_dag.getNodes()) node_name_to_node[node.result_name] = &node; } @@ -456,7 +456,7 @@ class ActionsScopeNode throw Exception(ErrorCodes::LOGICAL_ERROR, "No node with name {}. There are only nodes {}", node_name, - actions_dag->dumpNames()); + actions_dag.dumpNames()); return it->second; } @@ -467,7 +467,7 @@ class ActionsScopeNode if (it != node_name_to_node.end()) return it->second; - const auto * node = &actions_dag->addInput(node_name, column_type); + const auto * node = &actions_dag.addInput(node_name, column_type); node_name_to_node[node->result_name] = node; return node; @@ -479,7 +479,7 @@ class ActionsScopeNode if (it != node_name_to_node.end()) return it->second; - const auto * node = &actions_dag->addInput(column); + const auto * node = &actions_dag.addInput(column); node_name_to_node[node->result_name] = node; return node; @@ -491,7 +491,7 @@ class ActionsScopeNode if (it != node_name_to_node.end()) return it->second; - const auto * node = &actions_dag->addColumn(column); + const auto * node = &actions_dag.addColumn(column); node_name_to_node[node->result_name] = node; return node; @@ -504,7 +504,7 @@ class ActionsScopeNode if (it != node_name_to_node.end()) return it->second; - const auto * node = &actions_dag->addFunction(function, children, node_name); + const auto * node = &actions_dag.addFunction(function, children, node_name); node_name_to_node[node->result_name] = node; return node; @@ -516,7 +516,7 @@ class ActionsScopeNode if (it != node_name_to_node.end()) return it->second; - const auto * node = &actions_dag->addArrayJoin(*child, node_name); + const auto * node = &actions_dag.addArrayJoin(*child, node_name); node_name_to_node[node->result_name] = node; return node; @@ -524,14 +524,14 @@ class ActionsScopeNode private: std::unordered_map node_name_to_node; - ActionsDAGPtr actions_dag; + ActionsDAG & actions_dag; QueryTreeNodePtr scope_node; }; class PlannerActionsVisitorImpl { public: - PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag, + PlannerActionsVisitorImpl(ActionsDAG & actions_dag, const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_); @@ -595,14 +595,14 @@ class PlannerActionsVisitorImpl bool use_column_identifier_as_action_node_name; }; -PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAGPtr actions_dag, +PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAG & actions_dag, const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_) : planner_context(planner_context_) , action_node_name_helper(node_to_node_name, *planner_context, use_column_identifier_as_action_node_name_) , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) { - actions_stack.emplace_back(std::move(actions_dag), nullptr); + actions_stack.emplace_back(actions_dag, nullptr); } ActionsDAG::NodeRawConstPtrs PlannerActionsVisitorImpl::visit(QueryTreeNodePtr expression_node) @@ -758,7 +758,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi } auto lambda_actions_dag = std::make_shared(); - actions_stack.emplace_back(lambda_actions_dag, node); + actions_stack.emplace_back(*lambda_actions_dag, node); auto [lambda_expression_node_name, levels] = visitImpl(lambda_node.getExpression()); lambda_actions_dag->getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name)); @@ -1015,11 +1015,8 @@ PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_c ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAG & actions_dag, QueryTreeNodePtr expression_node) { - auto ptr = std::make_shared(std::move(actions_dag)); - PlannerActionsVisitorImpl actions_visitor_impl(ptr, planner_context, use_column_identifier_as_action_node_name); - auto res = actions_visitor_impl.visit(expression_node); - actions_dag = std::move(*ptr); - return res; + PlannerActionsVisitorImpl actions_visitor_impl(actions_dag, planner_context, use_column_identifier_as_action_node_name); + return actions_visitor_impl.visit(expression_node); } String calculateActionNodeName(const QueryTreeNodePtr & node, From 153268a29f011e9beacebafca0bc200c2eb6d933 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 18 Jun 2024 10:21:57 +0000 Subject: [PATCH 223/254] Renames. --- src/Interpreters/ActionsDAG.h | 7 +- src/Interpreters/ExpressionActions.cpp | 6 +- src/Interpreters/ExpressionActions.h | 33 +++-- src/Interpreters/ExpressionAnalyzer.cpp | 122 +++++++++--------- src/Interpreters/ExpressionAnalyzer.h | 32 ++--- src/Interpreters/InterpreterSelectQuery.cpp | 40 +++--- src/Interpreters/InterpreterSelectQuery.h | 12 +- src/Interpreters/MutationsInterpreter.cpp | 14 +- src/Planner/ActionsChain.cpp | 18 +-- src/Planner/ActionsChain.h | 8 +- src/Planner/Planner.cpp | 8 +- src/Planner/PlannerExpressionAnalysis.cpp | 82 ++++++------ src/Planner/PlannerExpressionAnalysis.h | 14 +- .../optimizeUseAggregateProjection.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 15 files changed, 205 insertions(+), 195 deletions(-) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index d251f66a1296..8709bf955a99 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -507,13 +507,12 @@ struct ActionDAGNodes ActionsDAG::NodeRawConstPtrs nodes; }; -struct ActionsAndFlags +struct ActionsAndProjectInputsFlag { - ActionsDAG actions; + ActionsDAG dag; bool project_input = false; - bool projected_output = false; }; -using ActionsAndFlagsPtr = std::shared_ptr; +using ActionsAndProjectInputsFlagPtr = std::shared_ptr; } diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index e19c4b0cd20a..7f96c927d82c 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -980,7 +980,7 @@ void ExpressionActionsChain::addStep(NameSet non_constant_inputs) if (column.column && isColumnConst(*column.column) && non_constant_inputs.contains(column.name)) column.column = nullptr; - steps.push_back(std::make_unique(std::make_shared(ActionsDAG(columns), false, false))); + steps.push_back(std::make_unique(std::make_shared(ActionsDAG(columns), false))); } void ExpressionActionsChain::finalize() @@ -1129,12 +1129,12 @@ void ExpressionActionsChain::JoinStep::finalize(const NameSet & required_output_ std::swap(result_columns, new_result_columns); } -ActionsAndFlagsPtr & ExpressionActionsChain::Step::actions() +ActionsAndProjectInputsFlagPtr & ExpressionActionsChain::Step::actions() { return typeid_cast(*this).actions_and_flags; } -const ActionsAndFlagsPtr & ExpressionActionsChain::Step::actions() const +const ActionsAndProjectInputsFlagPtr & ExpressionActionsChain::Step::actions() const { return typeid_cast(*this).actions_and_flags; } diff --git a/src/Interpreters/ExpressionActions.h b/src/Interpreters/ExpressionActions.h index c762abc1149b..ddffe0222155 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -179,15 +179,16 @@ struct ExpressionActionsChain : WithContext virtual std::string dump() const = 0; /// Only for ExpressionActionsStep - ActionsAndFlagsPtr & actions(); - const ActionsAndFlagsPtr & actions() const; + ActionsAndProjectInputsFlagPtr & actions(); + const ActionsAndProjectInputsFlagPtr & actions() const; }; struct ExpressionActionsStep : public Step { - ActionsAndFlagsPtr actions_and_flags; + ActionsAndProjectInputsFlagPtr actions_and_flags; + bool is_final_projection = false; - explicit ExpressionActionsStep(ActionsAndFlagsPtr actiactions_and_flags_, Names required_output_ = Names()) + explicit ExpressionActionsStep(ActionsAndProjectInputsFlagPtr actiactions_and_flags_, Names required_output_ = Names()) : Step(std::move(required_output_)) , actions_and_flags(std::move(actiactions_and_flags_)) { @@ -195,18 +196,18 @@ struct ExpressionActionsChain : WithContext NamesAndTypesList getRequiredColumns() const override { - return actions_and_flags->actions.getRequiredColumns(); + return actions_and_flags->dag.getRequiredColumns(); } ColumnsWithTypeAndName getResultColumns() const override { - return actions_and_flags->actions.getResultColumns(); + return actions_and_flags->dag.getResultColumns(); } void finalize(const NameSet & required_output_) override { - if (!actions_and_flags->projected_output) - actions_and_flags->actions.removeUnusedActions(required_output_); + if (!is_final_projection) + actions_and_flags->dag.removeUnusedActions(required_output_); } void prependProjectInput() override @@ -216,7 +217,7 @@ struct ExpressionActionsChain : WithContext std::string dump() const override { - return actions_and_flags->actions.dumpDAG(); + return actions_and_flags->dag.dumpDAG(); } }; @@ -265,7 +266,7 @@ struct ExpressionActionsChain : WithContext steps.clear(); } - ActionsAndFlagsPtr getLastActions(bool allow_empty = false) + ExpressionActionsStep * getLastExpressionStep(bool allow_empty = false) { if (steps.empty()) { @@ -274,7 +275,15 @@ struct ExpressionActionsChain : WithContext throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty ExpressionActionsChain"); } - return typeid_cast(steps.back().get())->actions_and_flags; + return typeid_cast(steps.back().get()); + } + + ActionsAndProjectInputsFlagPtr getLastActions(bool allow_empty = false) + { + if (auto * step = getLastExpressionStep(allow_empty)) + return step->actions_and_flags; + + return nullptr; } Step & getLastStep() @@ -294,7 +303,7 @@ struct ExpressionActionsChain : WithContext Step & addStep(const NamesAndTypesList & columns) { - return *steps.emplace_back(std::make_unique(std::make_shared(ActionsDAG(columns), false, false))); + return *steps.emplace_back(std::make_unique(std::make_shared(ActionsDAG(columns), false))); } std::string dumpChain() const; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index c28b49f4eb86..62cddd9caf7c 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -891,7 +891,7 @@ ArrayJoinActionPtr ExpressionAnalyzer::addMultipleArrayJoinAction(ActionsDAG & a return std::make_shared(result_columns, array_join_is_left, getContext()); } -ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, ActionsAndFlagsPtr & before_array_join, bool only_types) +ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, ActionsAndProjectInputsFlagPtr & before_array_join, bool only_types) { const auto * select_query = getSelectQuery(); @@ -901,9 +901,9 @@ ArrayJoinActionPtr SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActi ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns()); - getRootActions(array_join_expression_list, only_types, step.actions()->actions); + getRootActions(array_join_expression_list, only_types, step.actions()->dag); - auto array_join = addMultipleArrayJoinAction(step.actions()->actions, is_array_join_left); + auto array_join = addMultipleArrayJoinAction(step.actions()->dag, is_array_join_left); before_array_join = chain.getLastActions(); chain.steps.push_back(std::make_unique(array_join, step.getResultColumns())); @@ -917,13 +917,13 @@ bool SelectQueryExpressionAnalyzer::appendJoinLeftKeys(ExpressionActionsChain & { ExpressionActionsChain::Step & step = chain.lastStep(columns_after_array_join); - getRootActions(analyzedJoin().leftKeysList(), only_types, step.actions()->actions); + getRootActions(analyzedJoin().leftKeysList(), only_types, step.actions()->dag); return true; } JoinPtr SelectQueryExpressionAnalyzer::appendJoin( ExpressionActionsChain & chain, - ActionsAndFlagsPtr & converting_join_columns) + ActionsAndProjectInputsFlagPtr & converting_join_columns) { const ColumnsWithTypeAndName & left_sample_columns = chain.getLastStep().getResultColumns(); @@ -932,8 +932,8 @@ JoinPtr SelectQueryExpressionAnalyzer::appendJoin( if (converting_actions) { - converting_join_columns = std::make_shared(); - converting_join_columns->actions = std::move(*converting_actions); + converting_join_columns = std::make_shared(); + converting_join_columns->dag = std::move(*converting_actions); chain.steps.push_back(std::make_unique(converting_join_columns)); chain.addStep(); } @@ -1170,7 +1170,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeJoin( return join; } -ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendPrewhere( +ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendPrewhere( ExpressionActionsChain & chain, bool only_types) { const auto * select_query = getSelectQuery(); @@ -1182,17 +1182,17 @@ ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendPrewhere( first_action_names = chain.steps.front()->getRequiredColumns().getNames(); auto & step = chain.lastStep(sourceColumns()); - getRootActions(select_query->prewhere(), only_types, step.actions()->actions); + getRootActions(select_query->prewhere(), only_types, step.actions()->dag); String prewhere_column_name = select_query->prewhere()->getColumnName(); step.addRequiredOutput(prewhere_column_name); - const auto & node = step.actions()->actions.findInOutputs(prewhere_column_name); + const auto & node = step.actions()->dag.findInOutputs(prewhere_column_name); auto filter_type = node.result_type; if (!filter_type->canBeUsedInBooleanContext()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in PREWHERE: {}", filter_type->getName()); - ActionsAndFlagsPtr prewhere_actions; + ActionsAndProjectInputsFlagPtr prewhere_actions; { /// Remove unused source_columns from prewhere actions. ActionsDAG tmp_actions_dag(sourceColumns()); @@ -1208,7 +1208,7 @@ ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendPrewhere( NameSet required_source_columns(required_columns.begin(), required_columns.end()); required_source_columns.insert(first_action_names.begin(), first_action_names.end()); - auto names = step.actions()->actions.getNames(); + auto names = step.actions()->dag.getNames(); NameSet name_set(names.begin(), names.end()); for (const auto & column : sourceColumns()) @@ -1217,13 +1217,13 @@ ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendPrewhere( Names required_output(name_set.begin(), name_set.end()); prewhere_actions = chain.getLastActions(); - prewhere_actions->actions.removeUnusedActions(required_output); + prewhere_actions->dag.removeUnusedActions(required_output); } { - auto actions = std::make_shared(); + auto actions = std::make_shared(); - auto required_columns = prewhere_actions->actions.getRequiredColumns(); + auto required_columns = prewhere_actions->dag.getRequiredColumns(); NameSet prewhere_input_names; for (const auto & col : required_columns) prewhere_input_names.insert(col.name); @@ -1267,11 +1267,11 @@ ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendPrewhere( } } - actions->actions = ActionsDAG(required_columns); + actions->dag = ActionsDAG(required_columns); } else { - ColumnsWithTypeAndName columns = prewhere_actions->actions.getResultColumns(); + ColumnsWithTypeAndName columns = prewhere_actions->dag.getResultColumns(); for (const auto & column : sourceColumns()) { @@ -1282,7 +1282,7 @@ ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendPrewhere( } } - actions->actions = ActionsDAG(columns); + actions->dag = ActionsDAG(columns); } chain.steps.emplace_back( @@ -1304,12 +1304,12 @@ bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(columns_after_join); - getRootActions(select_query->where(), only_types, step.actions()->actions); + getRootActions(select_query->where(), only_types, step.actions()->dag); auto where_column_name = select_query->where()->getColumnName(); step.addRequiredOutput(where_column_name); - const auto & node = step.actions()->actions.findInOutputs(where_column_name); + const auto & node = step.actions()->dag.findInOutputs(where_column_name); auto filter_type = node.result_type; if (!filter_type->canBeUsedInBooleanContext()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER, "Invalid type for filter in WHERE: {}", @@ -1336,7 +1336,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain for (const auto & ast_element : ast->children) { step.addRequiredOutput(ast_element->getColumnName()); - getRootActions(ast_element, only_types, step.actions()->actions); + getRootActions(ast_element, only_types, step.actions()->dag); } } } @@ -1345,7 +1345,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain for (const auto & ast : asts) { step.addRequiredOutput(ast->getColumnName()); - getRootActions(ast, only_types, step.actions()->actions); + getRootActions(ast, only_types, step.actions()->dag); } } @@ -1391,7 +1391,7 @@ void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(Expression const ASTFunction & node = typeid_cast(*ast); if (node.arguments) for (auto & argument : node.arguments->children) - getRootActions(argument, only_types, step.actions()->actions); + getRootActions(argument, only_types, step.actions()->dag); } } @@ -1413,7 +1413,7 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments( // recursively together with (1b) as ASTFunction::window_definition. if (getSelectQuery()->window()) { - getRootActionsNoMakeSet(getSelectQuery()->window(), step.actions()->actions); + getRootActionsNoMakeSet(getSelectQuery()->window(), step.actions()->dag); } for (const auto & [_, w] : window_descriptions) @@ -1424,7 +1424,7 @@ void SelectQueryExpressionAnalyzer::appendWindowFunctionsArguments( // definitions (1a). // Requiring a constant reference to a shared pointer to non-const AST // doesn't really look sane, but the visitor does indeed require it. - getRootActionsNoMakeSet(f.function_node->clone(), step.actions()->actions); + getRootActionsNoMakeSet(f.function_node->clone(), step.actions()->dag); // (2b) Required function argument columns. for (const auto & a : f.function_node->arguments->children) @@ -1446,7 +1446,7 @@ void SelectQueryExpressionAnalyzer::appendExpressionsAfterWindowFunctions(Expres ExpressionActionsChain::Step & step = chain.lastStep(columns_after_window); for (const auto & expression : syntax->expressions_with_window_function) - getRootActionsForWindowFunctions(expression->clone(), true, step.actions()->actions); + getRootActionsForWindowFunctions(expression->clone(), true, step.actions()->dag); } void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAG & before_aggregation, ExpressionActionsChain & chain, bool /* only_types */) @@ -1470,7 +1470,7 @@ void SelectQueryExpressionAnalyzer::appendGroupByModifiers(ActionsDAG & before_a ExpressionActionsChain::Step & step = chain.addStep(before_aggregation.getNamesAndTypesList()); step.required_output = std::move(required_output); - step.actions()->actions = std::move(*ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position)); + step.actions()->dag = std::move(*ActionsDAG::makeConvertingActions(source_columns, result_columns, ActionsDAG::MatchColumnsMode::Position)); } void SelectQueryExpressionAnalyzer::appendSelectSkipWindowExpressions(ExpressionActionsChain::Step & step, ASTPtr const & node) @@ -1501,7 +1501,7 @@ bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); - getRootActionsForHaving(select_query->having(), only_types, step.actions()->actions); + getRootActionsForHaving(select_query->having(), only_types, step.actions()->dag); step.addRequiredOutput(select_query->having()->getColumnName()); @@ -1514,13 +1514,13 @@ void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); - getRootActions(select_query->select(), only_types, step.actions()->actions); + getRootActions(select_query->select(), only_types, step.actions()->dag); for (const auto & child : select_query->select()->children) appendSelectSkipWindowExpressions(step, child); } -ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, +ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions & order_by_elements_actions) { const auto * select_query = getSelectQuery(); @@ -1544,7 +1544,7 @@ ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionAction replaceForPositionalArguments(ast->children.at(0), select_query, ASTSelectQuery::Expression::ORDER_BY); } - getRootActions(select_query->orderBy(), only_types, step.actions()->actions); + getRootActions(select_query->orderBy(), only_types, step.actions()->dag); bool with_fill = false; @@ -1634,7 +1634,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain ExpressionActionsChain::Step & step = chain.lastStep(aggregated_columns); - getRootActions(select_query->limitBy(), only_types, step.actions()->actions); + getRootActions(select_query->limitBy(), only_types, step.actions()->dag); NameSet existing_column_names; for (const auto & column : aggregated_columns) @@ -1663,7 +1663,7 @@ bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain return true; } -ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const +ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const { const auto * select_query = getSelectQuery(); @@ -1711,18 +1711,20 @@ ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendProjectResult(Expression } } - auto actions = chain.getLastActions(); - actions->actions.project(result_columns); + auto * last_step = chain.getLastExpressionStep(); + auto & actions = last_step->actions_and_flags; + actions->dag.project(result_columns); if (!required_result_columns.empty()) { result_columns.clear(); for (const auto & column : required_result_columns) result_columns.emplace_back(column, std::string{}); - actions->actions.project(result_columns); + actions->dag.project(result_columns); } - actions->project_input = actions->projected_output = true; + actions->project_input = true; + last_step->is_final_projection = true; return actions; } @@ -1730,7 +1732,7 @@ ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::appendProjectResult(Expression void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const ASTPtr & expr, bool only_types) { ExpressionActionsChain::Step & step = chain.lastStep(sourceColumns()); - getRootActions(expr, only_types, step.actions()->actions); + getRootActions(expr, only_types, step.actions()->dag); step.addRequiredOutput(expr->getColumnName()); } @@ -1813,7 +1815,7 @@ std::unique_ptr SelectQueryExpressionAnalyzer::getJoinedPlan() return std::move(joined_plan); } -ActionsAndFlagsPtr SelectQueryExpressionAnalyzer::simpleSelectActions() +ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::simpleSelectActions() { ExpressionActionsChain new_chain(getContext()); appendSelect(new_chain, false); @@ -1853,7 +1855,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( ssize_t where_step_num = -1; ssize_t having_step_num = -1; - ActionsAndFlagsPtr prewhere_dag_and_flags; + ActionsAndProjectInputsFlagPtr prewhere_dag_and_flags; auto finalize_chain = [&](ExpressionActionsChain & chain) -> ColumnsWithTypeAndName { @@ -1861,7 +1863,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( { ExpressionActionsChain::Step & step = *chain.steps.at(prewhere_step_num); - auto prewhere_required_columns = prewhere_dag_and_flags->actions.getRequiredColumnsNames(); + auto prewhere_required_columns = prewhere_dag_and_flags->dag.getRequiredColumnsNames(); NameSet required_source_columns(prewhere_required_columns.begin(), prewhere_required_columns.end()); /// Add required columns to required output in order not to remove them after prewhere execution. /// TODO: add sampling and final execution to common chain. @@ -1876,7 +1878,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (prewhere_dag_and_flags) { - auto dag = std::make_shared(std::move(prewhere_dag_and_flags->actions)); + auto dag = std::make_shared(std::move(prewhere_dag_and_flags->dag)); prewhere_info = std::make_shared(std::move(dag), query.prewhere()->getColumnName()); prewhere_dag_and_flags.reset(); } @@ -1936,12 +1938,12 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( /// Prewhere is always the first one. prewhere_step_num = 0; - if (allowEarlyConstantFolding(prewhere_dag_and_flags->actions, settings)) + if (allowEarlyConstantFolding(prewhere_dag_and_flags->dag, settings)) { Block before_prewhere_sample = source_header; if (sanitizeBlock(before_prewhere_sample)) { - auto dag = prewhere_dag_and_flags->actions.clone(); + auto dag = prewhere_dag_and_flags->dag.clone(); ExpressionActions( dag, ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_prewhere_sample); @@ -1967,7 +1969,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( { where_step_num = chain.steps.size() - 1; before_where = chain.getLastActions(); - if (allowEarlyConstantFolding(before_where->actions, settings)) + if (allowEarlyConstantFolding(before_where->dag, settings)) { Block before_where_sample; if (chain.steps.size() > 1) @@ -1977,7 +1979,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (sanitizeBlock(before_where_sample)) { ExpressionActions( - before_where->actions.clone(), + before_where->dag.clone(), ExpressionActionsSettings::fromSettings(context->getSettingsRef())).execute(before_where_sample); auto & column_elem @@ -2003,7 +2005,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( before_aggregation = chain.getLastActions(); if (settings.group_by_use_nulls) - query_analyzer.appendGroupByModifiers(before_aggregation->actions, chain, only_types); + query_analyzer.appendGroupByModifiers(before_aggregation->dag, chain, only_types); auto columns_before_aggregation = finalize_chain(chain); @@ -2050,7 +2052,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( true); auto & step = chain.lastStep(query_analyzer.aggregated_columns); - auto & actions = step.actions()->actions; + auto & actions = step.actions()->dag; actions = std::move(*ActionsDAG::merge(std::move(actions), std::move(*converting))); } } @@ -2087,13 +2089,13 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( // the main SELECT, similar to what we do for aggregate functions. if (has_window) { - query_analyzer.makeWindowDescriptions(chain.getLastActions()->actions); + query_analyzer.makeWindowDescriptions(chain.getLastActions()->dag); query_analyzer.appendWindowFunctionsArguments(chain, only_types || !first_stage); // Build a list of output columns of the window step. // 1) We need the columns that are the output of ExpressionActions. - for (const auto & x : chain.getLastActions()->actions.getNamesAndTypesList()) + for (const auto & x : chain.getLastActions()->dag.getNamesAndTypesList()) { query_analyzer.columns_after_window.push_back(x); } @@ -2130,7 +2132,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( finalize_chain(chain); query_analyzer.appendExpressionsAfterWindowFunctions(chain, only_types || !first_stage); - for (const auto & x : chain.getLastActions()->actions.getNamesAndTypesList()) + for (const auto & x : chain.getLastActions()->dag.getNamesAndTypesList()) { query_analyzer.columns_after_window.push_back(x); } @@ -2254,7 +2256,7 @@ std::string ExpressionAnalysisResult::dump() const if (before_array_join) { - ss << "before_array_join " << before_array_join->actions.dumpDAG() << "\n"; + ss << "before_array_join " << before_array_join->dag.dumpDAG() << "\n"; } if (array_join) @@ -2264,12 +2266,12 @@ std::string ExpressionAnalysisResult::dump() const if (before_join) { - ss << "before_join " << before_join->actions.dumpDAG() << "\n"; + ss << "before_join " << before_join->dag.dumpDAG() << "\n"; } if (before_where) { - ss << "before_where " << before_where->actions.dumpDAG() << "\n"; + ss << "before_where " << before_where->dag.dumpDAG() << "\n"; } if (prewhere_info) @@ -2284,32 +2286,32 @@ std::string ExpressionAnalysisResult::dump() const if (before_aggregation) { - ss << "before_aggregation " << before_aggregation->actions.dumpDAG() << "\n"; + ss << "before_aggregation " << before_aggregation->dag.dumpDAG() << "\n"; } if (before_having) { - ss << "before_having " << before_having->actions.dumpDAG() << "\n"; + ss << "before_having " << before_having->dag.dumpDAG() << "\n"; } if (before_window) { - ss << "before_window " << before_window->actions.dumpDAG() << "\n"; + ss << "before_window " << before_window->dag.dumpDAG() << "\n"; } if (before_order_by) { - ss << "before_order_by " << before_order_by->actions.dumpDAG() << "\n"; + ss << "before_order_by " << before_order_by->dag.dumpDAG() << "\n"; } if (before_limit_by) { - ss << "before_limit_by " << before_limit_by->actions.dumpDAG() << "\n"; + ss << "before_limit_by " << before_limit_by->dag.dumpDAG() << "\n"; } if (final_projection) { - ss << "final_projection " << final_projection->actions.dumpDAG() << "\n"; + ss << "final_projection " << final_projection->dag.dumpDAG() << "\n"; } if (!selected_columns.empty()) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 6fa50bb70c9a..12d6dce8f725 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -231,20 +231,20 @@ struct ExpressionAnalysisResult bool use_grouping_set_key = false; - ActionsAndFlagsPtr before_array_join; + ActionsAndProjectInputsFlagPtr before_array_join; ArrayJoinActionPtr array_join; - ActionsAndFlagsPtr before_join; - ActionsAndFlagsPtr converting_join_columns; + ActionsAndProjectInputsFlagPtr before_join; + ActionsAndProjectInputsFlagPtr converting_join_columns; JoinPtr join; - ActionsAndFlagsPtr before_where; - ActionsAndFlagsPtr before_aggregation; - ActionsAndFlagsPtr before_having; + ActionsAndProjectInputsFlagPtr before_where; + ActionsAndProjectInputsFlagPtr before_aggregation; + ActionsAndProjectInputsFlagPtr before_having; String having_column_name; bool remove_having_filter = false; - ActionsAndFlagsPtr before_window; - ActionsAndFlagsPtr before_order_by; - ActionsAndFlagsPtr before_limit_by; - ActionsAndFlagsPtr final_projection; + ActionsAndProjectInputsFlagPtr before_window; + ActionsAndProjectInputsFlagPtr before_order_by; + ActionsAndProjectInputsFlagPtr before_limit_by; + ActionsAndProjectInputsFlagPtr final_projection; /// Columns from the SELECT list, before renaming them to aliases. Used to /// perform SELECT DISTINCT. @@ -351,12 +351,12 @@ class SelectQueryExpressionAnalyzer : public ExpressionAnalyzer /// Tables that will need to be sent to remote servers for distributed query processing. const TemporaryTablesMapping & getExternalTables() const { return external_tables; } - ActionsAndFlagsPtr simpleSelectActions(); + ActionsAndProjectInputsFlagPtr simpleSelectActions(); /// These appends are public only for tests void appendSelect(ExpressionActionsChain & chain, bool only_types); /// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases. - ActionsAndFlagsPtr appendProjectResult(ExpressionActionsChain & chain) const; + ActionsAndProjectInputsFlagPtr appendProjectResult(ExpressionActionsChain & chain) const; private: StorageMetadataPtr metadata_snapshot; @@ -386,13 +386,13 @@ class SelectQueryExpressionAnalyzer : public ExpressionAnalyzer */ /// Before aggregation: - ArrayJoinActionPtr appendArrayJoin(ExpressionActionsChain & chain, ActionsAndFlagsPtr & before_array_join, bool only_types); + ArrayJoinActionPtr appendArrayJoin(ExpressionActionsChain & chain, ActionsAndProjectInputsFlagPtr & before_array_join, bool only_types); bool appendJoinLeftKeys(ExpressionActionsChain & chain, bool only_types); - JoinPtr appendJoin(ExpressionActionsChain & chain, ActionsAndFlagsPtr & converting_join_columns); + JoinPtr appendJoin(ExpressionActionsChain & chain, ActionsAndProjectInputsFlagPtr & converting_join_columns); /// remove_filter is set in ExpressionActionsChain::finalize(); /// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier). - ActionsAndFlagsPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types); + ActionsAndProjectInputsFlagPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types); bool appendWhere(ExpressionActionsChain & chain, bool only_types); bool appendGroupBy(ExpressionActionsChain & chain, bool only_types, bool optimize_aggregation_in_order, ManyExpressionActions &); void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types); @@ -406,7 +406,7 @@ class SelectQueryExpressionAnalyzer : public ExpressionAnalyzer /// After aggregation: bool appendHaving(ExpressionActionsChain & chain, bool only_types); /// appendSelect - ActionsAndFlagsPtr appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions &); + ActionsAndProjectInputsFlagPtr appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions &); bool appendLimitBy(ExpressionActionsChain & chain, bool only_types); /// appendProjectResult }; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index b3c8140914fe..f5b54ec64cbc 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -175,7 +175,7 @@ FilterDAGInfoPtr generateFilterActions( /// Using separate expression analyzer to prevent any possible alias injection auto syntax_result = TreeRewriter(context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, storage_snapshot)); SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context, metadata_snapshot, {}, false, {}, prepared_sets); - filter_info->actions = std::make_unique(std::move(analyzer.simpleSelectActions()->actions)); + filter_info->actions = std::make_unique(std::move(analyzer.simpleSelectActions()->dag)); filter_info->column_name = expr_list->children.at(0)->getColumnName(); filter_info->actions->removeUnusedActions(NameSet{filter_info->column_name}); @@ -1077,15 +1077,15 @@ Block InterpreterSelectQuery::getSampleBlockImpl() // with this code. See // https://github.com/ClickHouse/ClickHouse/issues/19857 for details. if (analysis_result.before_window) - return analysis_result.before_window->actions.getResultColumns(); + return analysis_result.before_window->dag.getResultColumns(); // NOTE: should not handle before_limit_by specially since // WithMergeableState does not process LIMIT BY - return analysis_result.before_order_by->actions.getResultColumns(); + return analysis_result.before_order_by->dag.getResultColumns(); } - Block header = analysis_result.before_aggregation->actions.getResultColumns(); + Block header = analysis_result.before_aggregation->dag.getResultColumns(); Block res; @@ -1123,18 +1123,18 @@ Block InterpreterSelectQuery::getSampleBlockImpl() // It's different from selected_columns, see the comment above for // WithMergeableState stage. if (analysis_result.before_window) - return analysis_result.before_window->actions.getResultColumns(); + return analysis_result.before_window->dag.getResultColumns(); // In case of query on remote shards executed up to // WithMergeableStateAfterAggregation*, they can process LIMIT BY, // since the initiator will not apply LIMIT BY again. if (analysis_result.before_limit_by) - return analysis_result.before_limit_by->actions.getResultColumns(); + return analysis_result.before_limit_by->dag.getResultColumns(); - return analysis_result.before_order_by->actions.getResultColumns(); + return analysis_result.before_order_by->dag.getResultColumns(); } - return analysis_result.final_projection->actions.getResultColumns(); + return analysis_result.final_projection->dag.getResultColumns(); } @@ -2313,7 +2313,7 @@ std::optional InterpreterSelectQuery::getTrivialCount(UInt64 max_paralle } if (analysis_result.hasWhere()) { - filter_nodes.push_back(&analysis_result.before_where->actions.findInOutputs(analysis_result.where_column_name)); + filter_nodes.push_back(&analysis_result.before_where->dag.findInOutputs(analysis_result.where_column_name)); } auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes); @@ -2379,7 +2379,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc auto column = ColumnAggregateFunction::create(func); column->insertFrom(place); - Block header = analysis_result.before_aggregation->actions.getResultColumns(); + Block header = analysis_result.before_aggregation->dag.getResultColumns(); size_t arguments_size = desc.argument_names.size(); DataTypes argument_types(arguments_size); for (size_t j = 0; j < arguments_size; ++j) @@ -2576,9 +2576,9 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc } } -void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, bool remove_filter) +void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = expression->actions.clone(); + auto dag = expression->dag.clone(); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -2657,7 +2657,7 @@ static GroupingSetsParamsList getAggregatorGroupingSetsParams(const SelectQueryE return result; } -void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) +void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) { executeExpression(query_plan, expression, "Before GROUP BY"); @@ -2750,9 +2750,9 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool } -void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, bool remove_filter) +void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter) { - auto dag = expression->actions.clone(); + auto dag = expression->dag.clone(); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -2765,12 +2765,12 @@ void InterpreterSelectQuery::executeHaving(QueryPlan & query_plan, const Actions void InterpreterSelectQuery::executeTotalsAndHaving( - QueryPlan & query_plan, bool has_having, const ActionsAndFlagsPtr & expression, bool remove_filter, bool overflow_row, bool final) + QueryPlan & query_plan, bool has_having, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter, bool overflow_row, bool final) { ActionsDAGPtr dag; if (expression) { - dag = expression->actions.clone(); + dag = expression->dag.clone(); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); } @@ -2814,12 +2814,12 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific query_plan.addStep(std::move(step)); } -void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, const std::string & description) +void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, const std::string & description) { if (!expression) return; - auto dag = expression->actions.clone(); + auto dag = expression->dag.clone(); if (expression->project_input) dag->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -2993,7 +2993,7 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPlan & query_plan, const st } -void InterpreterSelectQuery::executeProjection(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression) +void InterpreterSelectQuery::executeProjection(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression) { executeExpression(query_plan, expression, "Projection"); } diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index c4012180b0f2..d4ed19d45ea9 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -174,13 +174,13 @@ class InterpreterSelectQuery : public IInterpreterUnionOrSelectQuery /// Different stages of query execution. void executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan); - void executeWhere(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, bool remove_filter); + void executeWhere(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter); void executeAggregation( - QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); + QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); void executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final, bool has_grouping_sets); - void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsAndFlagsPtr & expression, bool remove_filter, bool overflow_row, bool final); - void executeHaving(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, bool remove_filter); - static void executeExpression(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression, const std::string & description); + void executeTotalsAndHaving(QueryPlan & query_plan, bool has_having, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter, bool overflow_row, bool final); + void executeHaving(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool remove_filter); + static void executeExpression(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, const std::string & description); /// FIXME should go through ActionsDAG to behave as a proper function void executeWindow(QueryPlan & query_plan); void executeOrder(QueryPlan & query_plan, InputOrderInfoPtr sorting_info); @@ -191,7 +191,7 @@ class InterpreterSelectQuery : public IInterpreterUnionOrSelectQuery void executeLimitBy(QueryPlan & query_plan); void executeLimit(QueryPlan & query_plan); void executeOffset(QueryPlan & query_plan); - static void executeProjection(QueryPlan & query_plan, const ActionsAndFlagsPtr & expression); + static void executeProjection(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression); void executeDistinct(QueryPlan & query_plan, bool before_order, Names columns, bool pre_distinct); void executeExtremes(QueryPlan & query_plan); void executeSubqueriesInSetsAndJoins(QueryPlan & query_plan); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 309ab2691a90..6d3a4f30b341 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1137,9 +1137,9 @@ void MutationsInterpreter::prepareMutationStages(std::vector & prepared_s for (const auto & kv : stage.column_to_updated) { auto column_name = kv.second->getColumnName(); - const auto & dag_node = actions->actions.findInOutputs(column_name); - const auto & alias = actions->actions.addAlias(dag_node, kv.first); - actions->actions.addOrReplaceInOutputs(alias); + const auto & dag_node = actions->dag.findInOutputs(column_name); + const auto & alias = actions->dag.addAlias(dag_node, kv.first); + actions->dag.addOrReplaceInOutputs(alias); } } @@ -1202,7 +1202,7 @@ void MutationsInterpreter::Source::read( { ActionsDAG::NodeRawConstPtrs nodes(num_filters); for (size_t i = 0; i < num_filters; ++i) - nodes[i] = &steps[i]->actions()->actions.findInOutputs(names[i]); + nodes[i] = &steps[i]->actions()->dag.findInOutputs(names[i]); filter = ActionsDAG::buildFilterActionsDAG(nodes); } @@ -1273,12 +1273,12 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v for (size_t i = 0; i < stage.expressions_chain.steps.size(); ++i) { const auto & step = stage.expressions_chain.steps[i]; - if (step->actions()->actions.hasArrayJoin()) + if (step->actions()->dag.hasArrayJoin()) throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "arrayJoin is not allowed in mutations"); if (i < stage.filter_column_names.size()) { - auto dag = step->actions()->actions.clone(); + auto dag = step->actions()->dag.clone(); if (step->actions()->project_input) dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute DELETEs. @@ -1286,7 +1286,7 @@ QueryPipelineBuilder MutationsInterpreter::addStreamsForLaterStages(const std::v } else { - auto dag = step->actions()->actions.clone(); + auto dag = step->actions()->dag.clone(); if (step->actions()->project_input) dag->appendInputsForUnusedColumns(plan.getCurrentDataStream().header); /// Execute UPDATE or final projection. diff --git a/src/Planner/ActionsChain.cpp b/src/Planner/ActionsChain.cpp index 85d50429bab6..1b594c5f2a15 100644 --- a/src/Planner/ActionsChain.cpp +++ b/src/Planner/ActionsChain.cpp @@ -11,7 +11,7 @@ namespace DB { -ActionsChainStep::ActionsChainStep(ActionsAndFlagsPtr actions_, +ActionsChainStep::ActionsChainStep(ActionsAndProjectInputsFlagPtr actions_, bool use_actions_nodes_as_output_columns_, ColumnsWithTypeAndName additional_output_columns_) : actions(std::move(actions_)) @@ -28,12 +28,12 @@ void ActionsChainStep::finalizeInputAndOutputColumns(const NameSet & child_input auto child_input_columns_copy = child_input_columns; std::unordered_set output_nodes_names; - output_nodes_names.reserve(actions->actions.getOutputs().size()); + output_nodes_names.reserve(actions->dag.getOutputs().size()); - for (auto & output_node : actions->actions.getOutputs()) + for (auto & output_node : actions->dag.getOutputs()) output_nodes_names.insert(output_node->result_name); - for (const auto & node : actions->actions.getNodes()) + for (const auto & node : actions->dag.getNodes()) { auto it = child_input_columns_copy.find(node.result_name); if (it == child_input_columns_copy.end()) @@ -45,11 +45,11 @@ void ActionsChainStep::finalizeInputAndOutputColumns(const NameSet & child_input if (output_nodes_names.contains(node.result_name)) continue; - actions->actions.getOutputs().push_back(&node); + actions->dag.getOutputs().push_back(&node); output_nodes_names.insert(node.result_name); } - actions->actions.removeUnusedActions(); + actions->dag.removeUnusedActions(); /// TODO: Analyzer fix ActionsDAG input and constant nodes with same name actions->project_input = true; initialize(); @@ -58,7 +58,7 @@ void ActionsChainStep::finalizeInputAndOutputColumns(const NameSet & child_input void ActionsChainStep::dump(WriteBuffer & buffer) const { buffer << "DAG" << '\n'; - buffer << actions->actions.dumpDAG(); + buffer << actions->dag.dumpDAG(); if (!available_output_columns.empty()) { @@ -84,7 +84,7 @@ String ActionsChainStep::dump() const void ActionsChainStep::initialize() { - auto required_columns_names = actions->actions.getRequiredColumnsNames(); + auto required_columns_names = actions->dag.getRequiredColumnsNames(); input_columns_names = NameSet(required_columns_names.begin(), required_columns_names.end()); available_output_columns.clear(); @@ -93,7 +93,7 @@ void ActionsChainStep::initialize() { std::unordered_set available_output_columns_names; - for (const auto & node : actions->actions.getNodes()) + for (const auto & node : actions->dag.getNodes()) { if (available_output_columns_names.contains(node.result_name)) continue; diff --git a/src/Planner/ActionsChain.h b/src/Planner/ActionsChain.h index ab5823b99d75..3bce19786e64 100644 --- a/src/Planner/ActionsChain.h +++ b/src/Planner/ActionsChain.h @@ -48,18 +48,18 @@ class ActionsChainStep * If use_actions_nodes_as_output_columns = true output columns are initialized using actions dag nodes. * If additional output columns are specified they are added to output columns. */ - explicit ActionsChainStep(ActionsAndFlagsPtr actions_, + explicit ActionsChainStep(ActionsAndProjectInputsFlagPtr actions_, bool use_actions_nodes_as_output_columns = true, ColumnsWithTypeAndName additional_output_columns_ = {}); /// Get actions - ActionsAndFlagsPtr & getActions() + ActionsAndProjectInputsFlagPtr & getActions() { return actions; } /// Get actions - const ActionsAndFlagsPtr & getActions() const + const ActionsAndProjectInputsFlagPtr & getActions() const { return actions; } @@ -98,7 +98,7 @@ class ActionsChainStep private: void initialize(); - ActionsAndFlagsPtr actions; + ActionsAndProjectInputsFlagPtr actions; bool use_actions_nodes_as_output_columns = true; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 5496a7e42574..2d42ed732238 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -329,11 +329,11 @@ class QueryAnalysisResult }; void addExpressionStep(QueryPlan & query_plan, - const ActionsAndFlagsPtr & expression_actions, + const ActionsAndProjectInputsFlagPtr & expression_actions, const std::string & step_description, std::vector & result_actions_to_execute) { - auto actions = expression_actions->actions.clone(); + auto actions = expression_actions->dag.clone(); if (expression_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -348,7 +348,7 @@ void addFilterStep(QueryPlan & query_plan, const std::string & step_description, std::vector & result_actions_to_execute) { - auto actions = filter_analysis_result.filter_actions->actions.clone(); + auto actions = filter_analysis_result.filter_actions->dag.clone(); if (filter_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); @@ -556,7 +556,7 @@ void addTotalsHavingStep(QueryPlan & query_plan, ActionsDAGPtr actions; if (having_analysis_result.filter_actions) { - actions = having_analysis_result.filter_actions->actions.clone(); + actions = having_analysis_result.filter_actions->dag.clone(); if (having_analysis_result.filter_actions->project_input) actions->appendInputsForUnusedColumns(query_plan.getCurrentDataStream().header); diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 738b8c1e9d54..ceb506d1bbb9 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -45,10 +45,10 @@ std::optional analyzeFilter(const QueryTreeNodePtr & filte { FilterAnalysisResult result; - result.filter_actions = std::make_shared(); - result.filter_actions->actions = buildActionsDAGFromExpressionNode(filter_expression_node, input_columns, planner_context); + result.filter_actions = std::make_shared(); + result.filter_actions->dag = buildActionsDAGFromExpressionNode(filter_expression_node, input_columns, planner_context); - const auto * output = result.filter_actions->actions.getOutputs().at(0); + const auto * output = result.filter_actions->dag.getOutputs().at(0); if (output->column && ConstantFilterDescription(*output->column).always_true) return {}; @@ -118,9 +118,9 @@ std::optional analyzeAggregation(const QueryTreeNodeP Names aggregation_keys; - ActionsAndFlagsPtr before_aggregation_actions = std::make_shared(); - before_aggregation_actions->actions = ActionsDAG(input_columns); - before_aggregation_actions->actions.getOutputs().clear(); + ActionsAndProjectInputsFlagPtr before_aggregation_actions = std::make_shared(); + before_aggregation_actions->dag = ActionsDAG(input_columns); + before_aggregation_actions->dag.getOutputs().clear(); std::unordered_set before_aggregation_actions_output_node_names; @@ -155,7 +155,7 @@ std::optional analyzeAggregation(const QueryTreeNodeP if (constant_key && !aggregates_descriptions.empty() && (!check_constants_for_group_by_key || canRemoveConstantFromGroupByKey(*constant_key))) continue; - auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions->actions, grouping_set_key_node); + auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions->dag, grouping_set_key_node); aggregation_keys.reserve(expression_dag_nodes.size()); for (auto & expression_dag_node : expression_dag_nodes) @@ -168,7 +168,7 @@ std::optional analyzeAggregation(const QueryTreeNodeP auto column_after_aggregation = group_by_use_nulls && expression_dag_node->column != nullptr ? makeNullableSafe(expression_dag_node->column) : expression_dag_node->column; available_columns_after_aggregation.emplace_back(std::move(column_after_aggregation), expression_type_after_aggregation, expression_dag_node->result_name); aggregation_keys.push_back(expression_dag_node->result_name); - before_aggregation_actions->actions.getOutputs().push_back(expression_dag_node); + before_aggregation_actions->dag.getOutputs().push_back(expression_dag_node); before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name); } } @@ -207,7 +207,7 @@ std::optional analyzeAggregation(const QueryTreeNodeP if (constant_key && !aggregates_descriptions.empty() && (!check_constants_for_group_by_key || canRemoveConstantFromGroupByKey(*constant_key))) continue; - auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions->actions, group_by_key_node); + auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions->dag, group_by_key_node); aggregation_keys.reserve(expression_dag_nodes.size()); for (auto & expression_dag_node : expression_dag_nodes) @@ -219,7 +219,7 @@ std::optional analyzeAggregation(const QueryTreeNodeP auto column_after_aggregation = group_by_use_nulls && expression_dag_node->column != nullptr ? makeNullableSafe(expression_dag_node->column) : expression_dag_node->column; available_columns_after_aggregation.emplace_back(std::move(column_after_aggregation), expression_type_after_aggregation, expression_dag_node->result_name); aggregation_keys.push_back(expression_dag_node->result_name); - before_aggregation_actions->actions.getOutputs().push_back(expression_dag_node); + before_aggregation_actions->dag.getOutputs().push_back(expression_dag_node); before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name); } } @@ -233,13 +233,13 @@ std::optional analyzeAggregation(const QueryTreeNodeP auto & aggregate_function_node_typed = aggregate_function_node->as(); for (const auto & aggregate_function_node_argument : aggregate_function_node_typed.getArguments().getNodes()) { - auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions->actions, aggregate_function_node_argument); + auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions->dag, aggregate_function_node_argument); for (auto & expression_dag_node : expression_dag_nodes) { if (before_aggregation_actions_output_node_names.contains(expression_dag_node->result_name)) continue; - before_aggregation_actions->actions.getOutputs().push_back(expression_dag_node); + before_aggregation_actions->dag.getOutputs().push_back(expression_dag_node); before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name); } } @@ -286,9 +286,9 @@ std::optional analyzeWindow(const QueryTreeNodePtr & query PlannerActionsVisitor actions_visitor(planner_context); - ActionsAndFlagsPtr before_window_actions = std::make_shared(); - before_window_actions->actions = ActionsDAG(input_columns); - before_window_actions->actions.getOutputs().clear(); + ActionsAndProjectInputsFlagPtr before_window_actions = std::make_shared(); + before_window_actions->dag = ActionsDAG(input_columns); + before_window_actions->dag.getOutputs().clear(); std::unordered_set before_window_actions_output_node_names; @@ -297,25 +297,25 @@ std::optional analyzeWindow(const QueryTreeNodePtr & query auto & window_function_node_typed = window_function_node->as(); auto & window_node = window_function_node_typed.getWindowNode()->as(); - auto expression_dag_nodes = actions_visitor.visit(before_window_actions->actions, window_function_node_typed.getArgumentsNode()); + auto expression_dag_nodes = actions_visitor.visit(before_window_actions->dag, window_function_node_typed.getArgumentsNode()); for (auto & expression_dag_node : expression_dag_nodes) { if (before_window_actions_output_node_names.contains(expression_dag_node->result_name)) continue; - before_window_actions->actions.getOutputs().push_back(expression_dag_node); + before_window_actions->dag.getOutputs().push_back(expression_dag_node); before_window_actions_output_node_names.insert(expression_dag_node->result_name); } - expression_dag_nodes = actions_visitor.visit(before_window_actions->actions, window_node.getPartitionByNode()); + expression_dag_nodes = actions_visitor.visit(before_window_actions->dag, window_node.getPartitionByNode()); for (auto & expression_dag_node : expression_dag_nodes) { if (before_window_actions_output_node_names.contains(expression_dag_node->result_name)) continue; - before_window_actions->actions.getOutputs().push_back(expression_dag_node); + before_window_actions->dag.getOutputs().push_back(expression_dag_node); before_window_actions_output_node_names.insert(expression_dag_node->result_name); } @@ -326,14 +326,14 @@ std::optional analyzeWindow(const QueryTreeNodePtr & query for (auto & sort_node : order_by_node_list.getNodes()) { auto & sort_node_typed = sort_node->as(); - expression_dag_nodes = actions_visitor.visit(before_window_actions->actions, sort_node_typed.getExpression()); + expression_dag_nodes = actions_visitor.visit(before_window_actions->dag, sort_node_typed.getExpression()); for (auto & expression_dag_node : expression_dag_nodes) { if (before_window_actions_output_node_names.contains(expression_dag_node->result_name)) continue; - before_window_actions->actions.getOutputs().push_back(expression_dag_node); + before_window_actions->dag.getOutputs().push_back(expression_dag_node); before_window_actions_output_node_names.insert(expression_dag_node->result_name); } } @@ -366,8 +366,8 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { - auto projection_actions = std::make_shared(); - projection_actions->actions = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), input_columns, planner_context); + auto projection_actions = std::make_shared(); + projection_actions->dag = buildActionsDAGFromExpressionNode(query_node.getProjectionNode(), input_columns, planner_context); auto projection_columns = query_node.getProjectionColumns(); size_t projection_columns_size = projection_columns.size(); @@ -376,7 +376,7 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, NamesWithAliases projection_column_names_with_display_aliases; projection_column_names_with_display_aliases.reserve(projection_columns_size); - auto & projection_actions_outputs = projection_actions->actions.getOutputs(); + auto & projection_actions_outputs = projection_actions->dag.getOutputs(); size_t projection_outputs_size = projection_actions_outputs.size(); if (projection_columns_size != projection_outputs_size) @@ -414,9 +414,9 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, const PlannerContextPtr & planner_context, ActionsChain & actions_chain) { - auto before_sort_actions = std::make_shared(); - before_sort_actions->actions = ActionsDAG(input_columns); - auto & before_sort_actions_outputs = before_sort_actions->actions.getOutputs(); + auto before_sort_actions = std::make_shared(); + before_sort_actions->dag = ActionsDAG(input_columns); + auto & before_sort_actions_outputs = before_sort_actions->dag.getOutputs(); before_sort_actions_outputs.clear(); PlannerActionsVisitor actions_visitor(planner_context); @@ -430,7 +430,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, for (const auto & sort_node : order_by_node_list.getNodes()) { auto & sort_node_typed = sort_node->as(); - auto expression_dag_nodes = actions_visitor.visit(before_sort_actions->actions, sort_node_typed.getExpression()); + auto expression_dag_nodes = actions_visitor.visit(before_sort_actions->dag, sort_node_typed.getExpression()); has_with_fill |= sort_node_typed.withFill(); for (auto & action_dag_node : expression_dag_nodes) @@ -446,7 +446,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, if (has_with_fill) { for (auto & output_node : before_sort_actions_outputs) - output_node = &before_sort_actions->actions.materializeNode(*output_node); + output_node = &before_sort_actions->dag.materializeNode(*output_node); } /// We add only INPUT columns necessary for INTERPOLATE expression in before ORDER BY actions DAG @@ -464,7 +464,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, } std::unordered_map before_sort_actions_inputs_name_to_node; - for (const auto & node : before_sort_actions->actions.getInputs()) + for (const auto & node : before_sort_actions->dag.getInputs()) before_sort_actions_inputs_name_to_node.emplace(node->result_name, node); for (const auto & node : interpolate_actions_dag.getNodes()) @@ -477,7 +477,7 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, if (input_node_it == before_sort_actions_inputs_name_to_node.end()) { auto input_column = ColumnWithTypeAndName{node.column, node.result_type, node.result_name}; - const auto * input_node = &before_sort_actions->actions.addInput(std::move(input_column)); + const auto * input_node = &before_sort_actions->dag.addInput(std::move(input_column)); auto [it, _] = before_sort_actions_inputs_name_to_node.emplace(node.result_name, input_node); input_node_it = it; } @@ -502,23 +502,23 @@ LimitByAnalysisResult analyzeLimitBy(const QueryNode & query_node, const NameSet & required_output_nodes_names, ActionsChain & actions_chain) { - auto before_limit_by_actions = std::make_shared(); - before_limit_by_actions->actions = buildActionsDAGFromExpressionNode(query_node.getLimitByNode(), input_columns, planner_context); + auto before_limit_by_actions = std::make_shared(); + before_limit_by_actions->dag = buildActionsDAGFromExpressionNode(query_node.getLimitByNode(), input_columns, planner_context); NameSet limit_by_column_names_set; Names limit_by_column_names; - limit_by_column_names.reserve(before_limit_by_actions->actions.getOutputs().size()); - for (auto & output_node : before_limit_by_actions->actions.getOutputs()) + limit_by_column_names.reserve(before_limit_by_actions->dag.getOutputs().size()); + for (auto & output_node : before_limit_by_actions->dag.getOutputs()) { limit_by_column_names_set.insert(output_node->result_name); limit_by_column_names.push_back(output_node->result_name); } - for (const auto & node : before_limit_by_actions->actions.getNodes()) + for (const auto & node : before_limit_by_actions->dag.getNodes()) { if (required_output_nodes_names.contains(node.result_name) && !limit_by_column_names_set.contains(node.result_name)) - before_limit_by_actions->actions.getOutputs().push_back(&node); + before_limit_by_actions->dag.getOutputs().push_back(&node); } auto actions_step_before_limit_by = std::make_unique(before_limit_by_actions); @@ -612,7 +612,7 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo if (sort_analysis_result_optional.has_value() && planner_query_processing_info.isFirstStage() && planner_query_processing_info.getToStage() != QueryProcessingStage::Complete) { const auto & before_order_by_actions = sort_analysis_result_optional->before_order_by_actions; - for (const auto & output_node : before_order_by_actions->actions.getOutputs()) + for (const auto & output_node : before_order_by_actions->dag.getOutputs()) required_output_nodes_names.insert(output_node->result_name); } @@ -668,9 +668,9 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo } } - auto project_names_actions = std::make_shared(); - project_names_actions->actions = ActionsDAG(project_names_input); - project_names_actions->actions.project(projection_analysis_result.projection_column_names_with_display_aliases); + auto project_names_actions = std::make_shared(); + project_names_actions->dag = ActionsDAG(project_names_input); + project_names_actions->dag.project(projection_analysis_result.projection_column_names_with_display_aliases); project_names_actions->project_input = true; actions_chain.addStep(std::make_unique(project_names_actions)); diff --git a/src/Planner/PlannerExpressionAnalysis.h b/src/Planner/PlannerExpressionAnalysis.h index 3a9ed903bbc4..820df7131a77 100644 --- a/src/Planner/PlannerExpressionAnalysis.h +++ b/src/Planner/PlannerExpressionAnalysis.h @@ -17,22 +17,22 @@ namespace DB struct ProjectionAnalysisResult { - ActionsAndFlagsPtr projection_actions; + ActionsAndProjectInputsFlagPtr projection_actions; Names projection_column_names; NamesWithAliases projection_column_names_with_display_aliases; - ActionsAndFlagsPtr project_names_actions; + ActionsAndProjectInputsFlagPtr project_names_actions; }; struct FilterAnalysisResult { - ActionsAndFlagsPtr filter_actions; + ActionsAndProjectInputsFlagPtr filter_actions; std::string filter_column_name; bool remove_filter_column = false; }; struct AggregationAnalysisResult { - ActionsAndFlagsPtr before_aggregation_actions; + ActionsAndProjectInputsFlagPtr before_aggregation_actions; Names aggregation_keys; AggregateDescriptions aggregate_descriptions; GroupingSetsParamsList grouping_sets_parameters_list; @@ -41,19 +41,19 @@ struct AggregationAnalysisResult struct WindowAnalysisResult { - ActionsAndFlagsPtr before_window_actions; + ActionsAndProjectInputsFlagPtr before_window_actions; std::vector window_descriptions; }; struct SortAnalysisResult { - ActionsAndFlagsPtr before_order_by_actions; + ActionsAndProjectInputsFlagPtr before_order_by_actions; bool has_with_fill = false; }; struct LimitByAnalysisResult { - ActionsAndFlagsPtr before_limit_by_actions; + ActionsAndProjectInputsFlagPtr before_limit_by_actions; Names limit_by_column_names; }; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp index 8efc3b230b8e..70327bc95b48 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeUseAggregateProjection.cpp @@ -77,7 +77,7 @@ static AggregateProjectionInfo getAggregatingProjectionInfo( AggregateProjectionInfo info; info.context = interpreter.getContext(); - info.before_aggregation = analysis_result.before_aggregation->actions.clone(); + info.before_aggregation = analysis_result.before_aggregation->dag.clone(); info.keys = query_analyzer->aggregationKeys().getNames(); info.aggregates = query_analyzer->aggregates(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 5c312f48c838..2e0ea4cdbcdd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -7061,7 +7061,7 @@ ActionDAGNodes MergeTreeData::getFiltersForPrimaryKeyAnalysis(const InterpreterS filter_nodes.nodes.push_back(&additional_filter_info->actions->findInOutputs(additional_filter_info->column_name)); if (before_where) - filter_nodes.nodes.push_back(&before_where->actions.findInOutputs(where_column_name)); + filter_nodes.nodes.push_back(&before_where->dag.findInOutputs(where_column_name)); return filter_nodes; } From 053caedd923d600c235b3c98e49cc0ca47f26799 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 18 Jun 2024 10:29:55 +0000 Subject: [PATCH 224/254] Cleanup. --- src/Interpreters/ActionsDAG.cpp | 28 ++++++++-------------------- 1 file changed, 8 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 6e11dc96769a..34f3e0a98bd9 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1149,21 +1149,6 @@ void ActionsDAG::project(const NamesWithAliases & projection) removeUnusedActions(); } -static void appendInputsFromNamesMap( - ActionsDAG & dag, - const ColumnsWithTypeAndName & source_columns, - const std::unordered_map> & names_map) -{ - for (const auto & [_, positions] : names_map) - { - for (auto pos : positions) - { - const auto & col = source_columns[pos]; - dag.addInput(col.name, col.type); - } - } -} - void ActionsDAG::appendInputsForUnusedColumns(const Block & sample_block) { std::unordered_map> names_map; @@ -1181,7 +1166,14 @@ void ActionsDAG::appendInputsForUnusedColumns(const Block & sample_block) positions.pop_front(); } - appendInputsFromNamesMap(*this, sample_block.getColumnsWithTypeAndName(), names_map); + for (const auto & [_, positions] : names_map) + { + for (auto pos : positions) + { + const auto & col = sample_block.getByPosition(pos); + addInput(col.name, col.type); + } + } } bool ActionsDAG::tryRestoreColumn(const std::string & column_name) @@ -2753,11 +2745,7 @@ void ActionsDAG::removeUnusedConjunctions(NodeRawConstPtrs rejected_conjunctions std::unordered_set used_inputs; for (const auto * input : inputs) - { - // if (removes_filter && input == predicate) - // continue; used_inputs.insert(input); - } removeUnusedActions(used_inputs); } From e4981c656331a2dd754d1ed77d6b7599e52ba4c2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 18 Jun 2024 10:33:59 +0000 Subject: [PATCH 225/254] Cleanup. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 06fae1e810dc..b2d8aa0e2180 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -849,9 +849,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_ static ActionsDAGPtr createProjection(const Block & header) { - auto projection = std::make_shared(header.getNamesAndTypesList()); - // projection->removeUnusedActions(header.getNames()); - return projection; + return std::make_shared(header.getNamesAndTypesList()); } Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( From 121f45c8dcc19c1971c47b1f1ad54bd16b119776 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 18 Jun 2024 10:19:52 +0200 Subject: [PATCH 226/254] Fix alignment of Distinct combinator --- .../Combinators/AggregateFunctionDistinct.h | 5 +++++ .../03173_distinct_combinator_alignment.reference | 0 .../0_stateless/03173_distinct_combinator_alignment.sql | 1 + 3 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/03173_distinct_combinator_alignment.reference create mode 100644 tests/queries/0_stateless/03173_distinct_combinator_alignment.sql diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h b/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h index 4338dcff5c09..f532858b3d8d 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h +++ b/src/AggregateFunctions/Combinators/AggregateFunctionDistinct.h @@ -228,6 +228,11 @@ class AggregateFunctionDistinct : public IAggregateFunctionDataHelpersizeOfData(); } + size_t alignOfData() const override + { + return std::max(alignof(Data), nested_func->alignOfData()); + } + void create(AggregateDataPtr __restrict place) const override { new (place) Data; diff --git a/tests/queries/0_stateless/03173_distinct_combinator_alignment.reference b/tests/queries/0_stateless/03173_distinct_combinator_alignment.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03173_distinct_combinator_alignment.sql b/tests/queries/0_stateless/03173_distinct_combinator_alignment.sql new file mode 100644 index 000000000000..4a066be50860 --- /dev/null +++ b/tests/queries/0_stateless/03173_distinct_combinator_alignment.sql @@ -0,0 +1 @@ +SELECT toTypeName(topKDistinctState(toNullable(10))(toString(number)) IGNORE NULLS) FROM numbers(100) GROUP BY tuple((map((materialize(toNullable(1)), 2), 4, (3, 4), 5), 3)), map((1, 2), 4, (3, 4), toNullable(5)) WITH CUBE WITH TOTALS FORMAT Null From bc85577af098d48438388f358fbaa48d67df4fe7 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 18 Jun 2024 13:17:16 +0200 Subject: [PATCH 227/254] init --- .../reference/groupconcat.md | 90 +++++++++++++++++++ 1 file changed, 90 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/groupconcat.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md new file mode 100644 index 000000000000..072252de8c9f --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md @@ -0,0 +1,90 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/groupconcat +sidebar_position: 363 +sidebar_label: groupConcat +title: groupConcat +--- + +Calculates a concatenated string from a group of strings, optionally separated by a delimiter, and optionally limited by a maximum number of elements. + +**Syntax** + +``` sql +groupConcat(expression [, delimiter] [, limit]); +``` + +**Arguments** + +- `expression` — The expression or column name that outputs strings to be concatenated.. +- `delimiter` — A [string](../../../sql-reference/data-types/string.md) that will be used to separate concatenated values. This parameter is optional and defaults to an empty string if not specified. +- `limit` — A positive [integer](../../../sql-reference/data-types/int-uint.md) specifying the maximum number of elements to concatenate. If more elements are present, excess elements are ignored. This parameter is optional. + +:::note +If delimiter is specified without limit, it must be the first parameter following the expression. If both delimiter and limit are specified, delimiter must precede limit. +::: + +**Returned value** + +- Returns a [string](../../../sql-reference/data-types/string.md) consisting of the concatenated values of the column or expression. If the group has no elements or only null elements, and the function does not specify a handling for only null values, the result is a nullable string with a null value. + +**Examples** + +Input table: + +``` text +┌─id─┬─name─┐ +│ 1 │ John│ +│ 2 │ Jane│ +│ 3 │ Bob│ +└────┴──────┘ +``` + +1. Basic usage without a delimiter: + +Query: + +``` sql +SELECT groupConcat(Name) FROM Employees; +``` + +Result: + +``` text +JohnJaneBob +``` + +This concatenates all names into one continuous string without any separator. + + +2. Using comma as a delimiter: + +Query: + +``` sql +SELECT groupConcat(Name, ', ', 2) FROM Employees; +``` + +Result: + +``` text +John, Jane, Bob +``` + +This output shows the names separated by a comma followed by a space. + + +3. Limiting the number of concatenated elements + +Query: + +``` sql +SELECT groupConcat(Name, ', ', 2) FROM Employees; +``` + +Result: + +``` text +John, Jane +``` + +This query limits the output to the first two names, even though there are more names in the table. From 190c649c424ecd80f4bc30bcc2e1015e7bd42804 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 18 Jun 2024 11:18:21 +0000 Subject: [PATCH 228/254] fix build in tests --- src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp b/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp index 0e281607bc2e..07798f78080f 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_event_queue.cpp @@ -92,12 +92,12 @@ struct QueueTest { event_queue.enqueue([this, text] { log += " " + text; }); } - EventQueue::EventId postpone(EventQueue::TimePoint until, const String & text) + EventId postpone(EventQueue::TimePoint until, const String & text) { return event_queue.postpone(until, [this, text] { log += " " + text; }); } - void cancel(EventQueue::EventId event_id) + void cancel(EventId event_id) { event_queue.cancelPostponed(event_id); } From 31af06238c9db5e65a2f3ac18814c14eabd0c41a Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 18 Jun 2024 13:25:00 +0200 Subject: [PATCH 229/254] docs spelling --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 64ff3e8e2cb8..302331be8b06 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1706,6 +1706,7 @@ groupBitmap groupBitmapAnd groupBitmapOr groupBitmapXor +groupConcat groupUniqArray grouparray grouparrayinsertat @@ -1722,6 +1723,7 @@ groupbitmapor groupbitmapxor groupbitor groupbitxor +groupconcat groupuniqarray grpc grpcio From ed38f370c327698cf660b17d8c09369970b1eee7 Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 18 Jun 2024 13:17:42 +0200 Subject: [PATCH 230/254] CI: Add Non-blocking (Woolen wolfdog) CI mode --- .github/PULL_REQUEST_TEMPLATE.md | 2 +- .github/workflows/pull_request.yml | 3 ++- tests/ci/ci.py | 10 +++++++--- tests/ci/ci_config.py | 4 +++- tests/ci/ci_definitions.py | 1 + tests/ci/ci_settings.py | 4 ++++ tests/ci/test_ci_config.py | 31 ++++++++++++++++++++++++++++++ tests/ci/test_ci_options.py | 3 +++ 8 files changed, 52 insertions(+), 6 deletions(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 74a6f95dbb34..d9f9e9d6c8be 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -49,7 +49,6 @@ At a minimum, the following information should be added (but add more as needed) - [ ] Allow: Integration Tests - [ ] Allow: Performance tests - [ ] Allow: All Builds -- [ ] Allow: All NOT Required Checks - [ ] Allow: batch 1, 2 for multi-batch jobs - [ ] Allow: batch 3, 4, 5, 6 for multi-batch jobs --- @@ -60,6 +59,7 @@ At a minimum, the following information should be added (but add more as needed) - [ ] Exclude: All with aarch64, release, debug --- - [ ] Do not test +- [ ] Woolen Wolfdog - [ ] Upload binaries for special builds - [ ] Disable merge-commit - [ ] Disable CI cache diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 70b71da8fa5e..b19d246e1d01 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -126,8 +126,9 @@ jobs: with: stage: Builds_2 data: ${{ needs.RunConfig.outputs.data }} + # stage for running non-required checks without being blocked by required checks (Test_1) if corresponding settings is selected Tests_2: - needs: [RunConfig, Builds_2] + needs: [RunConfig, Builds_1] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2') }} uses: ./.github/workflows/reusable_test_stage.yml with: diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 135a4c91c567..4e34e6b61358 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -462,7 +462,9 @@ def _configure_jobs( return ci_cache -def _generate_ci_stage_config(jobs_data: Dict[str, Any]) -> Dict[str, Dict[str, Any]]: +def _generate_ci_stage_config( + jobs_data: Dict[str, Any], non_blocking_mode: bool = False +) -> Dict[str, Dict[str, Any]]: """ populates GH Actions' workflow with real jobs "Builds_1": [{"job_name": NAME, "runner_type": RUNNER_TYPE}] @@ -472,7 +474,7 @@ def _generate_ci_stage_config(jobs_data: Dict[str, Any]) -> Dict[str, Dict[str, result = {} # type: Dict[str, Any] stages_to_do = [] for job in jobs_data: - stage_type = CI.get_job_ci_stage(job) + stage_type = CI.get_job_ci_stage(job, non_blocking_ci=non_blocking_mode) if stage_type == CI.WorkflowStages.NA: continue if stage_type not in result: @@ -1007,7 +1009,9 @@ def main() -> int: result["docs"] = ci_cache.job_digests[CI.JobNames.DOCS_CHECK] result["ci_settings"] = ci_settings.as_dict() if not args.skip_jobs: - result["stages_data"] = _generate_ci_stage_config(ci_cache.jobs_to_do) + result["stages_data"] = _generate_ci_stage_config( + ci_cache.jobs_to_do, ci_settings.woolen_wolfdog + ) result["jobs_data"] = { "jobs_to_do": list(ci_cache.jobs_to_do), "jobs_to_skip": ci_cache.jobs_to_skip, diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 60c6a60af1ab..bef43083a35a 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -545,7 +545,7 @@ def get_tag_config(cls, label_name: str) -> Optional[LabelConfig]: return None @classmethod - def get_job_ci_stage(cls, job_name: str) -> str: + def get_job_ci_stage(cls, job_name: str, non_blocking_ci: bool = False) -> str: if job_name in [ JobNames.STYLE_CHECK, JobNames.FAST_TEST, @@ -572,6 +572,8 @@ def get_job_ci_stage(cls, job_name: str) -> str: else: stage_type = WorkflowStages.TESTS_3 assert stage_type, f"BUG [{job_name}]" + if non_blocking_ci and stage_type == WorkflowStages.TESTS_3: + stage_type = WorkflowStages.TESTS_2 return stage_type @classmethod diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index fdd5dc7a6711..94555158811e 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -46,6 +46,7 @@ class Tags(metaclass=WithIter): """ DO_NOT_TEST_LABEL = "do_not_test" + WOOLEN_WOLFDOG_LABEL = "woolen_wolfdog" NO_MERGE_COMMIT = "no_merge_commit" NO_CI_CACHE = "no_ci_cache" # to upload all binaries from build jobs diff --git a/tests/ci/ci_settings.py b/tests/ci/ci_settings.py index a36fcf953ae3..7b2dd12c310c 100644 --- a/tests/ci/ci_settings.py +++ b/tests/ci/ci_settings.py @@ -29,6 +29,7 @@ class CiSettings: no_ci_cache: bool = False upload_all: bool = False no_merge_commit: bool = False + woolen_wolfdog: bool = False def as_dict(self) -> Dict[str, Any]: return asdict(self) @@ -108,6 +109,9 @@ def create_from_pr_message( elif match == CI.Tags.NO_MERGE_COMMIT: res.no_merge_commit = True print("NOTE: Merge Commit will be disabled") + elif match == CI.Tags.WOOLEN_WOLFDOG_LABEL: + res.woolen_wolfdog = True + print("NOTE: Woolen Wolfdog mode enabled") elif match.startswith("batch_"): batches = [] try: diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 7a51a65b5d59..47247b918580 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -201,6 +201,37 @@ def test_job_stage_config(self): msg=f"Stage for [{job}] is not correct", ) + def test_job_stage_config_non_blocking(self): + """ + check runner is provided w/o exception + """ + # check stages + for job in CI.JobNames: + if job in CI.BuildNames: + self.assertTrue( + CI.get_job_ci_stage(job) + in (CI.WorkflowStages.BUILDS_1, CI.WorkflowStages.BUILDS_2) + ) + else: + if job in ( + CI.JobNames.STYLE_CHECK, + CI.JobNames.FAST_TEST, + CI.JobNames.JEPSEN_SERVER, + CI.JobNames.JEPSEN_KEEPER, + CI.JobNames.BUILD_CHECK, + ): + self.assertEqual( + CI.get_job_ci_stage(job), + CI.WorkflowStages.NA, + msg=f"Stage for [{job}] is not correct", + ) + else: + self.assertTrue( + CI.get_job_ci_stage(job, non_blocking_ci=True) + in (CI.WorkflowStages.TESTS_1, CI.WorkflowStages.TESTS_2), + msg=f"Stage for [{job}] is not correct", + ) + def test_build_jobs_configs(self): """ check build jobs have non-None build_config attribute diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index fc21c7dda4ec..ee256f73abcc 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -19,6 +19,7 @@ #### CI options: - [ ] do not test (only style check) +- [x] Woolen Wolfdog CI - [x] disable merge-commit (no merge from master before tests) - [ ] disable CI cache (job reuse) @@ -148,6 +149,7 @@ def test_pr_body_parsing(self): self.assertFalse(ci_options.do_not_test) self.assertFalse(ci_options.no_ci_cache) self.assertTrue(ci_options.no_merge_commit) + self.assertTrue(ci_options.woolen_wolfdog) self.assertEqual(ci_options.ci_sets, ["ci_set_non_required"]) self.assertCountEqual(ci_options.include_keywords, ["foo", "foo_bar"]) self.assertCountEqual(ci_options.exclude_keywords, ["foo", "foo_bar"]) @@ -157,6 +159,7 @@ def test_options_applied(self): ci_options = CiSettings.create_from_pr_message( _TEST_BODY_2, update_from_api=False ) + self.assertFalse(ci_options.woolen_wolfdog) self.assertCountEqual( ci_options.include_keywords, ["integration", "foo_bar", "stateless", "azure"], From b1d6a31af4f2a5af58ce7f265ae9e94c7af3c662 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 18 Jun 2024 12:04:16 +0000 Subject: [PATCH 231/254] Another try. --- src/Storages/SelectQueryInfo.h | 10 ++++++++++ src/Storages/StorageDistributed.cpp | 4 ++-- src/Storages/StorageMerge.cpp | 2 ++ 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 6901b6cb5ff4..646d99326e3b 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -140,6 +140,9 @@ class IMergeTreeDataPart; using ManyExpressionActions = std::vector; +struct StorageSnapshot; +using StorageSnapshotPtr = std::shared_ptr; + /** Query along with some additional data, * that can be used during query processing * inside storage engines. @@ -173,6 +176,13 @@ struct SelectQueryInfo /// Local storage limits StorageLimits local_storage_limits; + /// This is a leak of abstraction. + /// StorageMerge replaces storage into query_tree. However, column types may be changed for inner table. + /// So, resolved query tree might have incompatible types. + /// StorageDistributed uses this query tree to calculate a header, throws if we use storage snapshot. + /// To avoid this, we use initial merge_storage_snapshot. + StorageSnapshotPtr merge_storage_snapshot; + /// Cluster for the query. ClusterPtr cluster; /// Optimized cluster for the query. diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 1e013ff9a0c4..849fa5dbe0bb 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -846,10 +846,10 @@ void StorageDistributed::read( remote_storage_id = StorageID{remote_database, remote_table}; auto query_tree_distributed = buildQueryTreeDistributed(modified_query_info, - storage_snapshot, + query_info.merge_storage_snapshot ? query_info.merge_storage_snapshot : storage_snapshot, remote_storage_id, remote_table_function_ptr); - header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_distributed->toAST(), local_context, SelectQueryOptions(processed_stage).analyze()); + header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree_distributed, local_context, SelectQueryOptions(processed_stage).analyze()); /** For distributed tables we do not need constants in header, since we don't send them to remote servers. * Moreover, constants can break some functions like `hostName` that are constants only for local queries. */ diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 0e75282f4991..87d7d21874ad 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -889,6 +889,8 @@ SelectQueryInfo ReadFromMerge::getModifiedQueryInfo(const ContextMutablePtr & mo SelectQueryInfo modified_query_info = query_info; + modified_query_info.merge_storage_snapshot = merge_storage_snapshot; + if (modified_query_info.planner_context) modified_query_info.planner_context = std::make_shared(modified_context, modified_query_info.planner_context); From 260dec27f4a7ae1131ff0ed169aa6cfb95a2678d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 18 Jun 2024 12:23:20 +0000 Subject: [PATCH 232/254] Add a comment. --- src/Interpreters/ActionsDAG.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 8709bf955a99..c9974fd849c6 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -507,6 +507,9 @@ struct ActionDAGNodes ActionsDAG::NodeRawConstPtrs nodes; }; +/// Helper for query analysis. +/// If project_input is set, all columns not found in inputs should be removed. +/// Now, we do it before adding a step to query plan by calling appendInputsForUnusedColumns. struct ActionsAndProjectInputsFlag { ActionsDAG dag; From ec855651f3dff3ac370d4cafcb917b9b4dbc2c84 Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Tue, 18 Jun 2024 13:54:45 +0000 Subject: [PATCH 233/254] Fix compatibility release check --- tests/ci/compatibility_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index e7fee827320d..bb0c717160e2 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -196,7 +196,7 @@ def url_filter(url): # See https://sourceware.org/glibc/wiki/Glibc%20Timeline max_glibc_version = "" - if "amd64" in check_name: + if "amd64" in check_name or "release" in check_name: max_glibc_version = "2.4" elif "aarch64" in check_name: max_glibc_version = "2.18" # because of build with newer sysroot? From ece3efe09cc584f3ac0320e83e3829684153c1d9 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 18 Jun 2024 16:33:59 +0200 Subject: [PATCH 234/254] Update test.py --- tests/integration/test_keeper_profiler/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_profiler/test.py b/tests/integration/test_keeper_profiler/test.py index 69aaaeb61c8a..848929df086e 100644 --- a/tests/integration/test_keeper_profiler/test.py +++ b/tests/integration/test_keeper_profiler/test.py @@ -59,7 +59,7 @@ def test_profiler(started_cluster): arrayStringConcat( arrayMap(x, y -> concat(x, ': ', y), arrayMap(x -> addressToLine(x), trace), arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace -from system.trace_log where trace_type = ‘Real’ and (trace ilike '%KeeperTCPHandler%' or trace ilike '%KeeperDispatcher%') group by trace); +from system.trace_log where trace_type = ‘Real’ and (trace ilike '%KeeperTCPHandler%' or trace ilike '%KeeperDispatcher%') group by trace order by cnt desc) limit 1; """ ) From ca0da7a481325157b1436d1a997078db7ab2fa77 Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Wed, 29 May 2024 16:09:03 +0000 Subject: [PATCH 235/254] Add query privileges information to the query log. Introduced two additional columns in the system.query_log: used_privileges and missing_privileges. Used_privileges is populated with the privileges that were checked during query execution, and missing_privileges contains required privileges that are missing. --- src/Access/CachedAccessChecking.cpp | 4 +- src/Access/CachedAccessChecking.h | 7 +- src/Access/ContextAccess.cpp | 174 +++++++------ src/Access/ContextAccess.h | 232 +++++++++++++----- .../Access/InterpreterGrantQuery.cpp | 8 +- src/Interpreters/Context.cpp | 19 +- src/Interpreters/Context.h | 29 ++- src/Interpreters/QueryLog.cpp | 7 + src/Interpreters/QueryLog.h | 2 + src/Interpreters/Session.cpp | 2 +- src/Interpreters/executeQuery.cpp | 15 ++ src/Storages/System/StorageSystemColumns.cpp | 2 +- ...8_query_log_privileges_not_empty.reference | 5 + .../03168_query_log_privileges_not_empty.sh | 32 +++ 14 files changed, 382 insertions(+), 156 deletions(-) create mode 100644 tests/queries/0_stateless/03168_query_log_privileges_not_empty.reference create mode 100755 tests/queries/0_stateless/03168_query_log_privileges_not_empty.sh diff --git a/src/Access/CachedAccessChecking.cpp b/src/Access/CachedAccessChecking.cpp index aa8ef6073d3c..0d629e7b77a5 100644 --- a/src/Access/CachedAccessChecking.cpp +++ b/src/Access/CachedAccessChecking.cpp @@ -4,12 +4,12 @@ namespace DB { -CachedAccessChecking::CachedAccessChecking(const std::shared_ptr & access_, AccessFlags access_flags_) +CachedAccessChecking::CachedAccessChecking(const std::shared_ptr & access_, AccessFlags access_flags_) : CachedAccessChecking(access_, AccessRightsElement{access_flags_}) { } -CachedAccessChecking::CachedAccessChecking(const std::shared_ptr & access_, const AccessRightsElement & element_) +CachedAccessChecking::CachedAccessChecking(const std::shared_ptr & access_, const AccessRightsElement & element_) : access(access_), element(element_) { } diff --git a/src/Access/CachedAccessChecking.h b/src/Access/CachedAccessChecking.h index e87c28dd823c..aaeea6ceddcc 100644 --- a/src/Access/CachedAccessChecking.h +++ b/src/Access/CachedAccessChecking.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -13,14 +14,14 @@ class ContextAccess; class CachedAccessChecking { public: - CachedAccessChecking(const std::shared_ptr & access_, AccessFlags access_flags_); - CachedAccessChecking(const std::shared_ptr & access_, const AccessRightsElement & element_); + CachedAccessChecking(const std::shared_ptr & access_, AccessFlags access_flags_); + CachedAccessChecking(const std::shared_ptr & access_, const AccessRightsElement & element_); ~CachedAccessChecking(); bool checkAccess(bool throw_if_denied = true); private: - const std::shared_ptr access; + const std::shared_ptr access; const AccessRightsElement element; bool checked = false; bool result = false; diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 28a825de6cf0..a2807ecc5eac 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -20,6 +20,7 @@ #include #include #include +#include namespace DB @@ -271,7 +272,7 @@ namespace std::shared_ptr ContextAccess::fromContext(const ContextPtr & context) { - return context->getAccess(); + return ContextAccessWrapper::fromContext(context)->getAccess(); } @@ -560,7 +561,7 @@ std::shared_ptr ContextAccess::getAccessRightsWithImplicit() template -bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... args) const +bool ContextAccess::checkAccessImplHelper(const ContextPtr & context, AccessFlags flags, const Args &... args) const { if (user_was_dropped) { @@ -573,8 +574,10 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg if (params.full_access) return true; - auto access_granted = [] + auto access_granted = [&] { + if constexpr (throw_if_denied) + context->addQueryPrivilegesInfo(AccessRightsElement{flags, args...}.toStringWithoutOptions(), true); return true; }; @@ -583,7 +586,10 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg FmtArgs && ...fmt_args [[maybe_unused]]) { if constexpr (throw_if_denied) + { + context->addQueryPrivilegesInfo(AccessRightsElement{flags, args...}.toStringWithoutOptions(), false); throw Exception(error_code, std::move(fmt_string), getUserName(), std::forward(fmt_args)...); + } return false; }; @@ -686,102 +692,102 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg } template -bool ContextAccess::checkAccessImpl(const AccessFlags & flags) const +bool ContextAccess::checkAccessImpl(const ContextPtr & context, const AccessFlags & flags) const { - return checkAccessImplHelper(flags); + return checkAccessImplHelper(context, flags); } template -bool ContextAccess::checkAccessImpl(const AccessFlags & flags, std::string_view database, const Args &... args) const +bool ContextAccess::checkAccessImpl(const ContextPtr & context, const AccessFlags & flags, std::string_view database, const Args &... args) const { - return checkAccessImplHelper(flags, database.empty() ? params.current_database : database, args...); + return checkAccessImplHelper(context, flags, database.empty() ? params.current_database : database, args...); } template -bool ContextAccess::checkAccessImplHelper(const AccessRightsElement & element) const +bool ContextAccess::checkAccessImplHelper(const ContextPtr & context, const AccessRightsElement & element) const { assert(!element.grant_option || grant_option); if (element.isGlobalWithParameter()) { if (element.any_parameter) - return checkAccessImpl(element.access_flags); + return checkAccessImpl(context, element.access_flags); else - return checkAccessImpl(element.access_flags, element.parameter); + return checkAccessImpl(context, element.access_flags, element.parameter); } else if (element.any_database) - return checkAccessImpl(element.access_flags); + return checkAccessImpl(context, element.access_flags); else if (element.any_table) - return checkAccessImpl(element.access_flags, element.database); + return checkAccessImpl(context, element.access_flags, element.database); else if (element.any_column) - return checkAccessImpl(element.access_flags, element.database, element.table); + return checkAccessImpl(context, element.access_flags, element.database, element.table); else - return checkAccessImpl(element.access_flags, element.database, element.table, element.columns); + return checkAccessImpl(context, element.access_flags, element.database, element.table, element.columns); } template -bool ContextAccess::checkAccessImpl(const AccessRightsElement & element) const +bool ContextAccess::checkAccessImpl(const ContextPtr & context, const AccessRightsElement & element) const { if constexpr (grant_option) { - return checkAccessImplHelper(element); + return checkAccessImplHelper(context, element); } else { if (element.grant_option) - return checkAccessImplHelper(element); + return checkAccessImplHelper(context, element); else - return checkAccessImplHelper(element); + return checkAccessImplHelper(context, element); } } template -bool ContextAccess::checkAccessImpl(const AccessRightsElements & elements) const +bool ContextAccess::checkAccessImpl(const ContextPtr & context, const AccessRightsElements & elements) const { for (const auto & element : elements) - if (!checkAccessImpl(element)) + if (!checkAccessImpl(context, element)) return false; return true; } -bool ContextAccess::isGranted(const AccessFlags & flags) const { return checkAccessImpl(flags); } -bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database) const { return checkAccessImpl(flags, database); } -bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table) const { return checkAccessImpl(flags, database, table); } -bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return checkAccessImpl(flags, database, table, column); } -bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } -bool ContextAccess::isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } -bool ContextAccess::isGranted(const AccessRightsElement & element) const { return checkAccessImpl(element); } -bool ContextAccess::isGranted(const AccessRightsElements & elements) const { return checkAccessImpl(elements); } - -bool ContextAccess::hasGrantOption(const AccessFlags & flags) const { return checkAccessImpl(flags); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database) const { return checkAccessImpl(flags, database); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const { return checkAccessImpl(flags, database, table); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return checkAccessImpl(flags, database, table, column); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return checkAccessImpl(flags, database, table, columns); } -bool ContextAccess::hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); } -bool ContextAccess::hasGrantOption(const AccessRightsElement & element) const { return checkAccessImpl(element); } -bool ContextAccess::hasGrantOption(const AccessRightsElements & elements) const { return checkAccessImpl(elements); } - -void ContextAccess::checkAccess(const AccessFlags & flags) const { checkAccessImpl(flags); } -void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database) const { checkAccessImpl(flags, database); } -void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const { checkAccessImpl(flags, database, table); } -void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { checkAccessImpl(flags, database, table, column); } -void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { checkAccessImpl(flags, database, table, columns); } -void ContextAccess::checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { checkAccessImpl(flags, database, table, columns); } -void ContextAccess::checkAccess(const AccessRightsElement & element) const { checkAccessImpl(element); } -void ContextAccess::checkAccess(const AccessRightsElements & elements) const { checkAccessImpl(elements); } - -void ContextAccess::checkGrantOption(const AccessFlags & flags) const { checkAccessImpl(flags); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database) const { checkAccessImpl(flags, database); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const { checkAccessImpl(flags, database, table); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { checkAccessImpl(flags, database, table, column); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { checkAccessImpl(flags, database, table, columns); } -void ContextAccess::checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { checkAccessImpl(flags, database, table, columns); } -void ContextAccess::checkGrantOption(const AccessRightsElement & element) const { checkAccessImpl(element); } -void ContextAccess::checkGrantOption(const AccessRightsElements & elements) const { checkAccessImpl(elements); } +bool ContextAccess::isGranted(const ContextPtr & context, const AccessFlags & flags) const { return checkAccessImpl(context, flags); } +bool ContextAccess::isGranted(const ContextPtr & context, const AccessFlags & flags, std::string_view database) const { return checkAccessImpl(context, flags, database); } +bool ContextAccess::isGranted(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table) const { return checkAccessImpl(context, flags, database, table); } +bool ContextAccess::isGranted(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return checkAccessImpl(context, flags, database, table, column); } +bool ContextAccess::isGranted(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return checkAccessImpl(context, flags, database, table, columns); } +bool ContextAccess::isGranted(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return checkAccessImpl(context, flags, database, table, columns); } +bool ContextAccess::isGranted(const ContextPtr & context, const AccessRightsElement & element) const { return checkAccessImpl(context, element); } +bool ContextAccess::isGranted(const ContextPtr & context, const AccessRightsElements & elements) const { return checkAccessImpl(context, elements); } + +bool ContextAccess::hasGrantOption(const ContextPtr & context, const AccessFlags & flags) const { return checkAccessImpl(context, flags); } +bool ContextAccess::hasGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database) const { return checkAccessImpl(context, flags, database); } +bool ContextAccess::hasGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table) const { return checkAccessImpl(context, flags, database, table); } +bool ContextAccess::hasGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return checkAccessImpl(context, flags, database, table, column); } +bool ContextAccess::hasGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return checkAccessImpl(context, flags, database, table, columns); } +bool ContextAccess::hasGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return checkAccessImpl(context, flags, database, table, columns); } +bool ContextAccess::hasGrantOption(const ContextPtr & context, const AccessRightsElement & element) const { return checkAccessImpl(context, element); } +bool ContextAccess::hasGrantOption(const ContextPtr & context, const AccessRightsElements & elements) const { return checkAccessImpl(context, elements); } + +void ContextAccess::checkAccess(const ContextPtr & context, const AccessFlags & flags) const { checkAccessImpl(context, flags); } +void ContextAccess::checkAccess(const ContextPtr & context, const AccessFlags & flags, std::string_view database) const { checkAccessImpl(context, flags, database); } +void ContextAccess::checkAccess(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table) const { checkAccessImpl(context, flags, database, table); } +void ContextAccess::checkAccess(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { checkAccessImpl(context, flags, database, table, column); } +void ContextAccess::checkAccess(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { checkAccessImpl(context, flags, database, table, columns); } +void ContextAccess::checkAccess(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { checkAccessImpl(context, flags, database, table, columns); } +void ContextAccess::checkAccess(const ContextPtr & context, const AccessRightsElement & element) const { checkAccessImpl(context, element); } +void ContextAccess::checkAccess(const ContextPtr & context, const AccessRightsElements & elements) const { checkAccessImpl(context, elements); } + +void ContextAccess::checkGrantOption(const ContextPtr & context, const AccessFlags & flags) const { checkAccessImpl(context, flags); } +void ContextAccess::checkGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database) const { checkAccessImpl(context, flags, database); } +void ContextAccess::checkGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table) const { checkAccessImpl(context, flags, database, table); } +void ContextAccess::checkGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { checkAccessImpl(context, flags, database, table, column); } +void ContextAccess::checkGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { checkAccessImpl(context, flags, database, table, columns); } +void ContextAccess::checkGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { checkAccessImpl(context, flags, database, table, columns); } +void ContextAccess::checkGrantOption(const ContextPtr & context, const AccessRightsElement & element) const { checkAccessImpl(context, element); } +void ContextAccess::checkGrantOption(const ContextPtr & context, const AccessRightsElements & elements) const { checkAccessImpl(context, elements); } template -bool ContextAccess::checkAdminOptionImplHelper(const Container & role_ids, const GetNameFunction & get_name_function) const +bool ContextAccess::checkAdminOptionImplHelper(const ContextPtr & context, const Container & role_ids, const GetNameFunction & get_name_function) const { auto show_error = [](int error_code [[maybe_unused]], FormatStringHelper fmt_string [[maybe_unused]], @@ -804,7 +810,7 @@ bool ContextAccess::checkAdminOptionImplHelper(const Container & role_ids, const if (!std::size(role_ids)) return true; - if (isGranted(AccessType::ROLE_ADMIN)) + if (isGranted(context, AccessType::ROLE_ADMIN)) return true; auto info = getRolesInfo(); @@ -840,54 +846,54 @@ bool ContextAccess::checkAdminOptionImplHelper(const Container & role_ids, const } template -bool ContextAccess::checkAdminOptionImpl(const UUID & role_id) const +bool ContextAccess::checkAdminOptionImpl(const ContextPtr & context, const UUID & role_id) const { - return checkAdminOptionImplHelper(to_array(role_id), [this](const UUID & id, size_t) { return access_control->tryReadName(id); }); + return checkAdminOptionImplHelper(context, to_array(role_id), [this](const UUID & id, size_t) { return access_control->tryReadName(id); }); } template -bool ContextAccess::checkAdminOptionImpl(const UUID & role_id, const String & role_name) const +bool ContextAccess::checkAdminOptionImpl(const ContextPtr & context, const UUID & role_id, const String & role_name) const { - return checkAdminOptionImplHelper(to_array(role_id), [&role_name](const UUID &, size_t) { return std::optional{role_name}; }); + return checkAdminOptionImplHelper(context, to_array(role_id), [&role_name](const UUID &, size_t) { return std::optional{role_name}; }); } template -bool ContextAccess::checkAdminOptionImpl(const UUID & role_id, const std::unordered_map & names_of_roles) const +bool ContextAccess::checkAdminOptionImpl(const ContextPtr & context, const UUID & role_id, const std::unordered_map & names_of_roles) const { - return checkAdminOptionImplHelper(to_array(role_id), [&names_of_roles](const UUID & id, size_t) { auto it = names_of_roles.find(id); return (it != names_of_roles.end()) ? it->second : std::optional{}; }); + return checkAdminOptionImplHelper(context, to_array(role_id), [&names_of_roles](const UUID & id, size_t) { auto it = names_of_roles.find(id); return (it != names_of_roles.end()) ? it->second : std::optional{}; }); } template -bool ContextAccess::checkAdminOptionImpl(const std::vector & role_ids) const +bool ContextAccess::checkAdminOptionImpl(const ContextPtr & context, const std::vector & role_ids) const { - return checkAdminOptionImplHelper(role_ids, [this](const UUID & id, size_t) { return access_control->tryReadName(id); }); + return checkAdminOptionImplHelper(context, role_ids, [this](const UUID & id, size_t) { return access_control->tryReadName(id); }); } template -bool ContextAccess::checkAdminOptionImpl(const std::vector & role_ids, const Strings & names_of_roles) const +bool ContextAccess::checkAdminOptionImpl(const ContextPtr & context, const std::vector & role_ids, const Strings & names_of_roles) const { - return checkAdminOptionImplHelper(role_ids, [&names_of_roles](const UUID &, size_t i) { return std::optional{names_of_roles[i]}; }); + return checkAdminOptionImplHelper(context, role_ids, [&names_of_roles](const UUID &, size_t i) { return std::optional{names_of_roles[i]}; }); } template -bool ContextAccess::checkAdminOptionImpl(const std::vector & role_ids, const std::unordered_map & names_of_roles) const +bool ContextAccess::checkAdminOptionImpl(const ContextPtr & context, const std::vector & role_ids, const std::unordered_map & names_of_roles) const { - return checkAdminOptionImplHelper(role_ids, [&names_of_roles](const UUID & id, size_t) { auto it = names_of_roles.find(id); return (it != names_of_roles.end()) ? it->second : std::optional{}; }); + return checkAdminOptionImplHelper(context, role_ids, [&names_of_roles](const UUID & id, size_t) { auto it = names_of_roles.find(id); return (it != names_of_roles.end()) ? it->second : std::optional{}; }); } -bool ContextAccess::hasAdminOption(const UUID & role_id) const { return checkAdminOptionImpl(role_id); } -bool ContextAccess::hasAdminOption(const UUID & role_id, const String & role_name) const { return checkAdminOptionImpl(role_id, role_name); } -bool ContextAccess::hasAdminOption(const UUID & role_id, const std::unordered_map & names_of_roles) const { return checkAdminOptionImpl(role_id, names_of_roles); } -bool ContextAccess::hasAdminOption(const std::vector & role_ids) const { return checkAdminOptionImpl(role_ids); } -bool ContextAccess::hasAdminOption(const std::vector & role_ids, const Strings & names_of_roles) const { return checkAdminOptionImpl(role_ids, names_of_roles); } -bool ContextAccess::hasAdminOption(const std::vector & role_ids, const std::unordered_map & names_of_roles) const { return checkAdminOptionImpl(role_ids, names_of_roles); } +bool ContextAccess::hasAdminOption(const ContextPtr & context, const UUID & role_id) const { return checkAdminOptionImpl(context, role_id); } +bool ContextAccess::hasAdminOption(const ContextPtr & context, const UUID & role_id, const String & role_name) const { return checkAdminOptionImpl(context, role_id, role_name); } +bool ContextAccess::hasAdminOption(const ContextPtr & context, const UUID & role_id, const std::unordered_map & names_of_roles) const { return checkAdminOptionImpl(context, role_id, names_of_roles); } +bool ContextAccess::hasAdminOption(const ContextPtr & context, const std::vector & role_ids) const { return checkAdminOptionImpl(context, role_ids); } +bool ContextAccess::hasAdminOption(const ContextPtr & context, const std::vector & role_ids, const Strings & names_of_roles) const { return checkAdminOptionImpl(context, role_ids, names_of_roles); } +bool ContextAccess::hasAdminOption(const ContextPtr & context, const std::vector & role_ids, const std::unordered_map & names_of_roles) const { return checkAdminOptionImpl(context, role_ids, names_of_roles); } -void ContextAccess::checkAdminOption(const UUID & role_id) const { checkAdminOptionImpl(role_id); } -void ContextAccess::checkAdminOption(const UUID & role_id, const String & role_name) const { checkAdminOptionImpl(role_id, role_name); } -void ContextAccess::checkAdminOption(const UUID & role_id, const std::unordered_map & names_of_roles) const { checkAdminOptionImpl(role_id, names_of_roles); } -void ContextAccess::checkAdminOption(const std::vector & role_ids) const { checkAdminOptionImpl(role_ids); } -void ContextAccess::checkAdminOption(const std::vector & role_ids, const Strings & names_of_roles) const { checkAdminOptionImpl(role_ids, names_of_roles); } -void ContextAccess::checkAdminOption(const std::vector & role_ids, const std::unordered_map & names_of_roles) const { checkAdminOptionImpl(role_ids, names_of_roles); } +void ContextAccess::checkAdminOption(const ContextPtr & context, const UUID & role_id) const { checkAdminOptionImpl(context, role_id); } +void ContextAccess::checkAdminOption(const ContextPtr & context, const UUID & role_id, const String & role_name) const { checkAdminOptionImpl(context, role_id, role_name); } +void ContextAccess::checkAdminOption(const ContextPtr & context, const UUID & role_id, const std::unordered_map & names_of_roles) const { checkAdminOptionImpl(context, role_id, names_of_roles); } +void ContextAccess::checkAdminOption(const ContextPtr & context, const std::vector & role_ids) const { checkAdminOptionImpl(context, role_ids); } +void ContextAccess::checkAdminOption(const ContextPtr & context, const std::vector & role_ids, const Strings & names_of_roles) const { checkAdminOptionImpl(context, role_ids, names_of_roles); } +void ContextAccess::checkAdminOption(const ContextPtr & context, const std::vector & role_ids, const std::unordered_map & names_of_roles) const { checkAdminOptionImpl(context, role_ids, names_of_roles); } void ContextAccess::checkGranteeIsAllowed(const UUID & grantee_id, const IAccessEntity & grantee) const @@ -919,4 +925,10 @@ void ContextAccess::checkGranteesAreAllowed(const std::vector & grantee_id } } +std::shared_ptr ContextAccessWrapper::fromContext(const ContextPtr & context) +{ + return context->getAccess(); +} + + } diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 237c423d2618..465932af1d34 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -4,9 +4,12 @@ #include #include #include +#include +#include #include #include #include +#include #include #include #include @@ -71,59 +74,59 @@ class ContextAccess : public std::enable_shared_from_this /// Checks if a specified access is granted, and throws an exception if not. /// Empty database means the current database. - void checkAccess(const AccessFlags & flags) const; - void checkAccess(const AccessFlags & flags, std::string_view database) const; - void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const; - void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; - void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; - void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; - void checkAccess(const AccessRightsElement & element) const; - void checkAccess(const AccessRightsElements & elements) const; - - void checkGrantOption(const AccessFlags & flags) const; - void checkGrantOption(const AccessFlags & flags, std::string_view database) const; - void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const; - void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; - void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; - void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; - void checkGrantOption(const AccessRightsElement & element) const; - void checkGrantOption(const AccessRightsElements & elements) const; + void checkAccess(const ContextPtr & context, const AccessFlags & flags) const; + void checkAccess(const ContextPtr & context, const AccessFlags & flags, std::string_view database) const; + void checkAccess(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table) const; + void checkAccess(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + void checkAccess(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + void checkAccess(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; + void checkAccess(const ContextPtr & context, const AccessRightsElement & element) const; + void checkAccess(const ContextPtr & context, const AccessRightsElements & elements) const; + + void checkGrantOption(const ContextPtr & context, const AccessFlags & flags) const; + void checkGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database) const; + void checkGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table) const; + void checkGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + void checkGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + void checkGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; + void checkGrantOption(const ContextPtr & context, const AccessRightsElement & element) const; + void checkGrantOption(const ContextPtr & context, const AccessRightsElements & elements) const; /// Checks if a specified access is granted, and returns false if not. /// Empty database means the current database. - bool isGranted(const AccessFlags & flags) const; - bool isGranted(const AccessFlags & flags, std::string_view database) const; - bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table) const; - bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; - bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; - bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; - bool isGranted(const AccessRightsElement & element) const; - bool isGranted(const AccessRightsElements & elements) const; - - bool hasGrantOption(const AccessFlags & flags) const; - bool hasGrantOption(const AccessFlags & flags, std::string_view database) const; - bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const; - bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; - bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; - bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; - bool hasGrantOption(const AccessRightsElement & element) const; - bool hasGrantOption(const AccessRightsElements & elements) const; + bool isGranted(const ContextPtr & context, const AccessFlags & flags) const; + bool isGranted(const ContextPtr & context, const AccessFlags & flags, std::string_view database) const; + bool isGranted(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table) const; + bool isGranted(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + bool isGranted(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + bool isGranted(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; + bool isGranted(const ContextPtr & context, const AccessRightsElement & element) const; + bool isGranted(const ContextPtr & context, const AccessRightsElements & elements) const; + + bool hasGrantOption(const ContextPtr & context, const AccessFlags & flags) const; + bool hasGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database) const; + bool hasGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table) const; + bool hasGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const; + bool hasGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const; + bool hasGrantOption(const ContextPtr & context, const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const; + bool hasGrantOption(const ContextPtr & context, const AccessRightsElement & element) const; + bool hasGrantOption(const ContextPtr & context, const AccessRightsElements & elements) const; /// Checks if a specified role is granted with admin option, and throws an exception if not. - void checkAdminOption(const UUID & role_id) const; - void checkAdminOption(const UUID & role_id, const String & role_name) const; - void checkAdminOption(const UUID & role_id, const std::unordered_map & names_of_roles) const; - void checkAdminOption(const std::vector & role_ids) const; - void checkAdminOption(const std::vector & role_ids, const Strings & names_of_roles) const; - void checkAdminOption(const std::vector & role_ids, const std::unordered_map & names_of_roles) const; + void checkAdminOption(const ContextPtr & context, const UUID & role_id) const; + void checkAdminOption(const ContextPtr & context, const UUID & role_id, const String & role_name) const; + void checkAdminOption(const ContextPtr & context, const UUID & role_id, const std::unordered_map & names_of_roles) const; + void checkAdminOption(const ContextPtr & context, const std::vector & role_ids) const; + void checkAdminOption(const ContextPtr & context, const std::vector & role_ids, const Strings & names_of_roles) const; + void checkAdminOption(const ContextPtr & context, const std::vector & role_ids, const std::unordered_map & names_of_roles) const; /// Checks if a specified role is granted with admin option, and returns false if not. - bool hasAdminOption(const UUID & role_id) const; - bool hasAdminOption(const UUID & role_id, const String & role_name) const; - bool hasAdminOption(const UUID & role_id, const std::unordered_map & names_of_roles) const; - bool hasAdminOption(const std::vector & role_ids) const; - bool hasAdminOption(const std::vector & role_ids, const Strings & names_of_roles) const; - bool hasAdminOption(const std::vector & role_ids, const std::unordered_map & names_of_roles) const; + bool hasAdminOption(const ContextPtr & context, const UUID & role_id) const; + bool hasAdminOption(const ContextPtr & context, const UUID & role_id, const String & role_name) const; + bool hasAdminOption(const ContextPtr & context, const UUID & role_id, const std::unordered_map & names_of_roles) const; + bool hasAdminOption(const ContextPtr & context, const std::vector & role_ids) const; + bool hasAdminOption(const ContextPtr & context, const std::vector & role_ids, const Strings & names_of_roles) const; + bool hasAdminOption(const ContextPtr & context, const std::vector & role_ids, const std::unordered_map & names_of_roles) const; /// Checks if a grantee is allowed for the current user, throws an exception if not. void checkGranteeIsAllowed(const UUID & grantee_id, const IAccessEntity & grantee) const; @@ -142,43 +145,43 @@ class ContextAccess : public std::enable_shared_from_this void calculateAccessRights() const TSA_REQUIRES(mutex); template - bool checkAccessImpl(const AccessFlags & flags) const; + bool checkAccessImpl(const ContextPtr & context, const AccessFlags & flags) const; template - bool checkAccessImpl(const AccessFlags & flags, std::string_view database, const Args &... args) const; + bool checkAccessImpl(const ContextPtr & context, const AccessFlags & flags, std::string_view database, const Args &... args) const; template - bool checkAccessImpl(const AccessRightsElement & element) const; + bool checkAccessImpl(const ContextPtr & context, const AccessRightsElement & element) const; template - bool checkAccessImpl(const AccessRightsElements & elements) const; + bool checkAccessImpl(const ContextPtr & context, const AccessRightsElements & elements) const; template - bool checkAccessImplHelper(AccessFlags flags, const Args &... args) const; + bool checkAccessImplHelper(const ContextPtr & context, AccessFlags flags, const Args &... args) const; template - bool checkAccessImplHelper(const AccessRightsElement & element) const; + bool checkAccessImplHelper(const ContextPtr & context, const AccessRightsElement & element) const; template - bool checkAdminOptionImpl(const UUID & role_id) const; + bool checkAdminOptionImpl(const ContextPtr & context, const UUID & role_id) const; template - bool checkAdminOptionImpl(const UUID & role_id, const String & role_name) const; + bool checkAdminOptionImpl(const ContextPtr & context, const UUID & role_id, const String & role_name) const; template - bool checkAdminOptionImpl(const UUID & role_id, const std::unordered_map & names_of_roles) const; + bool checkAdminOptionImpl(const ContextPtr & context, const UUID & role_id, const std::unordered_map & names_of_roles) const; template - bool checkAdminOptionImpl(const std::vector & role_ids) const; + bool checkAdminOptionImpl(const ContextPtr & context, const std::vector & role_ids) const; template - bool checkAdminOptionImpl(const std::vector & role_ids, const Strings & names_of_roles) const; + bool checkAdminOptionImpl(const ContextPtr & context, const std::vector & role_ids, const Strings & names_of_roles) const; template - bool checkAdminOptionImpl(const std::vector & role_ids, const std::unordered_map & names_of_roles) const; + bool checkAdminOptionImpl(const ContextPtr & context, const std::vector & role_ids, const std::unordered_map & names_of_roles) const; template - bool checkAdminOptionImplHelper(const Container & role_ids, const GetNameFunction & get_name_function) const; + bool checkAdminOptionImplHelper(const ContextPtr & context, const Container & role_ids, const GetNameFunction & get_name_function) const; const AccessControl * access_control = nullptr; const Params params; @@ -203,4 +206,115 @@ class ContextAccess : public std::enable_shared_from_this mutable std::shared_ptr enabled_settings TSA_GUARDED_BY(mutex); }; +/// This wrapper was added to be able to pass the current context to the access +/// without the need to change the signature and all calls to the ContextAccess itself. +/// Right now a context is used to store privileges that are checked for a query, +/// and might be useful for something else in the future as well. +class ContextAccessWrapper : public std::enable_shared_from_this +{ +public: + using ContextAccessPtr = std::shared_ptr; + + ContextAccessWrapper(const ContextAccessPtr & access_, const ContextPtr & context_): access(access_), context(context_) {} + ~ContextAccessWrapper() = default; + + static std::shared_ptr fromContext(const ContextPtr & context); + + const ContextAccess::Params & getParams() const { return access->getParams(); } + + const ContextAccessPtr & getAccess() const { return access; } + + /// Returns the current user. Throws if user is nullptr. + ALWAYS_INLINE UserPtr getUser() const { return access->getUser(); } + /// Same as above, but can return nullptr. + ALWAYS_INLINE UserPtr tryGetUser() const { return access->tryGetUser(); } + ALWAYS_INLINE String getUserName() const { return access->getUserName(); } + ALWAYS_INLINE std::optional getUserID() const { return access->getUserID(); } + + /// Returns information about current and enabled roles. + ALWAYS_INLINE std::shared_ptr getRolesInfo() const { return access->getRolesInfo(); } + + /// Returns the row policy filter for a specified table. + /// The function returns nullptr if there is no filter to apply. + ALWAYS_INLINE RowPolicyFilterPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const { return access->getRowPolicyFilter(database, table_name, filter_type); } + + /// Returns the quota to track resource consumption. + ALWAYS_INLINE std::shared_ptr getQuota() const { return access->getQuota(); } + ALWAYS_INLINE std::optional getQuotaUsage() const { return access->getQuotaUsage(); } + + /// Returns the default settings, i.e. the settings which should be applied on user's login. + ALWAYS_INLINE SettingsChanges getDefaultSettings() const { return access->getDefaultSettings(); } + ALWAYS_INLINE std::shared_ptr getDefaultProfileInfo() const { return access->getDefaultProfileInfo(); } + + /// Returns the current access rights. + ALWAYS_INLINE std::shared_ptr getAccessRights() const { return access->getAccessRights(); } + ALWAYS_INLINE std::shared_ptr getAccessRightsWithImplicit() const { return access->getAccessRightsWithImplicit(); } + + /// Checks if a specified access is granted, and throws an exception if not. + /// Empty database means the current database. + ALWAYS_INLINE void checkAccess(const AccessFlags & flags) const { access->checkAccess(context, flags); } + ALWAYS_INLINE void checkAccess(const AccessFlags & flags, std::string_view database) const { access->checkAccess(context, flags, database); } + ALWAYS_INLINE void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table) const { access->checkAccess(context, flags, database, table); } + ALWAYS_INLINE void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { access->checkAccess(context, flags, database, table, column); } + ALWAYS_INLINE void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { access->checkAccess(context, flags, database, table, columns); } + ALWAYS_INLINE void checkAccess(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { access->checkAccess(context, flags, database, table, columns); } + ALWAYS_INLINE void checkAccess(const AccessRightsElement & element) const { access->checkAccess(context, element); } + ALWAYS_INLINE void checkAccess(const AccessRightsElements & elements) const { access->checkAccess(context, elements); } + + ALWAYS_INLINE void checkGrantOption(const AccessFlags & flags) const { access->checkGrantOption(context, flags); } + ALWAYS_INLINE void checkGrantOption(const AccessFlags & flags, std::string_view database) const { access->checkGrantOption(context, flags, database); } + ALWAYS_INLINE void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const { access->checkGrantOption(context, flags, database, table); } + ALWAYS_INLINE void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { access->checkGrantOption(context, flags, database, table, column); } + ALWAYS_INLINE void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { access->checkGrantOption(context, flags, database, table, columns); } + ALWAYS_INLINE void checkGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { access->checkGrantOption(context, flags, database, table, columns); } + ALWAYS_INLINE void checkGrantOption(const AccessRightsElement & element) const { access->checkGrantOption(context, element); } + ALWAYS_INLINE void checkGrantOption(const AccessRightsElements & elements) const { access->checkGrantOption(context, elements); } + + /// Checks if a specified access is granted, and returns false if not. + /// Empty database means the current database. + ALWAYS_INLINE bool isGranted(const AccessFlags & flags) const { return access->isGranted(context, flags); } + ALWAYS_INLINE bool isGranted(const AccessFlags & flags, std::string_view database) const { return access->isGranted(context, flags, database); } + ALWAYS_INLINE bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table) const { return access->isGranted(context, flags, database, table); } + ALWAYS_INLINE bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return access->isGranted(context, flags, database, table, column); } + ALWAYS_INLINE bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return access->isGranted(context, flags, database, table, columns); } + ALWAYS_INLINE bool isGranted(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return access->isGranted(context, flags, database, table, columns); } + ALWAYS_INLINE bool isGranted(const AccessRightsElement & element) const { return access->isGranted(context, element); } + ALWAYS_INLINE bool isGranted(const AccessRightsElements & elements) const { return access->isGranted(context, elements); } + + ALWAYS_INLINE bool hasGrantOption(const AccessFlags & flags) const { return access->hasGrantOption(context, flags); } + ALWAYS_INLINE bool hasGrantOption(const AccessFlags & flags, std::string_view database) const { return access->hasGrantOption(context, flags, database); } + ALWAYS_INLINE bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table) const { return access->hasGrantOption(context, flags, database, table); } + ALWAYS_INLINE bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, std::string_view column) const { return access->hasGrantOption(context, flags, database, table, column); } + ALWAYS_INLINE bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const std::vector & columns) const { return access->hasGrantOption(context, flags, database, table, columns); } + ALWAYS_INLINE bool hasGrantOption(const AccessFlags & flags, std::string_view database, std::string_view table, const Strings & columns) const { return access->hasGrantOption(context, flags, database, table, columns); } + ALWAYS_INLINE bool hasGrantOption(const AccessRightsElement & element) const { return access->hasGrantOption(context, element); } + ALWAYS_INLINE bool hasGrantOption(const AccessRightsElements & elements) const { return access->hasGrantOption(context, elements); } + + /// Checks if a specified role is granted with admin option, and throws an exception if not. + ALWAYS_INLINE void checkAdminOption(const UUID & role_id) const { access->checkAdminOption(context, role_id); } + ALWAYS_INLINE void checkAdminOption(const UUID & role_id, const String & role_name) const { access->checkAdminOption(context, role_id, role_name); } + ALWAYS_INLINE void checkAdminOption(const UUID & role_id, const std::unordered_map & names_of_roles) const { access->checkAdminOption(context, role_id, names_of_roles); } + ALWAYS_INLINE void checkAdminOption(const std::vector & role_ids) const { access->checkAdminOption(context, role_ids); } + ALWAYS_INLINE void checkAdminOption(const std::vector & role_ids, const Strings & names_of_roles) const { access->checkAdminOption(context, role_ids, names_of_roles); } + ALWAYS_INLINE void checkAdminOption(const std::vector & role_ids, const std::unordered_map & names_of_roles) const { access->checkAdminOption(context, role_ids, names_of_roles); } + + /// Checks if a specified role is granted with admin option, and returns false if not. + ALWAYS_INLINE bool hasAdminOption(const UUID & role_id) const { return access->hasAdminOption(context, role_id); } + ALWAYS_INLINE bool hasAdminOption(const UUID & role_id, const String & role_name) const { return access->hasAdminOption(context, role_id, role_name); } + ALWAYS_INLINE bool hasAdminOption(const UUID & role_id, const std::unordered_map & names_of_roles) const { return access->hasAdminOption(context, role_id, names_of_roles); } + ALWAYS_INLINE bool hasAdminOption(const std::vector & role_ids) const { return access->hasAdminOption(context, role_ids); } + ALWAYS_INLINE bool hasAdminOption(const std::vector & role_ids, const Strings & names_of_roles) const { return access->hasAdminOption(context, role_ids, names_of_roles); } + ALWAYS_INLINE bool hasAdminOption(const std::vector & role_ids, const std::unordered_map & names_of_roles) const { return access->hasAdminOption(context, role_ids, names_of_roles); } + + /// Checks if a grantee is allowed for the current user, throws an exception if not. + ALWAYS_INLINE void checkGranteeIsAllowed(const UUID & grantee_id, const IAccessEntity & grantee) const { access->checkGranteeIsAllowed(grantee_id, grantee); } + /// Checks if grantees are allowed for the current user, throws an exception if not. + ALWAYS_INLINE void checkGranteesAreAllowed(const std::vector & grantee_ids) const { access->checkGranteesAreAllowed(grantee_ids); } + +private: + ContextAccessPtr access; + ContextPtr context; +}; + + } diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index 6a46ac9c330a..ac3b549a5767 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -118,7 +118,7 @@ namespace /// Checks if the current user has enough access rights granted with grant option to grant or revoke specified access rights. void checkGrantOption( const AccessControl & access_control, - const ContextAccess & current_user_access, + const ContextAccessWrapper & current_user_access, const std::vector & grantees_from_query, bool & need_check_grantees_are_allowed, const AccessRightsElements & elements_to_grant, @@ -200,7 +200,7 @@ namespace /// Checks if the current user has enough roles granted with admin option to grant or revoke specified roles. void checkAdminOption( const AccessControl & access_control, - const ContextAccess & current_user_access, + const ContextAccessWrapper & current_user_access, const std::vector & grantees_from_query, bool & need_check_grantees_are_allowed, const std::vector & roles_to_grant, @@ -277,7 +277,7 @@ namespace /// This function is less accurate than checkAdminOption() because it cannot use any information about /// granted roles the grantees currently have (due to those grantees are located on multiple nodes, /// we just don't have the full information about them). - void checkAdminOptionForExecutingOnCluster(const ContextAccess & current_user_access, + void checkAdminOptionForExecutingOnCluster(const ContextAccessWrapper & current_user_access, const std::vector roles_to_grant, const RolesOrUsersSet & roles_to_revoke) { @@ -376,7 +376,7 @@ namespace /// Calculates all available rights to grant with current user intersection. void calculateCurrentGrantRightsWithIntersection( AccessRights & rights, - std::shared_ptr current_user_access, + std::shared_ptr current_user_access, const AccessRightsElements & elements_to_grant) { AccessRightsElements current_user_grantable_elements; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4b0ebc008ea3..2807807b2941 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -835,6 +835,7 @@ ContextMutablePtr Context::createGlobal(ContextSharedPart * shared_part) auto res = std::shared_ptr(new Context); res->shared = shared_part; res->query_access_info = std::make_shared(); + res->query_privileges_info = std::make_shared(); return res; } @@ -1427,7 +1428,7 @@ void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, void Context::checkAccess(const AccessRightsElement & element) const { checkAccessImpl(element); } void Context::checkAccess(const AccessRightsElements & elements) const { checkAccessImpl(elements); } -std::shared_ptr Context::getAccess() const +std::shared_ptr Context::getAccess() const { /// A helper function to collect parameters for calculating access rights, called with Context::getLocalSharedLock() acquired. auto get_params = [this]() @@ -1444,14 +1445,14 @@ std::shared_ptr Context::getAccess() const { SharedLockGuard lock(mutex); if (access && !need_recalculate_access) - return access; /// No need to recalculate access rights. + return std::make_shared(access, shared_from_this()); /// No need to recalculate access rights. params.emplace(get_params()); if (access && (access->getParams() == *params)) { need_recalculate_access = false; - return access; /// No need to recalculate access rights. + return std::make_shared(access, shared_from_this()); /// No need to recalculate access rights. } } @@ -1471,7 +1472,7 @@ std::shared_ptr Context::getAccess() const } } - return res; + return std::make_shared(res, shared_from_this()); } RowPolicyFilterPtr Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const @@ -1857,6 +1858,15 @@ void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String } } +void Context::addQueryPrivilegesInfo(const String & privilege, bool granted) const +{ + std::lock_guard lock(query_privileges_info->mutex); + if (granted) + query_privileges_info->used_privileges.emplace(privilege); + else + query_privileges_info->missing_privileges.emplace(privilege); +} + static bool findIdentifier(const ASTFunction * function) { if (!function || !function->arguments) @@ -2538,6 +2548,7 @@ void Context::makeQueryContext() local_read_query_throttler.reset(); local_write_query_throttler.reset(); backups_query_throttler.reset(); + query_privileges_info = std::make_shared(*query_privileges_info); } void Context::makeQueryContextForMerge(const MergeTreeSettings & merge_tree_settings) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f3073ccc09c4..b3ade94ccdc5 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -50,6 +50,7 @@ class ASTSelectQuery; struct ContextSharedPart; class ContextAccess; +class ContextAccessWrapper; struct User; using UserPtr = std::shared_ptr; struct SettingsProfilesInfo; @@ -403,9 +404,31 @@ class ContextData mutable std::mutex mutex; }; + struct QueryPrivilegesInfo + { + QueryPrivilegesInfo() = default; + + QueryPrivilegesInfo(const QueryPrivilegesInfo & rhs) + { + std::lock_guard lock(rhs.mutex); + used_privileges = rhs.used_privileges; + missing_privileges = rhs.missing_privileges; + } + + QueryPrivilegesInfo(QueryPrivilegesInfo && rhs) = delete; + + std::unordered_set used_privileges TSA_GUARDED_BY(mutex); + std::unordered_set missing_privileges TSA_GUARDED_BY(mutex); + + mutable std::mutex mutex; + }; + + using QueryPrivilegesInfoPtr = std::shared_ptr; + protected: /// Needs to be changed while having const context in factories methods mutable QueryFactoriesInfo query_factories_info; + QueryPrivilegesInfoPtr query_privileges_info; /// Query metrics for reading data asynchronously with IAsynchronousReader. mutable std::shared_ptr async_read_counters; @@ -612,7 +635,7 @@ class Context: public ContextData, public std::enable_shared_from_this void checkAccess(const AccessRightsElement & element) const; void checkAccess(const AccessRightsElements & elements) const; - std::shared_ptr getAccess() const; + std::shared_ptr getAccess() const; RowPolicyFilterPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; @@ -741,6 +764,10 @@ class Context: public ContextData, public std::enable_shared_from_this QueryFactoriesInfo getQueryFactoriesInfo() const; void addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const; + const QueryPrivilegesInfo & getQueryPrivilegesInfo() const { return *getQueryPrivilegesInfoPtr(); } + QueryPrivilegesInfoPtr getQueryPrivilegesInfoPtr() const { return query_privileges_info; } + void addQueryPrivilegesInfo(const String & privilege, bool granted) const; + /// For table functions s3/file/url/hdfs/input we can use structure from /// insertion table depending on select expression. StoragePtr executeTableFunction(const ASTPtr & table_expression, const ASTSelectQuery * select_query_hint = nullptr); diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 92f8ddae1416..e63a2ae31aab 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -136,6 +136,9 @@ ColumnsDescription QueryLogElement::getColumnsDescription() {"used_row_policies", array_low_cardinality_string, "The list of row policies names that were used during query execution."}, + {"used_privileges", array_low_cardinality_string, "Privileges which were successfully checked during query execution."}, + {"missing_privileges", array_low_cardinality_string, "Privileges that are missing during query execution."}, + {"transaction_id", getTransactionIDDataType(), "The identifier of the transaction in scope of which this query was executed."}, {"query_cache_usage", std::move(query_cache_usage_datatype), "Usage of the query cache during query execution. Values: 'Unknown' = Status unknown, 'None' = The query result was neither written into nor read from the query cache, 'Write' = The query result was written into the query cache, 'Read' = The query result was read from the query cache."}, @@ -267,6 +270,8 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const auto & column_storage_factory_objects = typeid_cast(*columns[i++]); auto & column_table_function_factory_objects = typeid_cast(*columns[i++]); auto & column_row_policies_names = typeid_cast(*columns[i++]); + auto & column_used_privileges = typeid_cast(*columns[i++]); + auto & column_missing_privileges = typeid_cast(*columns[i++]); auto fill_column = [](const auto & data, ColumnArray & column) { @@ -290,6 +295,8 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const fill_column(used_storages, column_storage_factory_objects); fill_column(used_table_functions, column_table_function_factory_objects); fill_column(used_row_policies, column_row_policies_names); + fill_column(used_privileges, column_used_privileges); + fill_column(missing_privileges, column_missing_privileges); } columns[i++]->insert(Tuple{tid.start_csn, tid.local_tid, tid.host_id}); diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 5072d2201608..bbaa71797574 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -81,6 +81,8 @@ struct QueryLogElement std::unordered_set used_storages; std::unordered_set used_table_functions; std::set used_row_policies; + std::unordered_set used_privileges; + std::unordered_set missing_privileges; Int32 exception_code{}; // because ErrorCodes are int String exception; diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 396562189e07..d73593b7f1c5 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -694,7 +694,7 @@ void Session::recordLoginSucess(ContextPtr login_context) const session_log->addLoginSuccess(auth_id, named_session ? named_session->key.second : "", settings, - access, + access->getAccess(), getClientInfo(), user); } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 9c5436517ab9..9f33cbf1c270 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -44,6 +44,7 @@ #include #include +#include #include #include #include @@ -221,6 +222,17 @@ static void logException(ContextPtr context, QueryLogElement & elem, bool log_er LOG_INFO(getLogger("executeQuery"), message); } +static void +addPrivilegesInfoToQueryLogElement(QueryLogElement & element, const ContextPtr context_ptr) +{ + const auto & privileges_info = context_ptr->getQueryPrivilegesInfo(); + { + std::lock_guard lock(privileges_info.mutex); + element.used_privileges = privileges_info.used_privileges; + element.missing_privileges = privileges_info.missing_privileges; + } +} + static void addStatusInfoToQueryLogElement(QueryLogElement & element, const QueryStatusInfo & info, const ASTPtr query_ast, const ContextPtr context_ptr) { @@ -286,6 +298,7 @@ addStatusInfoToQueryLogElement(QueryLogElement & element, const QueryStatusInfo } element.async_read_counters = context_ptr->getAsyncReadCounters(); + addPrivilegesInfoToQueryLogElement(element, context_ptr); } @@ -601,6 +614,8 @@ void logExceptionBeforeStart( elem.formatted_query = queryToString(ast); } + addPrivilegesInfoToQueryLogElement(elem, context); + // We don't calculate databases, tables and columns when the query isn't able to start elem.exception_code = getCurrentExceptionCode(); diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 49da1eba9ec2..8dd8d3b6154b 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -298,7 +298,7 @@ class ColumnsSource : public ISource ClientInfo::Interface client_info_interface; size_t db_table_num = 0; size_t total_tables; - std::shared_ptr access; + std::shared_ptr access; bool need_to_check_access_for_tables; String query_id; std::chrono::milliseconds lock_acquire_timeout; diff --git a/tests/queries/0_stateless/03168_query_log_privileges_not_empty.reference b/tests/queries/0_stateless/03168_query_log_privileges_not_empty.reference new file mode 100644 index 000000000000..e3ac97f9945a --- /dev/null +++ b/tests/queries/0_stateless/03168_query_log_privileges_not_empty.reference @@ -0,0 +1,5 @@ +1 +3168 8613 +[] ['SELECT(a, b) ON default.d_03168_query_log'] +[] [] +['SELECT(a, b) ON default.d_03168_query_log'] [] diff --git a/tests/queries/0_stateless/03168_query_log_privileges_not_empty.sh b/tests/queries/0_stateless/03168_query_log_privileges_not_empty.sh new file mode 100755 index 000000000000..9abc635a8741 --- /dev/null +++ b/tests/queries/0_stateless/03168_query_log_privileges_not_empty.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +user_name="u_03168_query_log" +table_name="default.d_03168_query_log" +test_query="select a, b from ${table_name}" + +${CLICKHOUSE_CLIENT_BINARY} --query "drop user if exists ${user_name}" +${CLICKHOUSE_CLIENT_BINARY} --query "create user ${user_name}" +${CLICKHOUSE_CLIENT_BINARY} --query "drop table if exists ${table_name}" +${CLICKHOUSE_CLIENT_BINARY} --query "create table ${table_name} (a UInt64, b UInt64) order by a" + +${CLICKHOUSE_CLIENT_BINARY} --query "insert into table ${table_name} values (3168, 8613)" + +error="$(${CLICKHOUSE_CLIENT_BINARY} --user ${user_name} --query "${test_query}" 2>&1 >/dev/null)" +echo "${error}" | grep -Fc "ACCESS_DENIED" + +${CLICKHOUSE_CLIENT_BINARY} --query "grant select(a, b) on ${table_name} to ${user_name}" + +${CLICKHOUSE_CLIENT_BINARY} --user ${user_name} --query "${test_query}" + +${CLICKHOUSE_CLIENT_BINARY} --query "system flush logs" +${CLICKHOUSE_CLIENT_BINARY} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'ExceptionBeforeStart' and current_database = currentDatabase() order by event_time desc limit 1" +${CLICKHOUSE_CLIENT_BINARY} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'QueryStart' and current_database = currentDatabase() order by event_time desc limit 1" +${CLICKHOUSE_CLIENT_BINARY} --query "select used_privileges, missing_privileges from system.query_log where query = '${test_query}' and type = 'QueryFinish' and current_database = currentDatabase() order by event_time desc limit 1" + +${CLICKHOUSE_CLIENT_BINARY} --query "drop table ${table_name}" +${CLICKHOUSE_CLIENT_BINARY} --query "drop user ${user_name}" From 13fcfd9784b824fda8dacab9dd608b2916846ca6 Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Wed, 29 May 2024 16:20:10 +0000 Subject: [PATCH 236/254] Add docs for privileges columns to the system.query_log --- docs/en/operations/system-tables/query_log.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 75b855966a31..47094eec3f09 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -113,6 +113,8 @@ Columns: - `used_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `functions`, which were used during query execution. - `used_storages` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `storages`, which were used during query execution. - `used_table_functions` ([Array(String)](../../sql-reference/data-types/array.md)) — Canonical names of `table functions`, which were used during query execution. +- `used_privileges` ([Array(String)](../../sql-reference/data-types/array.md)) - Privileges which were successfully checked during query execution. +- `missing_privileges` ([Array(String)](../../sql-reference/data-types/array.md)) - Privileges that are missing during query execution. - `query_cache_usage` ([Enum8](../../sql-reference/data-types/enum.md)) — Usage of the [query cache](../query-cache.md) during query execution. Values: - `'Unknown'` = Status unknown. - `'None'` = The query result was neither written into nor read from the query cache. @@ -194,6 +196,8 @@ used_formats: [] used_functions: [] used_storages: [] used_table_functions: [] +used_privileges: [] +missing_privileges: [] query_cache_usage: None ``` From 11e844c953380bbfda3b61dcd2dfda60b809f5e2 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 18 Jun 2024 19:12:58 +0000 Subject: [PATCH 237/254] fix tidy build --- src/Common/Scheduler/ISchedulerNode.h | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Common/Scheduler/ISchedulerNode.h b/src/Common/Scheduler/ISchedulerNode.h index 3f699c881fb4..81b491b0edaa 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -406,13 +406,19 @@ class EventQueue void processQueue(std::unique_lock && lock) { if (events.empty()) - return processActivation(std::move(lock)); + { + processActivation(std::move(lock)); + return; + } if (activations.empty()) - return processEvent(std::move(lock)); + { + processEvent(std::move(lock)); + return; + } if (activations.front().activation_event_id < events.front().event_id) - return processActivation(std::move(lock)); + processActivation(std::move(lock)); else - return processEvent(std::move(lock)); + processEvent(std::move(lock)); } void processActivation(std::unique_lock && lock) From 5014cd20fd204c1a34c1fc981553525f2d1c816b Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 18 Jun 2024 21:59:59 +0200 Subject: [PATCH 238/254] CI: Master workflow dependencies fix --- .github/workflows/master.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 88bc50a729d6..384bf6825f9e 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -104,10 +104,9 @@ jobs: with: stage: Tests_2 data: ${{ needs.RunConfig.outputs.data }} - # stage for jobs that do not prohibit merge Tests_3: # Test_3 should not wait for Test_1/Test_2 and should not be blocked by them on master branch since all jobs need to run there. - needs: [RunConfig, Builds_1, Builds_2] + needs: [RunConfig, Builds_1] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_3') }} uses: ./.github/workflows/reusable_test_stage.yml with: From 100916440bcc43df3c1b5ce2b6694ab29a5c8f3a Mon Sep 17 00:00:00 2001 From: Tim MacDonald Date: Wed, 19 Jun 2024 10:34:55 +1000 Subject: [PATCH 239/254] Typo --- docs/en/sql-reference/data-types/datetime.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index ac9a72c2641c..250e766f2b74 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -137,7 +137,7 @@ If the time transition (due to daylight saving time or for other reasons) was pe Non-monotonic calendar dates. For example, in Happy Valley - Goose Bay, the time was transitioned one hour backwards at 00:01:00 7 Nov 2010 (one minute after midnight). So after 6th Nov has ended, people observed a whole one minute of 7th Nov, then time was changed back to 23:01 6th Nov and after another 59 minutes the 7th Nov started again. ClickHouse does not (yet) support this kind of fun. During these days the results of time processing functions may be slightly incorrect. -Similar issue exists for Casey Antarctic station in year 2010. They changed time three hours back at 5 Mar, 02:00. If you are working in antarctic station, please don't afraid to use ClickHouse. Just make sure you set timezone to UTC or be aware of inaccuracies. +Similar issue exists for Casey Antarctic station in year 2010. They changed time three hours back at 5 Mar, 02:00. If you are working in antarctic station, please don't be afraid to use ClickHouse. Just make sure you set timezone to UTC or be aware of inaccuracies. Time shifts for multiple days. Some pacific islands changed their timezone offset from UTC+14 to UTC-12. That's alright but some inaccuracies may present if you do calculations with their timezone for historical time points at the days of conversion. From 466aec959a6681b0634f935501695fcb48673f18 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 18 Jun 2024 14:34:32 +0000 Subject: [PATCH 240/254] Revert "Suppress false positives in leaksan" This reverts commit 2418d673f11d91eed2e2aa477e763769ef7cbd86. --- base/poco/Crypto/src/OpenSSLInitializer.cpp | 21 ++++++--------------- src/IO/S3/Client.cpp | 13 +------------ 2 files changed, 7 insertions(+), 27 deletions(-) diff --git a/base/poco/Crypto/src/OpenSSLInitializer.cpp b/base/poco/Crypto/src/OpenSSLInitializer.cpp index 23447760b475..31798e8dd7e6 100644 --- a/base/poco/Crypto/src/OpenSSLInitializer.cpp +++ b/base/poco/Crypto/src/OpenSSLInitializer.cpp @@ -23,9 +23,6 @@ #include #endif -#if __has_feature(address_sanitizer) -#include -#endif using Poco::RandomInputStream; using Poco::Thread; @@ -70,18 +67,12 @@ void OpenSSLInitializer::initialize() SSL_library_init(); SSL_load_error_strings(); OpenSSL_add_all_algorithms(); - + char seed[SEEDSIZE]; RandomInputStream rnd; rnd.read(seed, sizeof(seed)); - { -# if __has_feature(address_sanitizer) - /// Leak sanitizer (part of address sanitizer) thinks that a few bytes of memory in OpenSSL are allocated during but never released. - __lsan::ScopedDisabler lsan_disabler; -#endif - RAND_seed(seed, SEEDSIZE); - } - + RAND_seed(seed, SEEDSIZE); + int nMutexes = CRYPTO_num_locks(); _mutexes = new Poco::FastMutex[nMutexes]; CRYPTO_set_locking_callback(&OpenSSLInitializer::lock); @@ -89,8 +80,8 @@ void OpenSSLInitializer::initialize() // https://sourceforge.net/p/poco/bugs/110/ // // From http://www.openssl.org/docs/crypto/threads.html : -// "If the application does not register such a callback using CRYPTO_THREADID_set_callback(), -// then a default implementation is used - on Windows and BeOS this uses the system's +// "If the application does not register such a callback using CRYPTO_THREADID_set_callback(), +// then a default implementation is used - on Windows and BeOS this uses the system's // default thread identifying APIs" CRYPTO_set_id_callback(&OpenSSLInitializer::id); CRYPTO_set_dynlock_create_callback(&OpenSSLInitializer::dynlockCreate); @@ -109,7 +100,7 @@ void OpenSSLInitializer::uninitialize() CRYPTO_set_locking_callback(0); CRYPTO_set_id_callback(0); delete [] _mutexes; - + CONF_modules_free(); } } diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 9229342b8c19..55441cfb86b2 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -30,10 +30,6 @@ #include -#ifdef ADDRESS_SANITIZER -#include -#endif - namespace ProfileEvents { extern const Event S3WriteRequestsErrors; @@ -880,14 +876,7 @@ void ClientCacheRegistry::clearCacheForAll() ClientFactory::ClientFactory() { aws_options = Aws::SDKOptions{}; - { -#ifdef ADDRESS_SANITIZER - /// Leak sanitizer (part of address sanitizer) thinks that memory in OpenSSL (called by AWS SDK) is allocated but not - /// released. Actually, the memory is released at the end of the program (ClientFactory is a singleton, see the dtor). - __lsan::ScopedDisabler lsan_disabler; -#endif - Aws::InitAPI(aws_options); - } + Aws::InitAPI(aws_options); Aws::Utils::Logging::InitializeAWSLogging(std::make_shared(false)); Aws::Http::SetHttpClientFactory(std::make_shared()); } From fd52e19b3ee3ade925d9e128040f84ef58fc1820 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 18 Jun 2024 20:01:53 +0000 Subject: [PATCH 241/254] Suppress leaksan false positive in OpenSSL --- contrib/openssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/openssl b/contrib/openssl index e0d6ae2bf93c..277de2ba202a 160000 --- a/contrib/openssl +++ b/contrib/openssl @@ -1 +1 @@ -Subproject commit e0d6ae2bf93cf6dc26bb86aa39992bc6a410869a +Subproject commit 277de2ba202af4eb2291b363456d32ff0960e559 From 619333b356a7328c5404be6ff483632219970913 Mon Sep 17 00:00:00 2001 From: allegrinisante <155471925+allegrinisante@users.noreply.github.com> Date: Wed, 19 Jun 2024 11:44:38 +0200 Subject: [PATCH 242/254] Mode value = 'unordered' may lead to confusion According to the documentation, the default mode was 'ordered' before version 24.6. Starting from version 24.6, there is no default value for mode. Using mode = 'unordered' can be confusing. --- docs/en/engines/table-engines/integrations/s3queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 0958680dc56b..97ca79501a75 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -13,7 +13,7 @@ This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ec CREATE TABLE s3_queue_engine_table (name String, value UInt32) ENGINE = S3Queue(path, [NOSIGN, | aws_access_key_id, aws_secret_access_key,] format, [compression]) [SETTINGS] - [mode = 'unordered',] + [mode = '',] [after_processing = 'keep',] [keeper_path = '',] [s3queue_loading_retries = 0,] From 9d79ec3979b12e03b5540968696b8fe2340e77d1 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 19 Jun 2024 12:21:26 +0200 Subject: [PATCH 243/254] CI: PR workflow dependencies fix --- .github/workflows/pull_request.yml | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 70b71da8fa5e..c9e7ae4450de 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -126,6 +126,8 @@ jobs: with: stage: Builds_2 data: ${{ needs.RunConfig.outputs.data }} + + # stages for jobs that do not prohibit merge Tests_2: needs: [RunConfig, Builds_2] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_2') }} @@ -133,9 +135,8 @@ jobs: with: stage: Tests_2 data: ${{ needs.RunConfig.outputs.data }} - # stage for jobs that do not prohibit merge Tests_3: - needs: [RunConfig, Builds_1, Tests_1, Builds_2, Tests_2] + needs: [RunConfig, Builds_1, Tests_1] if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).stages_data.stages_to_do, 'Tests_3') }} uses: ./.github/workflows/reusable_test_stage.yml with: @@ -156,7 +157,8 @@ jobs: CheckReadyForMerge: if: ${{ !cancelled() && needs.StyleCheck.result == 'success' }} - needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1, Tests_2] + # Test_2 or Test_3 must not have jobs required for Mergeable check + needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_Report, Tests_1] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code @@ -195,8 +197,7 @@ jobs: concurrency: group: jepsen if: ${{ !failure() && !cancelled() && contains(fromJson(needs.RunConfig.outputs.data).jobs_data.jobs_to_do, 'ClickHouse Keeper Jepsen') }} - # jepsen needs binary_release build which is in Builds_2 - needs: [RunConfig, Builds_2] + needs: [RunConfig, Builds_1] uses: ./.github/workflows/reusable_test.yml with: test_name: ClickHouse Keeper Jepsen From 959467f8fa12ea7e369fb392b293cb9aa80cf2f8 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 19 Jun 2024 14:03:24 +0200 Subject: [PATCH 244/254] Update test.py --- tests/integration/test_storage_s3_queue/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 66631c51b038..276df7ff40ab 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -877,7 +877,7 @@ def get_count(): assert "Cannot parse input" in node.query( "SELECT exception FROM system.s3queue WHERE file_name ilike '%fff.csv' ORDER BY processing_end_time DESC LIMIT 1" ) - assert 2 == int( + assert 1 < int( node.query( "SELECT count() FROM system.s3queue_log WHERE file_name ilike '%fff.csv' AND notEmpty(exception)" ) From cb7d97988a21f77c73a7c63de5a2391163af0b67 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 19 Jun 2024 15:07:16 +0200 Subject: [PATCH 245/254] CI: Rename Sync status --- tests/ci/ci_definitions.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 94555158811e..414331829c96 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -208,7 +208,7 @@ class StatusNames(metaclass=WithIter): # mergeable status MERGEABLE = "Mergeable Check" # status of a sync pr - SYNC = "A Sync" + SYNC = "Cloud repo sync (only for Clickhouse employees)" # PR formatting check status PR_CHECK = "PR Check" From 84d282c5738414b18b6f5a7cc269c018b75f3143 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 19 Jun 2024 17:29:24 +0200 Subject: [PATCH 246/254] CI: Rename sync status --- tests/ci/ci_definitions.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 414331829c96..d60c67d318dd 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -208,7 +208,7 @@ class StatusNames(metaclass=WithIter): # mergeable status MERGEABLE = "Mergeable Check" # status of a sync pr - SYNC = "Cloud repo sync (only for Clickhouse employees)" + SYNC = "Cloud fork sync (only for ClickHouse Inc. employees)" # PR formatting check status PR_CHECK = "PR Check" From b49908a808cacda400e813b3fb39b72aa8a62584 Mon Sep 17 00:00:00 2001 From: slvrtrn Date: Wed, 19 Jun 2024 19:42:09 +0200 Subject: [PATCH 247/254] Update MySQL interface docs - multiple users in CH Cloud --- docs/en/interfaces/mysql.md | 50 +++++++++++++++++++++++++++++++++++++ 1 file changed, 50 insertions(+) diff --git a/docs/en/interfaces/mysql.md b/docs/en/interfaces/mysql.md index ce5ab24ecb08..428205054063 100644 --- a/docs/en/interfaces/mysql.md +++ b/docs/en/interfaces/mysql.md @@ -31,6 +31,56 @@ Alternatively, in order to enable the MySQL interface for an existing service: 3. After entering the password, you will get prompted the MySQL connection string for this service ![Connection screen - MySQL Enabled](./images/mysql5.png) +## Creating multiple MySQL users in ClickHouse Cloud + +By default, there is a built-in `mysql4` user, which uses the same password as the `default` one. The `` part is the first segment of your ClickHouse Cloud hostname. This format is necessary to work with the tools that implement secure connection, but don't provide [SNI information in their TLS handshake](https://www.cloudflare.com/learning/ssl/what-is-sni), which makes it impossible to do the internal routing without an extra hint in the username (MySQL console client is one of such tools). + +Because of this, we _highly recommend_ following the `mysql4_` format when creating a new user intended to be used with the MySQL interface, where `` is a hint to identify your Cloud service, and `` is an arbitrary suffix of your choice. + +:::tip +For ClickHouse Cloud hostname like `foobar.us-east1.aws.clickhouse.cloud`, the `` part equals to `foobar`, and a custom MySQL username could look like `mysql4foobar_team1`. +::: + +You can create extra users to use with the MySQL interface if, for example, you need to apply extra settings. + +1. Optional - create a [settings profile](https://clickhouse.com/docs/en/sql-reference/statements/create/settings-profile) to apply for your custom user. For example, `my_custom_profile` with an extra setting which will be applied by default when we connect with the user we create later: + + ```sql + CREATE SETTINGS PROFILE my_custom_profile SETTINGS prefer_column_name_to_alias=1; + ``` + + `prefer_column_name_to_alias` is used just as an example, you can use other settings there. +2. [Create a user](https://clickhouse.com/docs/en/sql-reference/statements/create/user) using the following format: `mysql4_` ([see above](#creating-multiple-mysql-users-in-clickhouse-cloud)). The password must be in double SHA1 format. For example: + + ```sql + CREATE USER mysql4foobar_team1 IDENTIFIED WITH double_sha1_password BY 'YourPassword42$'; + ``` + + or if you want to use a custom profile for this user: + + ```sql + CREATE USER mysql4foobar_team1 IDENTIFIED WITH double_sha1_password BY 'YourPassword42$' SETTINGS PROFILE 'my_custom_profile'; + ``` + + where `my_custom_profile` is the name of the profile you created earlier. +3. [Grant](https://clickhouse.com/docs/en/sql-reference/statements/grant) the new user the necessary permissions to interact with the desired tables or databases. For example, if you want to grant access to `system.query_log` only: + + ```sql + GRANT SELECT ON system.query_log TO mysql4foobar_team1; + ``` + +4. Use the created user to connect to your ClickHouse Cloud service with the MySQL interface. + +### Troubleshooting multiple MySQL users in ClickHouse Cloud + +If you created a new MySQL user, and you see the following error while connecting via MySQL CLI client: + +``` +ERROR 2013 (HY000): Lost connection to MySQL server at 'reading authorization packet', system error: 54 +``` + +In this case, ensure that the username follows the `mysql4_` format, as described ([above](#creating-multiple-mysql-users-in-clickhouse-cloud)). + ## Enabling the MySQL Interface On Self-managed ClickHouse Add the [mysql_port](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-mysql_port) setting to your server's configuration file. For example, you could define the port in a new XML file in your `config.d/` [folder](../operations/configuration-files): From fc8c1030454524974e1033b7b4ede2970d0b22a1 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 19 Jun 2024 18:17:18 +0000 Subject: [PATCH 248/254] Fix 01926_order_by_desc_limit test for 1MB HTTP buffers --- tests/queries/0_stateless/01926_order_by_desc_limit.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01926_order_by_desc_limit.sql b/tests/queries/0_stateless/01926_order_by_desc_limit.sql index a0047a2925ac..89aeb5b22e5a 100644 --- a/tests/queries/0_stateless/01926_order_by_desc_limit.sql +++ b/tests/queries/0_stateless/01926_order_by_desc_limit.sql @@ -12,10 +12,10 @@ INSERT INTO order_by_desc SELECT number, repeat('a', 1024) FROM numbers(1024 * 3 OPTIMIZE TABLE order_by_desc FINAL; SELECT s FROM order_by_desc ORDER BY u DESC LIMIT 10 FORMAT Null -SETTINGS max_memory_usage = '400M'; +SETTINGS max_memory_usage = '600M'; SELECT s FROM order_by_desc ORDER BY u LIMIT 10 FORMAT Null -SETTINGS max_memory_usage = '400M'; +SETTINGS max_memory_usage = '600M'; SYSTEM FLUSH LOGS; From c5b16f4b8d790b901ab597dbe231bdadee8b39a6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 20 Jun 2024 09:26:07 +0200 Subject: [PATCH 249/254] Revert "Merge pull request #65384 from yariks5s/docs_for_group_concat" This reverts commit ab7ba4efd993084523a33889ba6848c7c417869f, reversing changes made to 87b91c611567f86b5282a3e9508446c1e0ebd0d8. --- .../reference/groupconcat.md | 90 ------------------- .../aspell-ignore/en/aspell-dict.txt | 2 - 2 files changed, 92 deletions(-) delete mode 100644 docs/en/sql-reference/aggregate-functions/reference/groupconcat.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md b/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md deleted file mode 100644 index 072252de8c9f..000000000000 --- a/docs/en/sql-reference/aggregate-functions/reference/groupconcat.md +++ /dev/null @@ -1,90 +0,0 @@ ---- -slug: /en/sql-reference/aggregate-functions/reference/groupconcat -sidebar_position: 363 -sidebar_label: groupConcat -title: groupConcat ---- - -Calculates a concatenated string from a group of strings, optionally separated by a delimiter, and optionally limited by a maximum number of elements. - -**Syntax** - -``` sql -groupConcat(expression [, delimiter] [, limit]); -``` - -**Arguments** - -- `expression` — The expression or column name that outputs strings to be concatenated.. -- `delimiter` — A [string](../../../sql-reference/data-types/string.md) that will be used to separate concatenated values. This parameter is optional and defaults to an empty string if not specified. -- `limit` — A positive [integer](../../../sql-reference/data-types/int-uint.md) specifying the maximum number of elements to concatenate. If more elements are present, excess elements are ignored. This parameter is optional. - -:::note -If delimiter is specified without limit, it must be the first parameter following the expression. If both delimiter and limit are specified, delimiter must precede limit. -::: - -**Returned value** - -- Returns a [string](../../../sql-reference/data-types/string.md) consisting of the concatenated values of the column or expression. If the group has no elements or only null elements, and the function does not specify a handling for only null values, the result is a nullable string with a null value. - -**Examples** - -Input table: - -``` text -┌─id─┬─name─┐ -│ 1 │ John│ -│ 2 │ Jane│ -│ 3 │ Bob│ -└────┴──────┘ -``` - -1. Basic usage without a delimiter: - -Query: - -``` sql -SELECT groupConcat(Name) FROM Employees; -``` - -Result: - -``` text -JohnJaneBob -``` - -This concatenates all names into one continuous string without any separator. - - -2. Using comma as a delimiter: - -Query: - -``` sql -SELECT groupConcat(Name, ', ', 2) FROM Employees; -``` - -Result: - -``` text -John, Jane, Bob -``` - -This output shows the names separated by a comma followed by a space. - - -3. Limiting the number of concatenated elements - -Query: - -``` sql -SELECT groupConcat(Name, ', ', 2) FROM Employees; -``` - -Result: - -``` text -John, Jane -``` - -This query limits the output to the first two names, even though there are more names in the table. diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a2a2fcd967ab..3967179f0789 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1709,7 +1709,6 @@ groupBitmap groupBitmapAnd groupBitmapOr groupBitmapXor -groupConcat groupUniqArray grouparray grouparrayinsertat @@ -1726,7 +1725,6 @@ groupbitmapor groupbitmapxor groupbitor groupbitxor -groupconcat groupuniqarray grpc grpcio From bbd7e310c6bab5d3b08b82304384bccdbb07bf39 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 20 Jun 2024 09:27:28 +0200 Subject: [PATCH 250/254] Revert "Merge pull request #65009 from yariks5s/group_concat" This reverts commit fb50f9475d314769d9f1d9ea994c1fc6f1417354, reversing changes made to 9894383aef1a4442eefc6ebeda266681505186b7. --- .../AggregateFunctionGroupConcat.cpp | 265 ------------------ .../registerAggregateFunctions.cpp | 2 - .../0_stateless/03156_group_concat.reference | 14 - .../0_stateless/03156_group_concat.sql | 40 --- 4 files changed, 321 deletions(-) delete mode 100644 src/AggregateFunctions/AggregateFunctionGroupConcat.cpp delete mode 100644 tests/queries/0_stateless/03156_group_concat.reference delete mode 100644 tests/queries/0_stateless/03156_group_concat.sql diff --git a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp deleted file mode 100644 index 7541d64af4a3..000000000000 --- a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp +++ /dev/null @@ -1,265 +0,0 @@ -#include -#include -#include - -#include -#include -#include - -#include -#include - -#include -#include -#include - -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ -struct Settings; - -namespace ErrorCodes -{ - extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int BAD_ARGUMENTS; -} - -namespace -{ - -struct GroupConcatDataBase -{ - UInt64 data_size = 0; - UInt64 allocated_size = 0; - char * data = nullptr; - - void checkAndUpdateSize(UInt64 add, Arena * arena) - { - if (data_size + add >= allocated_size) - { - auto old_size = allocated_size; - allocated_size = std::max(2 * allocated_size, data_size + add); - data = arena->realloc(data, old_size, allocated_size); - } - } - - void insertChar(const char * str, UInt64 str_size, Arena * arena) - { - checkAndUpdateSize(str_size, arena); - memcpy(data + data_size, str, str_size); - data_size += str_size; - } - -}; - -struct GroupConcatData : public GroupConcatDataBase -{ - using Offset = UInt64; - using Allocator = MixedAlignedArenaAllocator; - using Offsets = PODArray; - - /// offset[i * 2] - beginning of the i-th row, offset[i * 2 + 1] - end of the i-th row - Offsets offsets; - UInt64 num_rows = 0; - - UInt64 getSize(size_t i) const { return offsets[i * 2 + 1] - offsets[i * 2]; } - - UInt64 getString(size_t i) const { return offsets[i * 2]; } - - void insert(const IColumn * column, const SerializationPtr & serialization, size_t row_num, Arena * arena) - { - WriteBufferFromOwnString buff; - serialization->serializeText(*column, row_num, buff, {}); - auto string = buff.stringView(); - - checkAndUpdateSize(string.size(), arena); - memcpy(data + data_size, string.data(), string.size()); - offsets.push_back(data_size, arena); - data_size += string.size(); - offsets.push_back(data_size, arena); - num_rows++; - } -}; - -template -class GroupConcatImpl final - : public IAggregateFunctionDataHelper> -{ - static constexpr auto name = "groupConcat"; - - SerializationPtr serialization; - UInt64 limit; - const String delimiter; - -public: - GroupConcatImpl(const DataTypePtr & data_type_, const Array & parameters_, UInt64 limit_, const String & delimiter_) - : IAggregateFunctionDataHelper>( - {data_type_}, parameters_, std::make_shared()) - , serialization(this->argument_types[0]->getDefaultSerialization()) - , limit(limit_) - , delimiter(delimiter_) - { - } - - String getName() const override { return name; } - - void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override - { - auto & cur_data = this->data(place); - - if constexpr (has_limit) - if (cur_data.num_rows >= limit) - return; - - if (cur_data.data_size != 0) - cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena); - - cur_data.insert(columns[0], serialization, row_num, arena); - } - - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override - { - auto & cur_data = this->data(place); - auto & rhs_data = this->data(rhs); - - if (rhs_data.data_size == 0) - return; - - if constexpr (has_limit) - { - UInt64 new_elems_count = std::min(rhs_data.num_rows, limit - cur_data.num_rows); - for (UInt64 i = 0; i < new_elems_count; ++i) - { - if (cur_data.data_size != 0) - cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena); - - cur_data.offsets.push_back(cur_data.data_size, arena); - cur_data.insertChar(rhs_data.data + rhs_data.getString(i), rhs_data.getSize(i), arena); - cur_data.num_rows++; - cur_data.offsets.push_back(cur_data.data_size, arena); - } - } - else - { - if (cur_data.data_size != 0) - cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena); - - cur_data.insertChar(rhs_data.data, rhs_data.data_size, arena); - } - } - - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override - { - auto & cur_data = this->data(place); - - writeVarUInt(cur_data.data_size, buf); - writeVarUInt(cur_data.allocated_size, buf); - - buf.write(cur_data.data, cur_data.data_size); - - if constexpr (has_limit) - { - writeVarUInt(cur_data.num_rows, buf); - for (const auto & offset : cur_data.offsets) - writeVarUInt(offset, buf); - } - } - - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override - { - auto & cur_data = this->data(place); - - readVarUInt(cur_data.data_size, buf); - readVarUInt(cur_data.allocated_size, buf); - - buf.readStrict(cur_data.data, cur_data.data_size); - - if constexpr (has_limit) - { - readVarUInt(cur_data.num_rows, buf); - cur_data.offsets.resize_exact(cur_data.num_rows * 2, arena); - for (auto & offset : cur_data.offsets) - readVarUInt(offset, buf); - } - } - - void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override - { - auto & cur_data = this->data(place); - - if (cur_data.data_size == 0) - { - auto column_nullable = IColumn::mutate(makeNullable(to.getPtr())); - column_nullable->insertDefault(); - return; - } - - auto & column_string = assert_cast(to); - column_string.insertData(cur_data.data, cur_data.data_size); - } - - bool allocatesMemoryInArena() const override { return true; } -}; - -AggregateFunctionPtr createAggregateFunctionGroupConcat( - const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) -{ - assertUnary(name, argument_types); - - bool has_limit = false; - UInt64 limit = 0; - String delimiter; - - if (parameters.size() > 2) - throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, - "Incorrect number of parameters for aggregate function {}, should be 0, 1 or 2, got: {}", name, parameters.size()); - - if (!parameters.empty()) - { - auto type = parameters[0].getType(); - if (type != Field::Types::String) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First parameter for aggregate function {} should be string", name); - - delimiter = parameters[0].get(); - } - if (parameters.size() == 2) - { - auto type = parameters[1].getType(); - - if (type != Field::Types::Int64 && type != Field::Types::UInt64) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number", name); - - if ((type == Field::Types::Int64 && parameters[1].get() <= 0) || - (type == Field::Types::UInt64 && parameters[1].get() == 0)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second parameter for aggregate function {} should be a positive number, got: {}", name, parameters[1].get()); - - has_limit = true; - limit = parameters[1].get(); - } - - if (has_limit) - return std::make_shared>(argument_types[0], parameters, limit, delimiter); - else - return std::make_shared>(argument_types[0], parameters, limit, delimiter); -} - -} - -void registerAggregateFunctionGroupConcat(AggregateFunctionFactory & factory) -{ - AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; - - factory.registerFunction("groupConcat", { createAggregateFunctionGroupConcat, properties }); - factory.registerAlias("group_concat", "groupConcat", AggregateFunctionFactory::CaseInsensitive); -} - -} diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 4ac25e14ee64..58e657d37233 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -19,7 +19,6 @@ void registerAggregateFunctionGroupArraySorted(AggregateFunctionFactory & factor void registerAggregateFunctionGroupUniqArray(AggregateFunctionFactory &); void registerAggregateFunctionGroupArrayInsertAt(AggregateFunctionFactory &); void registerAggregateFunctionGroupArrayIntersect(AggregateFunctionFactory &); -void registerAggregateFunctionGroupConcat(AggregateFunctionFactory &); void registerAggregateFunctionsQuantile(AggregateFunctionFactory &); void registerAggregateFunctionsQuantileDeterministic(AggregateFunctionFactory &); void registerAggregateFunctionsQuantileExact(AggregateFunctionFactory &); @@ -121,7 +120,6 @@ void registerAggregateFunctions() registerAggregateFunctionGroupUniqArray(factory); registerAggregateFunctionGroupArrayInsertAt(factory); registerAggregateFunctionGroupArrayIntersect(factory); - registerAggregateFunctionGroupConcat(factory); registerAggregateFunctionsQuantile(factory); registerAggregateFunctionsQuantileDeterministic(factory); registerAggregateFunctionsQuantileExact(factory); diff --git a/tests/queries/0_stateless/03156_group_concat.reference b/tests/queries/0_stateless/03156_group_concat.reference deleted file mode 100644 index 75b347be0c46..000000000000 --- a/tests/queries/0_stateless/03156_group_concat.reference +++ /dev/null @@ -1,14 +0,0 @@ -0 95 abc [1,2,3] -1 \N a [993,986,979,972] -2 123 makson95 [] -95123 -abcamakson95 -[1,2,3][993,986,979,972][] -95,123 -abc,a,makson95 -[1,2,3],[993,986,979,972] -\N -951239512395123 -abc,a,makson95,abc,a,makson95,abc,a,makson95 -[1,2,3][993,986,979,972][][1,2,3][993,986,979,972][][1,2,3][993,986,979,972][] -488890 diff --git a/tests/queries/0_stateless/03156_group_concat.sql b/tests/queries/0_stateless/03156_group_concat.sql deleted file mode 100644 index c14fde8943aa..000000000000 --- a/tests/queries/0_stateless/03156_group_concat.sql +++ /dev/null @@ -1,40 +0,0 @@ -DROP TABLE IF EXISTS test_groupConcat; -CREATE TABLE test_groupConcat -( - id UInt64, - p_int Int32 NULL, - p_string String, - p_array Array(Int32) -) ENGINE = MergeTree ORDER BY id; - -SET max_insert_threads = 1, max_threads = 1, min_insert_block_size_rows = 0, min_insert_block_size_bytes = 0; -INSERT INTO test_groupConcat VALUES (0, 95, 'abc', [1, 2, 3]), (1, NULL, 'a', [993, 986, 979, 972]), (2, 123, 'makson95', []); - -SELECT * FROM test_groupConcat; - -SELECT groupConcat(p_int) FROM test_groupConcat; -SELECT groupConcat(p_string) FROM test_groupConcat; -SELECT groupConcat(p_array) FROM test_groupConcat; - -SELECT groupConcat(',')(p_int) FROM test_groupConcat; -SELECT groupConcat(',')(p_string) FROM test_groupConcat; -SELECT groupConcat(',', 2)(p_array) FROM test_groupConcat; - -SELECT groupConcat(p_int) FROM test_groupConcat WHERE id = 1; - -INSERT INTO test_groupConcat VALUES (0, 95, 'abc', [1, 2, 3]), (1, NULL, 'a', [993, 986, 979, 972]), (2, 123, 'makson95', []); -INSERT INTO test_groupConcat VALUES (0, 95, 'abc', [1, 2, 3]), (1, NULL, 'a', [993, 986, 979, 972]), (2, 123, 'makson95', []); - -SELECT groupConcat(p_int) FROM test_groupConcat; -SELECT groupConcat(',')(p_string) FROM test_groupConcat; -SELECT groupConcat(p_array) FROM test_groupConcat; - -SELECT groupConcat(123)(number) FROM numbers(10); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT groupConcat(',', '3')(number) FROM numbers(10); -- { serverError BAD_ARGUMENTS } -SELECT groupConcat(',', 0)(number) FROM numbers(10); -- { serverError BAD_ARGUMENTS } -SELECT groupConcat(',', -1)(number) FROM numbers(10); -- { serverError BAD_ARGUMENTS } -SELECT groupConcat(',', 3, 3)(number) FROM numbers(10); -- { serverError TOO_MANY_ARGUMENTS_FOR_FUNCTION } - -SELECT length(groupConcat(number)) FROM numbers(100000); - -DROP TABLE IF EXISTS test_groupConcat; From 0df48b9abfed0dc7c68df0beef50cdc00295c6e3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 20 Jun 2024 08:16:36 +0000 Subject: [PATCH 251/254] Update autogenerated version to 24.7.1.1 and contributors --- cmake/autogenerated_versions.txt | 12 ++++----- .../StorageSystemContributors.generated.cpp | 25 +++++++++++++++++++ 2 files changed, 31 insertions(+), 6 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index dfbbb66a1e90..bb776fa9506f 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -1,12 +1,12 @@ # This variables autochanged by tests/ci/version_helper.py: -# NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, +# NOTE: VERSION_REVISION has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54487) +SET(VERSION_REVISION 54488) SET(VERSION_MAJOR 24) -SET(VERSION_MINOR 6) +SET(VERSION_MINOR 7) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 70a1d3a63d47f0be077d67b8deb907230fc7cfb0) -SET(VERSION_DESCRIBE v24.6.1.1-testing) -SET(VERSION_STRING 24.6.1.1) +SET(VERSION_GITHASH aa023477a9265e403982fca5ee29a714db5133d9) +SET(VERSION_DESCRIBE v24.7.1.1-testing) +SET(VERSION_STRING 24.7.1.1) # end of autochange diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index b42b070d5188..9201eef185f6 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -194,6 +194,7 @@ const char * auto_contributors[] { "Artem Gavrilov", "Artem Hnilov", "Artem Konovalov", + "Artem Mustafin", "Artem Pershin", "Artem Streltsov", "Artem Zuikov", @@ -307,6 +308,7 @@ const char * auto_contributors[] { "Daniil Ivanik", "Daniil Rubin", "Danila Kutenin", + "Danila Puzov", "Daniël van Eeden", "Dao", "Dao Minh Thuc", @@ -417,6 +419,7 @@ const char * auto_contributors[] { "Filippov Denis", "Fille", "Flowyi", + "Francesco Ciocchetti", "Francisco Barón", "Francisco Javier Jurado Moreno", "Frank Chen", @@ -449,6 +452,7 @@ const char * auto_contributors[] { "Gleb-Tretyakov", "GoGoWen2021", "Gregory", + "Grigorii Sokolik", "Grigory", "Grigory Buteyko", "Grigory Pervakov", @@ -464,6 +468,7 @@ const char * auto_contributors[] { "Hamoon", "Han Fei", "Han Shukai", + "HappenLee", "Harry Lee", "Harry-Lee", "HarryLeeIBM", @@ -627,6 +632,7 @@ const char * auto_contributors[] { "Kostiantyn Storozhuk", "Kozlov Ivan", "KrJin", + "Kris Buytaert", "Krisztián Szűcs", "Kruglov Pavel", "Krzysztof Góralski", @@ -644,6 +650,7 @@ const char * auto_contributors[] { "Latysheva Alexandra", "Laurie Li", "LaurieLY", + "Lee sungju", "Lemore", "Leonardo Cecchi", "Leonardo Maciel", @@ -770,6 +777,7 @@ const char * auto_contributors[] { "Mikhail Filimonov", "Mikhail Fursov", "Mikhail Gaidamaka", + "Mikhail Gorshkov", "Mikhail Guzov", "Mikhail Korotov", "Mikhail Koviazin", @@ -904,11 +912,13 @@ const char * auto_contributors[] { "Petr Vasilev", "Pham Anh Tuan", "Philip Hallstrom", + "Philipp Schreiber", "Philippe Ombredanne", "PigInCloud", "Potya", "Pradeep Chhetri", "Prashant Shahi", + "Pratima Patel", "Priyansh Agrawal", "Pxl", "Pysaoke", @@ -978,6 +988,7 @@ const char * auto_contributors[] { "Samuel Colvin", "San", "Sanjam Panda", + "Sariel", "Saulius Valatka", "Sean Haynes", "Sean Lafferty", @@ -1067,6 +1078,7 @@ const char * auto_contributors[] { "TABLUM.IO", "TAC", "TCeason", + "TTPO100AJIEX", "Tagir Kuskarov", "Tai White", "Taleh Zaliyev", @@ -1089,6 +1101,7 @@ const char * auto_contributors[] { "Tiaonmmn", "Tigran Khudaverdyan", "Tim Liou", + "Tim MacDonald", "Tim Windelschmidt", "Timur Magomedov", "Timur Solodovnikov", @@ -1201,6 +1214,7 @@ const char * auto_contributors[] { "Xiaofei Hu", "Xin Wang", "Xoel Lopez Barata", + "Xu Jia", "Xudong Zhang", "Y Lu", "Yakko Majuri", @@ -1237,6 +1251,7 @@ const char * auto_contributors[] { "Yusuke Tanaka", "Zach Naimon", "Zheng Miao", + "ZhiHong Zhang", "ZhiYong Wang", "Zhichang Yu", "Zhichun Wu", @@ -1276,6 +1291,7 @@ const char * auto_contributors[] { "alexeyerm", "alexeypavlenko", "alfredlu", + "allegrinisante", "amesaru", "amoschen", "amudong", @@ -1287,6 +1303,7 @@ const char * auto_contributors[] { "anneji", "anneji-dev", "annvsh", + "anonymous", "anrodigina", "antikvist", "anton", @@ -1346,6 +1363,7 @@ const char * auto_contributors[] { "chenxing-xc", "chenxing.xc", "chertus", + "chloro", "chou.fan", "christophe.kalenzaga", "clarkcaoliu", @@ -1458,6 +1476,7 @@ const char * auto_contributors[] { "gyuton", "hanqf-git", "hao.he", + "haohang", "hardstep33", "hchen9", "hcz", @@ -1479,6 +1498,7 @@ const char * auto_contributors[] { "iammagicc", "ianton-ru", "ice1x", + "iceFireser", "idfer", "ifinik", "igomac", @@ -1642,6 +1662,7 @@ const char * auto_contributors[] { "mo-avatar", "mochi", "monchickey", + "morning-color", "morty", "moscas", "mosinnik", @@ -1695,6 +1716,7 @@ const char * auto_contributors[] { "philip.han", "pingyu", "pkubaj", + "pn", "potya", "pppeace", "presto53", @@ -1742,6 +1764,7 @@ const char * auto_contributors[] { "sanjam", "santaux", "santrancisco", + "sarielwxm", "satanson", "save-my-heart", "sdk2", @@ -1846,6 +1869,7 @@ const char * auto_contributors[] { "whysage", "wineternity", "woodlzm", + "wudidapaopao", "wuxiaobai24", "wxybear", "wzl", @@ -1860,6 +1884,7 @@ const char * auto_contributors[] { "xleoken", "xlwh", "xmy", + "xogoodnow", "xuelei", "xuzifu666", "yakkomajuri", From 9c66375e6849bf2b427900018bb5ae553f4f5ae9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 19 Jun 2024 17:54:40 +0300 Subject: [PATCH 252/254] Support query plan LIMIT optimization --- .../QueryPlan/Optimizations/Optimizations.h | 2 +- .../optimizePrimaryKeyCondition.cpp | 13 +-- .../QueryPlan/Optimizations/optimizeTree.cpp | 2 +- .../QueryPlan/ReadFromMergeTree.cpp | 8 +- .../QueryPlan/ReadFromSystemNumbersStep.cpp | 4 +- .../QueryPlan/ReadFromSystemNumbersStep.h | 2 +- .../QueryPlan/SourceStepWithFilter.h | 9 +- src/Storages/StoragePostgreSQL.cpp | 98 +++++++++++++++---- src/Storages/StoragePostgreSQL.h | 5 +- .../transformQueryForExternalDatabase.cpp | 15 ++- .../transformQueryForExternalDatabase.h | 5 +- 11 files changed, 123 insertions(+), 40 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index b33a373a970e..b1ab5561958b 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -107,7 +107,7 @@ struct Frame using Stack = std::vector; /// Second pass optimizations -void optimizePrimaryKeyCondition(const Stack & stack); +void optimizePrimaryKeyConditionAndLimit(const Stack & stack); void optimizePrewhere(Stack & stack, QueryPlan::Nodes & nodes); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index dbcaf5f00a7a..51f6bc83dacd 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -1,13 +1,13 @@ #include #include #include -#include +#include #include namespace DB::QueryPlanOptimizations { -void optimizePrimaryKeyCondition(const Stack & stack) +void optimizePrimaryKeyConditionAndLimit(const Stack & stack) { const auto & frame = stack.back(); @@ -27,11 +27,12 @@ void optimizePrimaryKeyCondition(const Stack & stack) { if (auto * filter_step = typeid_cast(iter->node->step.get())) source_step_with_filter->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); - - /// Note: actually, plan optimizations merge Filter and Expression steps. - /// Ideally, chain should look like (Expression -> ...) -> (Filter -> ...) -> ReadFromStorage, - /// So this is likely not needed. + else if (auto * limit_step = typeid_cast(iter->node->step.get())) + source_step_with_filter->setLimit(limit_step->getLimitForSorting()); else if (typeid_cast(iter->node->step.get())) + /// Note: actually, plan optimizations merge Filter and Expression steps. + /// Ideally, chain should look like (Expression -> ...) -> (Filter -> ...) -> ReadFromStorage, + /// So this is likely not needed. continue; else break; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 136d474751aa..25895788e2e3 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -115,7 +115,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s while (!stack.empty()) { - optimizePrimaryKeyCondition(stack); + optimizePrimaryKeyConditionAndLimit(stack); /// NOTE: optimizePrewhere can modify the stack. /// Prewhere optimization relies on PK optimization (getConditionSelectivityEstimatorByPredicate) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 0dacdc0b9582..671b3b2e05c9 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -500,11 +500,11 @@ Pipe ReadFromMergeTree::readInOrder( Names required_columns, PoolSettings pool_settings, ReadType read_type, - UInt64 limit) + UInt64 read_limit) { /// For reading in order it makes sense to read only /// one range per task to reduce number of read rows. - bool has_limit_below_one_block = read_type != ReadType::Default && limit && limit < block_size.max_block_size_rows; + bool has_limit_below_one_block = read_type != ReadType::Default && read_limit && read_limit < block_size.max_block_size_rows; MergeTreeReadPoolPtr pool; if (is_parallel_reading_from_replicas) @@ -1672,7 +1672,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead( return std::make_shared(std::move(result)); } -bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, size_t limit) +bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, size_t read_limit) { /// if dirction is not set, use current one if (!direction) @@ -1683,7 +1683,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, if (direction != 1 && query_info.isFinal()) return false; - query_info.input_order_info = std::make_shared(SortDescription{}, prefix_size, direction, limit); + query_info.input_order_info = std::make_shared(SortDescription{}, prefix_size, direction, read_limit); reader_settings.read_in_order = true; /// In case or read-in-order, don't create too many reading streams. diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 11371578c79c..5dbf6fa3318d 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -393,7 +393,7 @@ ReadFromSystemNumbersStep::ReadFromSystemNumbersStep( , num_streams{num_streams_} , limit_length_and_offset(InterpreterSelectQuery::getLimitLengthAndOffset(query_info.query->as(), context)) , should_pushdown_limit(shouldPushdownLimit(query_info, limit_length_and_offset.first)) - , limit(query_info.limit) + , query_info_limit(query_info.limit) , storage_limits(query_info.storage_limits) { storage_snapshot->check(column_names); @@ -563,7 +563,7 @@ Pipe ReadFromSystemNumbersStep::makePipe() { auto rows_appr = (*numbers_storage.limit - 1) / numbers_storage.step + 1; if (limit > 0 && limit < rows_appr) - rows_appr = limit; + rows_appr = query_info_limit; source->addTotalRowsApprox(rows_appr); } diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h index bc84e31be626..e33d67d71507 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h @@ -41,7 +41,7 @@ class ReadFromSystemNumbersStep final : public SourceStepWithFilter size_t num_streams; std::pair limit_length_and_offset; bool should_pushdown_limit; - UInt64 limit; + UInt64 query_info_limit; std::shared_ptr storage_limits; }; diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h index 126d4824fffd..ca4ea4f37044 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.h +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -8,8 +8,9 @@ namespace DB { -/** Source step that can use filters for more efficient pipeline initialization. +/** Source step that can use filters and limit for more efficient pipeline initialization. * Filters must be added before pipeline initialization. + * Limit must be set before pipeline initialization. */ class SourceStepWithFilter : public ISourceStep { @@ -49,6 +50,11 @@ class SourceStepWithFilter : public ISourceStep filter_dags.push_back(std::move(filter_dag)); } + void setLimit(size_t limit_value) + { + limit = limit_value; + } + /// Apply filters that can optimize reading from storage. void applyFilters() { @@ -72,6 +78,7 @@ class SourceStepWithFilter : public ISourceStep PrewhereInfoPtr prewhere_info; StorageSnapshotPtr storage_snapshot; ContextPtr context; + std::optional limit; ActionsDAGPtr filter_actions_dag; diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 9379cb5a1c6c..a8713c61e4de 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -35,9 +35,12 @@ #include #include +#include +#include #include #include +#include #include #include @@ -106,28 +109,79 @@ ColumnsDescription StoragePostgreSQL::getTableStructureFromData( return ColumnsDescription{columns_info->columns}; } -Pipe StoragePostgreSQL::read( - const Names & column_names_, +namespace +{ + +class ReadFromPostgreSQL : public SourceStepWithFilter +{ +public: + ReadFromPostgreSQL( + const Names & column_names_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const ContextPtr & context_, + Block sample_block, + size_t max_block_size_, + String remote_table_schema_, + String remote_table_name_, + postgres::ConnectionHolderPtr connection_) + : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}, column_names_, query_info_, storage_snapshot_, context_) + , logger(getLogger("ReadFromPostgreSQL")) + , max_block_size(max_block_size_) + , remote_table_schema(remote_table_schema_) + , remote_table_name(remote_table_name_) + , connection(std::move(connection_)) + { + } + + std::string getName() const override { return "ReadFromPostgreSQL"; } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override + { + std::optional transform_query_limit; + if (limit && !filter_actions_dag) + transform_query_limit = limit; + + /// Connection is already made to the needed database, so it should not be present in the query; + /// remote_table_schema is empty if it is not specified, will access only table_name. + String query = transformQueryForExternalDatabase( + query_info, + required_source_columns, + storage_snapshot->metadata->getColumns().getOrdinary(), + IdentifierQuotingStyle::DoubleQuotes, + LiteralEscapingStyle::PostgreSQL, + remote_table_schema, + remote_table_name, + context, + transform_query_limit); + LOG_TRACE(logger, "Query: {}", query); + + pipeline.init(Pipe(std::make_shared>(std::move(connection), query, getOutputStream().header, max_block_size))); + } + + LoggerPtr logger; + size_t max_block_size; + String remote_table_schema; + String remote_table_name; + postgres::ConnectionHolderPtr connection; +}; + +} + +void StoragePostgreSQL::read( + QueryPlan & query_plan, + const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info_, - ContextPtr context_, + SelectQueryInfo & query_info, + ContextPtr local_context, QueryProcessingStage::Enum /*processed_stage*/, - size_t max_block_size_, + size_t max_block_size, size_t /*num_streams*/) { - storage_snapshot->check(column_names_); - - /// Connection is already made to the needed database, so it should not be present in the query; - /// remote_table_schema is empty if it is not specified, will access only table_name. - String query = transformQueryForExternalDatabase( - query_info_, - column_names_, - storage_snapshot->metadata->getColumns().getOrdinary(), - IdentifierQuotingStyle::DoubleQuotes, LiteralEscapingStyle::PostgreSQL, remote_table_schema, remote_table_name, context_); - LOG_TRACE(log, "Query: {}", query); + storage_snapshot->check(column_names); Block sample_block; - for (const String & column_name : column_names_) + for (const String & column_name : column_names) { auto column_data = storage_snapshot->metadata->getColumns().getPhysical(column_name); WhichDataType which(column_data.type); @@ -136,7 +190,17 @@ Pipe StoragePostgreSQL::read( sample_block.insert({ column_data.type, column_data.name }); } - return Pipe(std::make_shared>(pool->get(), query, sample_block, max_block_size_)); + auto reading = std::make_unique( + column_names, + query_info, + storage_snapshot, + local_context, + sample_block, + max_block_size, + remote_table_schema, + remote_table_name, + pool->get()); + query_plan.addStep(std::move(reading)); } diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 1ed4f7a7611b..a8fa22f71b22 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -37,11 +37,12 @@ class StoragePostgreSQL final : public IStorage String getName() const override { return "PostgreSQL"; } - Pipe read( + void read( + QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, - ContextPtr context, + ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, size_t num_streams) override; diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index afc458ea6124..fc85bde11d93 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -288,7 +288,8 @@ String transformQueryForExternalDatabaseImpl( LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, - ContextPtr context) + ContextPtr context, + std::optional limit) { bool strict = context->getSettingsRef().external_table_strict_query; @@ -374,6 +375,9 @@ String transformQueryForExternalDatabaseImpl( select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(original_where)); } + if (limit) + select->setExpression(ASTSelectQuery::Expression::LIMIT_LENGTH, std::make_shared(*limit)); + ASTPtr select_ptr = select; dropAliases(select_ptr); @@ -399,7 +403,8 @@ String transformQueryForExternalDatabase( LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, - ContextPtr context) + ContextPtr context, + std::optional limit) { if (!query_info.syntax_analyzer_result) { @@ -424,7 +429,8 @@ String transformQueryForExternalDatabase( literal_escaping_style, database, table, - context); + context, + limit); } auto clone_query = query_info.query->clone(); @@ -436,7 +442,8 @@ String transformQueryForExternalDatabase( literal_escaping_style, database, table, - context); + context, + limit); } } diff --git a/src/Storages/transformQueryForExternalDatabase.h b/src/Storages/transformQueryForExternalDatabase.h index fb6af21907ea..2cd7e3676b5d 100644 --- a/src/Storages/transformQueryForExternalDatabase.h +++ b/src/Storages/transformQueryForExternalDatabase.h @@ -21,6 +21,8 @@ class IAST; * and WHERE contains subset of (AND-ed) conditions from original query, * that contain only compatible expressions. * + * If limit is passed additionally apply LIMIT in result query. + * * Compatible expressions are comparisons of identifiers, constants, and logical operations on them. * * Throws INCORRECT_QUERY if external_table_strict_query (from context settings) @@ -34,6 +36,7 @@ String transformQueryForExternalDatabase( LiteralEscapingStyle literal_escaping_style, const String & database, const String & table, - ContextPtr context); + ContextPtr context, + std::optional limit = {}); } From 894e7c785c26c89c298dd4c85e4841fbf790878c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 19 Jun 2024 19:54:26 +0300 Subject: [PATCH 253/254] Updated implementation --- ...ition.cpp => optimizePrimaryKeyConditionAndLimit.cpp} | 9 +++++++++ 1 file changed, 9 insertions(+) rename src/Processors/QueryPlan/Optimizations/{optimizePrimaryKeyCondition.cpp => optimizePrimaryKeyConditionAndLimit.cpp} (94%) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp similarity index 94% rename from src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp rename to src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp index 51f6bc83dacd..da4e104d18be 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyConditionAndLimit.cpp @@ -26,16 +26,25 @@ void optimizePrimaryKeyConditionAndLimit(const Stack & stack) for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { if (auto * filter_step = typeid_cast(iter->node->step.get())) + { source_step_with_filter->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); + } else if (auto * limit_step = typeid_cast(iter->node->step.get())) + { source_step_with_filter->setLimit(limit_step->getLimitForSorting()); + break; + } else if (typeid_cast(iter->node->step.get())) + { /// Note: actually, plan optimizations merge Filter and Expression steps. /// Ideally, chain should look like (Expression -> ...) -> (Filter -> ...) -> ReadFromStorage, /// So this is likely not needed. continue; + } else + { break; + } } source_step_with_filter->applyFilters(); From a49e31a5c2d81874eb000668033f0015ce8f86be Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Jun 2024 15:01:50 +0200 Subject: [PATCH 254/254] Fix bad test `02922_deduplication_with_zero_copy` --- .../02922_deduplication_with_zero_copy.sh | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh index dad4e6747e0e..d1cbc54d2940 100755 --- a/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh +++ b/tests/queries/0_stateless/02922_deduplication_with_zero_copy.sh @@ -31,11 +31,11 @@ create table r2 (n int) function get_shared_locks() { table_shared_id="$1" - for part in $($CLICKHOUSE_KEEPER_CLIENT -q "ls /clickhouse/zero_copy/zero_copy_s3/${table_shared_id}") + for part in $($CLICKHOUSE_KEEPER_CLIENT -q "ls '/clickhouse/zero_copy/zero_copy_s3/${table_shared_id}'") do - for blob in $($CLICKHOUSE_KEEPER_CLIENT -q "ls /clickhouse/zero_copy/zero_copy_s3/${table_shared_id}/${part}") + for blob in $($CLICKHOUSE_KEEPER_CLIENT -q "ls '/clickhouse/zero_copy/zero_copy_s3/${table_shared_id}/${part}'") do - for lock in $($CLICKHOUSE_KEEPER_CLIENT -q "ls /clickhouse/zero_copy/zero_copy_s3/${table_shared_id}/${part}/${blob}") + for lock in $($CLICKHOUSE_KEEPER_CLIENT -q "ls '/clickhouse/zero_copy/zero_copy_s3/${table_shared_id}/${part}/${blob}'") do echo "/clickhouse/zero_copy/zero_copy_s3/${table_shared_id}/${part}/${blob}/${lock}" done @@ -48,7 +48,7 @@ function filter_temporary_locks() { while read -r lock do - owner="$($CLICKHOUSE_KEEPER_CLIENT -q "get_stat ${lock}" | grep 'ephemeralOwner' | sed 's/.*= //')" + owner="$($CLICKHOUSE_KEEPER_CLIENT -q "get_stat '${lock}'" | grep 'ephemeralOwner' | sed 's/.*= //')" if [[ "${owner}" -eq "0" ]] then echo "${lock}" @@ -111,7 +111,7 @@ export -f insert_duplicates export -f get_shared_locks export -f loop -table_shared_id="$($CLICKHOUSE_KEEPER_CLIENT -q "get /test/02922/${CLICKHOUSE_DATABASE}/table/table_shared_id")" +table_shared_id="$($CLICKHOUSE_KEEPER_CLIENT -q "get '/test/02922/${CLICKHOUSE_DATABASE}/table/table_shared_id'")" exit_code=0 timeout 40 bash -c "loop '${table_shared_id}'" || exit_code="${?}" @@ -128,11 +128,11 @@ function list_keeper_nodes() { table_shared_id=$1 echo "zero_copy:" - $CLICKHOUSE_KEEPER_CLIENT -q "ls /clickhouse/zero_copy/zero_copy_s3" | grep -o "${table_shared_id}" | \ + $CLICKHOUSE_KEEPER_CLIENT -q "ls '/clickhouse/zero_copy/zero_copy_s3'" | grep -o "${table_shared_id}" | \ sed "s/${table_shared_id}//g" || : echo "tables:" - $CLICKHOUSE_KEEPER_CLIENT -q "ls /test/02922/${CLICKHOUSE_DATABASE}" | grep -o "table" || : + $CLICKHOUSE_KEEPER_CLIENT -q "ls '/test/02922/${CLICKHOUSE_DATABASE}'" | grep -o "table" || : } list_keeper_nodes "${table_shared_id}"