From 3c1e6b5d2c52d8f54705428199d007a0f58857bb Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 7 Mar 2024 13:11:42 +0000 Subject: [PATCH 001/318] 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 e27a8bd414b..0041a0f0846 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 4ed0dddc191..0d976bd967a 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 b04d012bcd1..0c2fe1ef12b 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 @@ private: 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 00000000000..b899702561e --- /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 00000000000..d992a14cdd4 --- /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 7de9538e435..22ce3ba9359 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 f82e9e46a61..f140f5274d7 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -23,7 +23,7 @@ protected: void onFinish() override; private: - SquashingTransform squashing; + NewSquashingTransform squashing; Chunk cur_chunk; Chunk finish_chunk; }; @@ -32,7 +32,7 @@ private: 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 @@ protected: 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 002/318] 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 0d976bd967a..a52b54653c1 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 0c2fe1ef12b..fb6834e03be 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 b899702561e..52d24fc9d01 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 d992a14cdd4..8812a0b8c17 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 @@ public: 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 22ce3ba9359..5b68df6b6c6 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 f140f5274d7..5c7ad12889f 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -46,33 +46,4 @@ private: 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 003/318] 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 a52b54653c1..4a259d0b7c9 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 52d24fc9d01..58c2f052c1a 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 8812a0b8c17..1b1d3ec6295 100644 --- a/src/Processors/Transforms/BalancingTransform.h +++ b/src/Processors/Transforms/BalancingTransform.h @@ -16,8 +16,6 @@ public: 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 004/318] 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 fa792e7c8b5..575a78d2f6c 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 8cb59aa12e2..3b452118241 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -5,6 +5,7 @@ #include #include +#include namespace DB @@ -34,6 +35,8 @@ public: } 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 8f9977c0b8d..ec39860b133 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 005/318] 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 fa792e7c8b5..575a78d2f6c 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 192f2f55e6a..9c78bc3c57b 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -5,6 +5,7 @@ #include #include +#include namespace DB @@ -34,6 +35,8 @@ public: } 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 7aad0b010a5..54c05ebc816 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 006/318] 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 575a78d2f6c..6ef01c163d3 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 9c78bc3c57b..0dbea122cf2 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -35,8 +35,6 @@ public: } 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 54c05ebc816..990870da4d7 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 007/318] 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 00000000000..f83d884fd78 --- /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 00000000000..762201ed5fc --- /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 008/318] 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 0dbea122cf2..cf370a523ac 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -5,7 +5,6 @@ #include #include -#include namespace DB @@ -35,6 +34,8 @@ public: } 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 787349ce6f5..2b67bbeb92a 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 009/318] 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 05691529f43..9e176bfd107 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2163,7 +2163,7 @@ public: 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 d7364cb5fd7..c31c8711008 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 fbf2110e664..3ca873bd986 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 010/318] 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 3ca873bd986..7668779356e 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 011/318] 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 767a9b2b9f9..aa7b857c07d 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 7668779356e..83c2aa64add 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 34a009873f8..4501a34cd84 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 6ef01c163d3..4c7400950d6 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 cf370a523ac..7bea09d3d29 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -24,7 +24,10 @@ private: 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 012/318] 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 83c2aa64add..0ac8eec00b7 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 013/318] 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 1d77007a37c..061d6e8420d 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 3ec823a61b5..43c8bcd7993 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 41ce296bd25..b67bbf96e19 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 b060ce8d2c7..df71b954edc 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 0eb460757b1..dfa09272c9c 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 9beb1c39e34..2e9ad5a56c8 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 f1ad9d9738b..505ba049571 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 37483439f14..24f1fcfae77 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 eab9ee082c5..b0debbd408d 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 99b2d1da890..9e8994bfa33 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 852ba50698d..9797cd96157 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 47d706d0b4b..75187828365 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 014/318] 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 aa7b857c07d..17a862ea548 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 015/318] 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 17a862ea548..03d26c5dcb6 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 016/318] 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 0ac8eec00b7..77c15aa7561 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 017/318] 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 77c15aa7561..059744682e5 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 fe9b9adfa2a..b1cd782d36b 100644 --- a/src/Parsers/Kusto/ParserKQLStatement.h +++ b/src/Parsers/Kusto/ParserKQLStatement.h @@ -45,7 +45,7 @@ protected: 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 018/318] fix for one thread --- src/Interpreters/SquashingTransform.cpp | 5 +- src/Interpreters/SquashingTransform.h | 4 + .../Transforms/BalancingTransform.cpp | 86 +++++++++++-------- 3 files changed, 54 insertions(+), 41 deletions(-) diff --git a/src/Interpreters/SquashingTransform.cpp b/src/Interpreters/SquashingTransform.cpp index 4a259d0b7c9..62c819a6105 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 fb6834e03be..4551b76e25f 100644 --- a/src/Interpreters/SquashingTransform.h +++ b/src/Interpreters/SquashingTransform.h @@ -86,6 +86,10 @@ public: 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 58c2f052c1a..a6a79f65ea4 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(); + { + if (output.isFinished()) + continue; - return Status::Finished; - } + all_finished = false; + } - input.setNeeded(); - if (!input.hasData()) - return Status::NeedData; + if (all_finished) + { + input.close(); + return Status::Finished; + } - 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 (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 019/318] 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 a6a79f65ea4..73672be5da4 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 020/318] 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 73672be5da4..deb1abdb2fe 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 021/318] 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 62c819a6105..ca74bb7894a 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 4551b76e25f..b04d12b9bcd 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 @@ private: 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 @@ private: 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 022/318] 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 ca74bb7894a..ebd8a5f0c6e 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 b04d12b9bcd..792a8c2efcf 100644 --- a/src/Interpreters/SquashingTransform.h +++ b/src/Interpreters/SquashingTransform.h @@ -113,4 +113,61 @@ private: 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 1b1d3ec6295..a50b38c773f 100644 --- a/src/Processors/Transforms/BalancingTransform.h +++ b/src/Processors/Transforms/BalancingTransform.h @@ -31,7 +31,7 @@ private: 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 023/318] changed Block -> Chunk and removed dataTypes, beautified --- src/Interpreters/InterpreterInsertQuery.cpp | 14 +- src/Interpreters/SquashingTransform.cpp | 245 +++--------------- src/Interpreters/SquashingTransform.h | 75 +----- .../Transforms/BalancingTransform.cpp | 10 +- .../Transforms/BalancingTransform.h | 2 +- .../Transforms/SquashingChunksTransform.cpp | 11 +- .../Transforms/SquashingChunksTransform.h | 4 +- 7 files changed, 60 insertions(+), 301 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 0041a0f0846..31fef267afc 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 ebd8a5f0c6e..d4370b45119 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)); + 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; -} - -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,6 +183,14 @@ void NewSquashingTransform::append(ReferenceType input_chunk, DataTypes data_typ } } +const ChunksToSquash* NewSquashingTransform::getInfoFromChunk(const Chunk & chunk) +{ + const auto& info = chunk.getChunkInfo(); + const auto * agg_info = typeid_cast(info.get()); + + return agg_info; +} + 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_) @@ -213,31 +203,13 @@ BalanceTransform::BalanceTransform(Block header_, size_t min_block_size_rows_, s Chunk BalanceTransform::add(Block && input_block) { - return addImpl(std::move(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 BalanceTransform::addImpl(Block && 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); @@ -248,7 +220,7 @@ Chunk BalanceTransform::addImpl(ReferenceType input_block) chunks_to_merge_vec.clear(); if (input_chunk) - chunks_to_merge_vec.push_back(input_chunk.clone()); + chunks_to_merge_vec.push_back(std::move(input_chunk)); if (isEnoughSize(chunks_to_merge_vec)) { @@ -258,6 +230,20 @@ Chunk BalanceTransform::addImpl(ReferenceType input_block) return input_chunk; } +Chunk BalanceTransform::convertToChunk(std::vector &chunks) +{ + if (chunks.empty()) + return {}; + + auto info = std::make_shared(); + for (auto &chunk : chunks) + info->chunks.push_back(std::move(chunk)); + + chunks.clear(); // we can remove this + + return Chunk(header.cloneEmptyColumns(), 0, info); +} + bool BalanceTransform::isEnoughSize(const std::vector & chunks) { size_t rows = 0; @@ -268,14 +254,17 @@ bool BalanceTransform::isEnoughSize(const std::vector & chunks) rows += chunk.getNumRows(); bytes += chunk.bytes(); } + + if (!isEnoughSize(rows, bytes)) + return false; + checkAndWaitMemoryAvailability(bytes); - return isEnoughSize(rows, bytes); + return true; } 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(); @@ -284,166 +273,10 @@ void BalanceTransform::checkAndWaitMemoryAvailability(size_t bytes) } } -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 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 792a8c2efcf..ce54c49e441 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 @@ private: 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 @@ private: 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 @@ private: 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 deb1abdb2fe..7a9edbe5d86 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 a50b38c773f..1b1d3ec6295 100644 --- a/src/Processors/Transforms/BalancingTransform.h +++ b/src/Processors/Transforms/BalancingTransform.h @@ -31,7 +31,7 @@ private: 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 5b68df6b6c6..ec226a56548 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 5c7ad12889f..4bcf2216182 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -32,7 +32,7 @@ private: 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 @@ protected: 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 024/318] 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 31fef267afc..412fba0c763 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 7a9edbe5d86..dab73eae2c0 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 1b1d3ec6295..a8a8bc5cfab 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 @@ protected: 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 08ebe3a10d0..55639641c95 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 025/318] empty commit From 0065a4cc6e8ac7eff3e72765f5ae0a7eb593ed2d Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 24 Apr 2024 14:55:01 +0000 Subject: [PATCH 026/318] 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 412fba0c763..06ae92f7f0b 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 ec226a56548..00e430933be 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 4bcf2216182..1db3d46371c 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 @@ protected: 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 dd07d043599..5c16cdbe84c 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 55639641c95..155c32e30ae 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 027/318] 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 0cfff7e3f6c..bb672f5cfbd 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 06ae92f7f0b..412fba0c763 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 00e430933be..ec226a56548 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 1db3d46371c..4bcf2216182 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 @@ protected: 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 5c16cdbe84c..da7f10990e5 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 2f011567b90..b019660cf5e 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 155c32e30ae..14713d151d7 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 b1984a947c8..a3bc09f8d72 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 028/318] 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 bb672f5cfbd..650d606295f 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 029/318] 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 650d606295f..0cfff7e3f6c 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 412fba0c763..50f7bc91d90 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 ec226a56548..62b87061344 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 4bcf2216182..7c7948d1af9 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.h +++ b/src/Processors/Transforms/SquashingChunksTransform.h @@ -22,6 +22,27 @@ protected: 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 da7f10990e5..e05f3643874 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 b019660cf5e..2f011567b90 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 14713d151d7..f6922efc272 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 a3bc09f8d72..b1984a947c8 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 030/318] 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 e05f3643874..804af037c58 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 031/318] empty commit From 0bc664ac5bd0104219e061660cb6cd1cb0698b7c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 29 Apr 2024 18:21:19 +0000 Subject: [PATCH 032/318] 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 50f7bc91d90..a30616b0f3e 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 804af037c58..3b4304dc39b 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 f6922efc272..731ac04a8c8 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 033/318] 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 d4370b45119..dc2ce69ed2f 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 ce54c49e441..95e8c2a8fa1 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 @@ private: 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 3b4304dc39b..93cfc8f6d10 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 034/318] fix problems with ports --- .../Transforms/BalancingTransform.cpp | 70 ++++++++++++------- 1 file changed, 46 insertions(+), 24 deletions(-) diff --git a/src/Processors/Transforms/BalancingTransform.cpp b/src/Processors/Transforms/BalancingTransform.cpp index dab73eae2c0..def12407019 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; + bool all_finished = true; + for (auto & output : outputs) + { + if (output.isFinished()) + continue; + + all_finished = false; + } + + if (all_finished) /// If all outputs are closed, we close inputs (just in case) + { + for (auto & in : inputs) + in.close(); + 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 + { + 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) { - 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() && !balance.isDataLeft()) - { - for (auto & output : outputs) - output.finish(); - - return Status::Finished; - } + 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 035/318] 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 0cfff7e3f6c..1e118bd6a32 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 a30616b0f3e..e39af9e2804 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 dc2ce69ed2f..6063714e8db 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 95e8c2a8fa1..82d7fe616f6 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 @@ private: 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 @@ private: 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 @@ private: 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 74da97f2199..7ca9cd754b2 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 00000000000..26507d9c496 --- /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 def12407019..62ff3a0bf39 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 a8a8bc5cfab..c30569fffb5 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 @@ protected: 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 62b87061344..00000000000 --- 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 7c7948d1af9..00000000000 --- 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 00000000000..43f72262846 --- /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 00000000000..c5b727ac6ec --- /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 93cfc8f6d10..e106cbf8d42 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 d883029408c..08d8b1b9fbc 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 2f011567b90..b274518e248 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 1c33f018a5d..514e7b8299b 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 @@ private: 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 731ac04a8c8..d1bcc89cbe0 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 b1984a947c8..f6ffaf679ed 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 c87a1b216ca..977a803bd28 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 036/318] fix for projections --- src/Interpreters/Squashing.cpp | 10 +--- .../Transforms/PlanSquashingTransform.cpp | 60 +++++++++---------- src/Storages/ProjectionsDescription.cpp | 2 +- 3 files changed, 31 insertions(+), 41 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 6063714e8db..ece124e8a15 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 62ff3a0bf39..fe0f6ed39f5 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 - { - 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()) { + 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 + { + 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; + } + } + 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 d1bcc89cbe0..87e203e8665 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 037/318] 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 038/318] 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 41f024df7a7..00000000000 --- 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 039/318] 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 1e118bd6a32..00000000000 --- 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 040/318] 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 ece124e8a15..78d1b9fc643 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 26507d9c496..584fb72cccb 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 041/318] 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 b94cb188086..2f7568c2bb0 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 042/318] full refactoring of planning --- .../Transforms/PlanSquashingTransform.cpp | 270 ++++++++++++------ .../Transforms/PlanSquashingTransform.h | 26 +- 2 files changed, 203 insertions(+), 93 deletions(-) diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index fe0f6ed39f5..89b59354722 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::init() +{ + for (auto input : inputs) + { + input.setNeeded(); + if (input.hasData()) + available_inputs++; + } + + 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; + for (auto & input : inputs) + { + if (input.hasData()) + { + inputs_have_no_data = false; + chunk = input.pull(); + transform(chunk); + + available_inputs--; + if (chunk.hasChunkInfo()) + { + planning_status = PlanningStatus::WAIT_OUT_AND_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 & output : outputs) { - if (output.isFinished()) + 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); + } + else /// If we don't have data, We send FINISHED + planning_status = PlanningStatus::FINISH; + return true; + } + return false; +} + +IProcessor::Status PlanSquashingTransform::waitForDataIn() +{ + bool all_finished = true; + for (auto & input : inputs) + { + if (input.isFinished()) continue; all_finished = false; - } - if (all_finished) /// If all outputs are closed, we close inputs (just in case) + if (!input.hasData()) + continue; + + available_inputs++; + } + if (all_finished) { - for (auto & in : inputs) - in.close(); - return Status::Finished; + checkInputs(); + return Status::Ready; } - - while (!chunk.hasChunkInfo()) + + if (available_inputs > 0) { - 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 - { - 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; - } - } - - for (auto & input : inputs) - { - if (input.isFinished()) - continue; - - input.setNeeded(); - if (!input.hasData()) - { - if (!balance.isDataLeft()) - return Status::NeedData; - else - continue; - } - - chunk = input.pull(); - transform(chunk); - was_output_processed.assign(outputs.size(), false); - if (chunk.hasChunkInfo()) - { - has_data = true; - return Status::Ready; - } - else - return Status::NeedData; - } + planning_status = PlanningStatus::READ_IF_CAN; + return Status::Ready; } - 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; } - - if (was_processed) - continue; - - if (output.isFinished()) - continue; - - if (!output.canPush()) - { - all_outputs_processed = false; - continue; - } - - output.push(std::move(chunk)); - was_processed = true; - break; } - - if (all_outputs_processed) - { - has_data = false; - return Status::Ready; - } - return Status::PortFull; } + +IProcessor::Status PlanSquashingTransform::prepareSendFlush() +{ + if (!chunk) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should be available in prepareSendFlush"); + + for (auto &output : outputs) + { + + if (output.canPush()) + { + planning_status = PlanningStatus::FINISH; + output.push(std::move(chunk)); + return Status::Ready; + } + } + return Status::PortFull; +} + +IProcessor::Status PlanSquashingTransform::finish() +{ + for (auto & in : inputs) + in.close(); + for (auto & output : outputs) + output.finish(); + + return Status::Finished; +} } diff --git a/src/Processors/Transforms/PlanSquashingTransform.h b/src/Processors/Transforms/PlanSquashingTransform.h index c30569fffb5..a500787ad0c 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.h +++ b/src/Processors/Transforms/PlanSquashingTransform.h @@ -4,9 +4,23 @@ #include #include +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 @@ public: 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 043/318] 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 89b59354722..5125c28fb06 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 a500787ad0c..39f3a70a4a2 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 044/318] 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 5125c28fb06..5600c30b1ba 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 39f3a70a4a2..57c77274863 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.h +++ b/src/Processors/Transforms/PlanSquashingTransform.h @@ -37,6 +37,7 @@ public: 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 045/318] 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 5600c30b1ba..95d3b454a4c 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 57c77274863..55685b0c532 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 @@ public: 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 @@ private: 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 046/318] 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 78d1b9fc643..2d87b47798c 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 82d7fe616f6..0bb6acf9372 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -85,6 +85,7 @@ public: 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 95d3b454a4c..9c42b846a7b 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; + if (inputs_have_no_data) + planning_status = PlanningStatus::WAIT_IN; - 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; + 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 55685b0c532..dc5b6d669b1 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 @@ public: 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 047/318] 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 9c42b846a7b..f4106204462 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 92df0f973772a91cf583a35f7c596fecd95d3751 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 16 May 2024 16:17:07 +0200 Subject: [PATCH 048/318] More parameters for tls layer in protocols --- .../include/Poco/Net/SSLManager.h | 68 +++++++++---------- src/Server/TLSHandler.cpp | 68 +++++++++++++++++++ src/Server/TLSHandler.h | 32 +++++---- src/Server/TLSHandlerFactory.h | 4 +- .../configs/config.xml | 20 ++++++ .../test_composable_protocols/test.py | 33 ++++++++- 6 files changed, 176 insertions(+), 49 deletions(-) create mode 100644 src/Server/TLSHandler.cpp diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h index e4037c87927..4269df95246 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h @@ -258,6 +258,40 @@ namespace Net static const std::string CFG_SERVER_PREFIX; static const std::string CFG_CLIENT_PREFIX; + static const std::string CFG_PRIV_KEY_FILE; + static const std::string CFG_CERTIFICATE_FILE; + static const std::string CFG_CA_LOCATION; + static const std::string CFG_VER_MODE; + static const Context::VerificationMode VAL_VER_MODE; + static const std::string CFG_VER_DEPTH; + static const int VAL_VER_DEPTH; + static const std::string CFG_ENABLE_DEFAULT_CA; + static const bool VAL_ENABLE_DEFAULT_CA; + static const std::string CFG_CIPHER_LIST; + static const std::string CFG_CYPHER_LIST; // for backwards compatibility + static const std::string VAL_CIPHER_LIST; + static const std::string CFG_PREFER_SERVER_CIPHERS; + static const std::string CFG_DELEGATE_HANDLER; + static const std::string VAL_DELEGATE_HANDLER; + static const std::string CFG_CERTIFICATE_HANDLER; + static const std::string VAL_CERTIFICATE_HANDLER; + static const std::string CFG_CACHE_SESSIONS; + static const std::string CFG_SESSION_ID_CONTEXT; + static const std::string CFG_SESSION_CACHE_SIZE; + static const std::string CFG_SESSION_TIMEOUT; + static const std::string CFG_EXTENDED_VERIFICATION; + static const std::string CFG_REQUIRE_TLSV1; + static const std::string CFG_REQUIRE_TLSV1_1; + static const std::string CFG_REQUIRE_TLSV1_2; + static const std::string CFG_DISABLE_PROTOCOLS; + static const std::string CFG_DH_PARAMS_FILE; + static const std::string CFG_ECDH_CURVE; + +#ifdef OPENSSL_FIPS + static const std::string CFG_FIPS_MODE; + static const bool VAL_FIPS_MODE; +#endif + protected: static int verifyClientCallback(int ok, X509_STORE_CTX * pStore); /// The return value of this method defines how errors in @@ -314,40 +348,6 @@ namespace Net InvalidCertificateHandlerPtr _ptrClientCertificateHandler; Poco::FastMutex _mutex; - static const std::string CFG_PRIV_KEY_FILE; - static const std::string CFG_CERTIFICATE_FILE; - static const std::string CFG_CA_LOCATION; - static const std::string CFG_VER_MODE; - static const Context::VerificationMode VAL_VER_MODE; - static const std::string CFG_VER_DEPTH; - static const int VAL_VER_DEPTH; - static const std::string CFG_ENABLE_DEFAULT_CA; - static const bool VAL_ENABLE_DEFAULT_CA; - static const std::string CFG_CIPHER_LIST; - static const std::string CFG_CYPHER_LIST; // for backwards compatibility - static const std::string VAL_CIPHER_LIST; - static const std::string CFG_PREFER_SERVER_CIPHERS; - static const std::string CFG_DELEGATE_HANDLER; - static const std::string VAL_DELEGATE_HANDLER; - static const std::string CFG_CERTIFICATE_HANDLER; - static const std::string VAL_CERTIFICATE_HANDLER; - static const std::string CFG_CACHE_SESSIONS; - static const std::string CFG_SESSION_ID_CONTEXT; - static const std::string CFG_SESSION_CACHE_SIZE; - static const std::string CFG_SESSION_TIMEOUT; - static const std::string CFG_EXTENDED_VERIFICATION; - static const std::string CFG_REQUIRE_TLSV1; - static const std::string CFG_REQUIRE_TLSV1_1; - static const std::string CFG_REQUIRE_TLSV1_2; - static const std::string CFG_DISABLE_PROTOCOLS; - static const std::string CFG_DH_PARAMS_FILE; - static const std::string CFG_ECDH_CURVE; - -#ifdef OPENSSL_FIPS - static const std::string CFG_FIPS_MODE; - static const bool VAL_FIPS_MODE; -#endif - friend class Poco::SingletonHolder; friend class Context; }; diff --git a/src/Server/TLSHandler.cpp b/src/Server/TLSHandler.cpp new file mode 100644 index 00000000000..11d3f4e199d --- /dev/null +++ b/src/Server/TLSHandler.cpp @@ -0,0 +1,68 @@ +#include +#include +#include + +DB::TLSHandler::TLSHandler(const StreamSocket & socket, const LayeredConfiguration & config, const std::string & prefix, TCPProtocolStackData & stack_data_) + : Poco::Net::TCPServerConnection(socket) + , stack_data(stack_data_) +{ +#if USE_SSL + params.privateKeyFile = config.getString(prefix + SSLManager::CFG_PRIV_KEY_FILE, ""); + params.certificateFile = config.getString(prefix + SSLManager::CFG_CERTIFICATE_FILE, params.privateKeyFile); + if (!params.privateKeyFile.empty() && !params.certificateFile.empty()) + { + // for backwards compatibility + auto ctx = SSLManager::instance().defaultServerContext(); + params.caLocation = config.getString(prefix + SSLManager::CFG_CA_LOCATION, ctx->getCAPaths().caLocation); + + // optional options for which we have defaults defined + params.verificationMode = SSLManager::VAL_VER_MODE; + if (config.hasProperty(prefix + SSLManager::CFG_VER_MODE)) + { + // either: none, relaxed, strict, once + std::string mode = config.getString(prefix + SSLManager::CFG_VER_MODE); + params.verificationMode = Poco::Net::Utility::convertVerificationMode(mode); + } + + params.verificationDepth = config.getInt(prefix + SSLManager::CFG_VER_DEPTH, SSLManager::VAL_VER_DEPTH); + params.loadDefaultCAs = config.getBool(prefix + SSLManager::CFG_ENABLE_DEFAULT_CA, SSLManager::VAL_ENABLE_DEFAULT_CA); + params.cipherList = config.getString(prefix + SSLManager::CFG_CIPHER_LIST, SSLManager::VAL_CIPHER_LIST); + params.cipherList = config.getString(prefix + SSLManager::CFG_CYPHER_LIST, params.cipherList); // for backwards compatibility + + bool requireTLSv1 = config.getBool(prefix + SSLManager::CFG_REQUIRE_TLSV1, false); + bool requireTLSv1_1 = config.getBool(prefix + SSLManager::CFG_REQUIRE_TLSV1_1, false); + bool requireTLSv1_2 = config.getBool(prefix + SSLManager::CFG_REQUIRE_TLSV1_2, false); + if (requireTLSv1_2) + usage = Context::TLSV1_2_SERVER_USE; + else if (requireTLSv1_1) + usage = Context::TLSV1_1_SERVER_USE; + else if (requireTLSv1) + usage = Context::TLSV1_SERVER_USE; + else + usage = Context::SERVER_USE; + + params.dhParamsFile = config.getString(prefix + SSLManager::CFG_DH_PARAMS_FILE, ""); + params.ecdhCurve = config.getString(prefix + SSLManager::CFG_ECDH_CURVE, ""); + + std::string disabledProtocolsList = config.getString(prefix + SSLManager::CFG_DISABLE_PROTOCOLS, ""); + Poco::StringTokenizer dpTok(disabledProtocolsList, ";,", Poco::StringTokenizer::TOK_TRIM | Poco::StringTokenizer::TOK_IGNORE_EMPTY); + disabledProtocols = 0; + for (Poco::StringTokenizer::Iterator it = dpTok.begin(); it != dpTok.end(); ++it) + { + if (*it == "sslv2") + disabledProtocols |= Context::PROTO_SSLV2; + else if (*it == "sslv3") + disabledProtocols |= Context::PROTO_SSLV3; + else if (*it == "tlsv1") + disabledProtocols |= Context::PROTO_TLSV1; + else if (*it == "tlsv1_1") + disabledProtocols |= Context::PROTO_TLSV1_1; + else if (*it == "tlsv1_2") + disabledProtocols |= Context::PROTO_TLSV1_2; + } + + extendedVerification = config.getBool(prefix + SSLManager::CFG_EXTENDED_VERIFICATION, false); + preferServerCiphers = config.getBool(prefix + SSLManager::CFG_PREFER_SERVER_CIPHERS, false); + } +#endif +} diff --git a/src/Server/TLSHandler.h b/src/Server/TLSHandler.h index dd025e3e165..5fcb688663a 100644 --- a/src/Server/TLSHandler.h +++ b/src/Server/TLSHandler.h @@ -4,6 +4,9 @@ #include #include #include +#include + +#include "config.h" #if USE_SSL # include @@ -24,33 +27,38 @@ class TLSHandler : public Poco::Net::TCPServerConnection #if USE_SSL using SecureStreamSocket = Poco::Net::SecureStreamSocket; using SSLManager = Poco::Net::SSLManager; - using Context = Poco::Net::Context; #endif + using Context = Poco::Net::Context; using StreamSocket = Poco::Net::StreamSocket; + using LayeredConfiguration = Poco::Util::LayeredConfiguration; public: - explicit TLSHandler(const StreamSocket & socket, const std::string & key_, const std::string & certificate_, TCPProtocolStackData & stack_data_) - : Poco::Net::TCPServerConnection(socket) - , key(key_) - , certificate(certificate_) - , stack_data(stack_data_) - {} + explicit TLSHandler(const StreamSocket & socket, const LayeredConfiguration & config, const std::string & prefix, TCPProtocolStackData & stack_data_); void run() override { #if USE_SSL auto ctx = SSLManager::instance().defaultServerContext(); - if (!key.empty() && !certificate.empty()) - ctx = new Context(Context::Usage::SERVER_USE, key, certificate, ctx->getCAPaths().caLocation); + if (!params.privateKeyFile.empty() && !params.certificateFile.empty()) + { + ctx = new Context(usage, params); + ctx->disableProtocols(disabledProtocols); + ctx->enableExtendedCertificateVerification(extendedVerification); + if (preferServerCiphers) + ctx->preferServerCiphers(); + } socket() = SecureStreamSocket::attach(socket(), ctx); stack_data.socket = socket(); - stack_data.certificate = certificate; + stack_data.certificate = params.certificateFile; #else throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); #endif } private: - std::string key [[maybe_unused]]; - std::string certificate [[maybe_unused]]; + Context::Params params [[maybe_unused]]; + Context::Usage usage [[maybe_unused]]; + int disabledProtocols = 0; + bool extendedVerification = false; + bool preferServerCiphers = false; TCPProtocolStackData & stack_data [[maybe_unused]]; }; diff --git a/src/Server/TLSHandlerFactory.h b/src/Server/TLSHandlerFactory.h index 19602c7d25e..e8f3a1b7853 100644 --- a/src/Server/TLSHandlerFactory.h +++ b/src/Server/TLSHandlerFactory.h @@ -48,8 +48,8 @@ public: LOG_TRACE(log, "TCP Request. Address: {}", socket.peerAddress().toString()); return new TLSHandler( socket, - server.config().getString(conf_name + ".privateKeyFile", ""), - server.config().getString(conf_name + ".certificateFile", ""), + server.config(), + conf_name + ".", stack_data); } catch (const Poco::Net::NetException &) diff --git a/tests/integration/test_composable_protocols/configs/config.xml b/tests/integration/test_composable_protocols/configs/config.xml index 09a512eb5a4..7d39363bc62 100644 --- a/tests/integration/test_composable_protocols/configs/config.xml +++ b/tests/integration/test_composable_protocols/configs/config.xml @@ -58,6 +58,26 @@ 8444 https protocol endpoint + + tls + http + 0.0.0.0 + 8445 + https protocol with TLSv1_2 minimum version + sslv2,sslv3,tlsv1,tlsv1_1 + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + + + tls + http + 0.0.0.0 + 8446 + https protocol with TLSv1_3 minimum version + sslv2,sslv3,tlsv1,tlsv1_1,tlsv1_2 + /etc/clickhouse-server/config.d/server.crt + /etc/clickhouse-server/config.d/server.key + diff --git a/tests/integration/test_composable_protocols/test.py b/tests/integration/test_composable_protocols/test.py index aa5a1e766e6..62c25311185 100644 --- a/tests/integration/test_composable_protocols/test.py +++ b/tests/integration/test_composable_protocols/test.py @@ -7,6 +7,7 @@ from helpers.client import Client import urllib.request, urllib.parse import subprocess import socket +import warnings SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -27,18 +28,34 @@ def setup_nodes(): cluster.shutdown() -def execute_query_https(host, port, query): +def execute_query_https(host, port, query, version=None): url = f"https://{host}:{port}/?query={urllib.parse.quote(query)}" ctx = ssl.create_default_context() ctx.check_hostname = False ctx.verify_mode = ssl.CERT_NONE + if version: + ctx.minimum_version = version + ctx.maximum_version = version request = urllib.request.Request(url) response = urllib.request.urlopen(request, context=ctx).read() return response.decode("utf-8") +def execute_query_https_unsupported(host, port, query, version=None): + try: + execute_query_https(host, port, query, version) + except Exception as e: + e_text = str(e) + if "NO_PROTOCOLS_AVAILABLE" in e_text: + return True + if "TLSV1_ALERT_PROTOCOL_VERSION" in e_text: + return True + raise + return False + + def execute_query_http(host, port, query): url = f"http://{host}:{port}/?query={urllib.parse.quote(query)}" @@ -84,6 +101,20 @@ def test_connections(): assert execute_query_https(server.ip_address, 8444, "SELECT 1") == "1\n" + warnings.filterwarnings("ignore", category=DeprecationWarning) + + assert execute_query_https_unsupported(server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.SSLv3) + assert execute_query_https_unsupported(server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.TLSv1) + assert execute_query_https_unsupported(server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.TLSv1_1) + assert execute_query_https(server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.TLSv1_2) == "1\n" + assert execute_query_https(server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.TLSv1_3) == "1\n" + + assert execute_query_https_unsupported(server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.SSLv3) + assert execute_query_https_unsupported(server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.TLSv1) + assert execute_query_https_unsupported(server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.TLSv1_1) + assert execute_query_https_unsupported(server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.TLSv1_2) + assert execute_query_https(server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.TLSv1_3) == "1\n" + data = "PROXY TCP4 255.255.255.255 255.255.255.255 65535 65535\r\n\0\021ClickHouse client\024\r\253\251\003\0\007default\0\004\001\0\001\0\0\t0.0.0.0:0\001\tmilovidov\021milovidov-desktop\21ClickHouse client\024\r\253\251\003\0\001\0\0\0\002\001\025SELECT 'Hello, world'\002\0\247\203\254l\325\\z|\265\254F\275\333\206\342\024\202\024\0\0\0\n\0\0\0\240\01\0\02\377\377\377\377\0\0\0" assert ( netcat(server.ip_address, 9100, bytearray(data, "latin-1")).find( From fed6c65858f26e31ad8f3d63a2cb6e9a0b404ff7 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 16 May 2024 17:57:01 +0000 Subject: [PATCH 049/318] add setting to enable planning --- src/Core/Settings.h | 1 + src/Interpreters/InterpreterInsertQuery.cpp | 72 +++++++++++++------ .../Transforms/buildPushingToViewsChain.cpp | 28 +++++--- src/Server/TCPHandler.cpp | 67 +++++++++++++++++ src/Storages/ProjectionsDescription.cpp | 11 ++- 5 files changed, 145 insertions(+), 34 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4a0de354a03..393d8202d05 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 55f84080b13..f0340bd1f48 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())); - - pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + if (settings.allow_insert_threads_reduction_optimizaion) { - 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.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 + { + 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 + { + 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); - - chain.addSource(std::move(squashing)); - - auto balancing = std::make_shared( + 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, - 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 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); + + chain.addSource(std::move(squashing)); + } + } auto context_ptr = getContext(); diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index e7c831c3e0e..1d0098f0cd9 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)); + 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 + 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 be3c1384501..56c97d0305e 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 b31cc1e94f1..8d28d68dc39 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 aa6110289e494bac0eaec973dddf6c035821f139 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Fri, 17 May 2024 13:58:29 +0200 Subject: [PATCH 050/318] Fix test style --- .../test_composable_protocols/test.py | 49 +++++++++++++++---- 1 file changed, 39 insertions(+), 10 deletions(-) diff --git a/tests/integration/test_composable_protocols/test.py b/tests/integration/test_composable_protocols/test.py index 62c25311185..241d1505433 100644 --- a/tests/integration/test_composable_protocols/test.py +++ b/tests/integration/test_composable_protocols/test.py @@ -103,17 +103,46 @@ def test_connections(): warnings.filterwarnings("ignore", category=DeprecationWarning) - assert execute_query_https_unsupported(server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.SSLv3) - assert execute_query_https_unsupported(server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.TLSv1) - assert execute_query_https_unsupported(server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.TLSv1_1) - assert execute_query_https(server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.TLSv1_2) == "1\n" - assert execute_query_https(server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.TLSv1_3) == "1\n" + assert execute_query_https_unsupported( + server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.SSLv3 + ) + assert execute_query_https_unsupported( + server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.TLSv1 + ) + assert execute_query_https_unsupported( + server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.TLSv1_1 + ) + assert ( + execute_query_https( + server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.TLSv1_2 + ) + == "1\n" + ) + assert ( + execute_query_https( + server.ip_address, 8445, "SELECT 1", version=ssl.TLSVersion.TLSv1_3 + ) + == "1\n" + ) - assert execute_query_https_unsupported(server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.SSLv3) - assert execute_query_https_unsupported(server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.TLSv1) - assert execute_query_https_unsupported(server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.TLSv1_1) - assert execute_query_https_unsupported(server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.TLSv1_2) - assert execute_query_https(server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.TLSv1_3) == "1\n" + assert execute_query_https_unsupported( + server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.SSLv3 + ) + assert execute_query_https_unsupported( + server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.TLSv1 + ) + assert execute_query_https_unsupported( + server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.TLSv1_1 + ) + assert execute_query_https_unsupported( + server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.TLSv1_2 + ) + assert ( + execute_query_https( + server.ip_address, 8446, "SELECT 1", version=ssl.TLSVersion.TLSv1_3 + ) + == "1\n" + ) data = "PROXY TCP4 255.255.255.255 255.255.255.255 65535 65535\r\n\0\021ClickHouse client\024\r\253\251\003\0\007default\0\004\001\0\001\0\0\t0.0.0.0:0\001\tmilovidov\021milovidov-desktop\21ClickHouse client\024\r\253\251\003\0\001\0\0\0\002\001\025SELECT 'Hello, world'\002\0\247\203\254l\325\\z|\265\254F\275\333\206\342\024\202\024\0\0\0\n\0\0\0\240\01\0\02\377\377\377\377\0\0\0" assert ( From 1f7198b3d3576de29485cd7b96bbc9bf97d181bb Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 17 May 2024 12:15:58 +0000 Subject: [PATCH 051/318] 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 f0340bd1f48..3a6329997bd 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 2d87b47798c..526af3db2e4 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 0bb6acf9372..a68b581d40a 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -58,14 +58,11 @@ private: 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 @@ private: 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 20b23b0ff93735e844f273cfb71956d2a5ff0e8b Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Fri, 17 May 2024 14:28:40 +0200 Subject: [PATCH 052/318] Fix variable name style --- src/Server/TLSHandler.cpp | 34 +++++++++++++++++----------------- src/Server/TLSHandler.h | 12 ++++++------ 2 files changed, 23 insertions(+), 23 deletions(-) diff --git a/src/Server/TLSHandler.cpp b/src/Server/TLSHandler.cpp index 11d3f4e199d..2702b7e676b 100644 --- a/src/Server/TLSHandler.cpp +++ b/src/Server/TLSHandler.cpp @@ -29,14 +29,14 @@ DB::TLSHandler::TLSHandler(const StreamSocket & socket, const LayeredConfigurati params.cipherList = config.getString(prefix + SSLManager::CFG_CIPHER_LIST, SSLManager::VAL_CIPHER_LIST); params.cipherList = config.getString(prefix + SSLManager::CFG_CYPHER_LIST, params.cipherList); // for backwards compatibility - bool requireTLSv1 = config.getBool(prefix + SSLManager::CFG_REQUIRE_TLSV1, false); - bool requireTLSv1_1 = config.getBool(prefix + SSLManager::CFG_REQUIRE_TLSV1_1, false); - bool requireTLSv1_2 = config.getBool(prefix + SSLManager::CFG_REQUIRE_TLSV1_2, false); - if (requireTLSv1_2) + bool require_tlsv1 = config.getBool(prefix + SSLManager::CFG_REQUIRE_TLSV1, false); + bool require_tlsv1_1 = config.getBool(prefix + SSLManager::CFG_REQUIRE_TLSV1_1, false); + bool require_tlsv1_2 = config.getBool(prefix + SSLManager::CFG_REQUIRE_TLSV1_2, false); + if (require_tlsv1_2) usage = Context::TLSV1_2_SERVER_USE; - else if (requireTLSv1_1) + else if (require_tlsv1_1) usage = Context::TLSV1_1_SERVER_USE; - else if (requireTLSv1) + else if (require_tlsv1) usage = Context::TLSV1_SERVER_USE; else usage = Context::SERVER_USE; @@ -44,25 +44,25 @@ DB::TLSHandler::TLSHandler(const StreamSocket & socket, const LayeredConfigurati params.dhParamsFile = config.getString(prefix + SSLManager::CFG_DH_PARAMS_FILE, ""); params.ecdhCurve = config.getString(prefix + SSLManager::CFG_ECDH_CURVE, ""); - std::string disabledProtocolsList = config.getString(prefix + SSLManager::CFG_DISABLE_PROTOCOLS, ""); - Poco::StringTokenizer dpTok(disabledProtocolsList, ";,", Poco::StringTokenizer::TOK_TRIM | Poco::StringTokenizer::TOK_IGNORE_EMPTY); - disabledProtocols = 0; - for (Poco::StringTokenizer::Iterator it = dpTok.begin(); it != dpTok.end(); ++it) + std::string disabled_protocols_list = config.getString(prefix + SSLManager::CFG_DISABLE_PROTOCOLS, ""); + Poco::StringTokenizer dp_tok(disabled_protocols_list, ";,", Poco::StringTokenizer::TOK_TRIM | Poco::StringTokenizer::TOK_IGNORE_EMPTY); + disabled_protocols = 0; + for (Poco::StringTokenizer::Iterator it = dp_tok.begin(); it != dp_tok.end(); ++it) { if (*it == "sslv2") - disabledProtocols |= Context::PROTO_SSLV2; + disabled_protocols |= Context::PROTO_SSLV2; else if (*it == "sslv3") - disabledProtocols |= Context::PROTO_SSLV3; + disabled_protocols |= Context::PROTO_SSLV3; else if (*it == "tlsv1") - disabledProtocols |= Context::PROTO_TLSV1; + disabled_protocols |= Context::PROTO_TLSV1; else if (*it == "tlsv1_1") - disabledProtocols |= Context::PROTO_TLSV1_1; + disabled_protocols |= Context::PROTO_TLSV1_1; else if (*it == "tlsv1_2") - disabledProtocols |= Context::PROTO_TLSV1_2; + disabled_protocols |= Context::PROTO_TLSV1_2; } - extendedVerification = config.getBool(prefix + SSLManager::CFG_EXTENDED_VERIFICATION, false); - preferServerCiphers = config.getBool(prefix + SSLManager::CFG_PREFER_SERVER_CIPHERS, false); + extended_verification = config.getBool(prefix + SSLManager::CFG_EXTENDED_VERIFICATION, false); + prefer_server_ciphers = config.getBool(prefix + SSLManager::CFG_PREFER_SERVER_CIPHERS, false); } #endif } diff --git a/src/Server/TLSHandler.h b/src/Server/TLSHandler.h index 5fcb688663a..901855fbb07 100644 --- a/src/Server/TLSHandler.h +++ b/src/Server/TLSHandler.h @@ -41,9 +41,9 @@ public: if (!params.privateKeyFile.empty() && !params.certificateFile.empty()) { ctx = new Context(usage, params); - ctx->disableProtocols(disabledProtocols); - ctx->enableExtendedCertificateVerification(extendedVerification); - if (preferServerCiphers) + ctx->disableProtocols(disabled_protocols); + ctx->enableExtendedCertificateVerification(extended_verification); + if (prefer_server_ciphers) ctx->preferServerCiphers(); } socket() = SecureStreamSocket::attach(socket(), ctx); @@ -56,9 +56,9 @@ public: private: Context::Params params [[maybe_unused]]; Context::Usage usage [[maybe_unused]]; - int disabledProtocols = 0; - bool extendedVerification = false; - bool preferServerCiphers = false; + int disabled_protocols = 0; + bool extended_verification = false; + bool prefer_server_ciphers = false; TCPProtocolStackData & stack_data [[maybe_unused]]; }; From 183223b7ba332b05147d6d25248d27a35cfdc745 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Fri, 17 May 2024 14:53:43 +0200 Subject: [PATCH 053/318] Fix tabs instead of spaces --- src/Server/TLSHandler.cpp | 4 ++-- src/Server/TLSHandler.h | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Server/TLSHandler.cpp b/src/Server/TLSHandler.cpp index 2702b7e676b..dc8008d1278 100644 --- a/src/Server/TLSHandler.cpp +++ b/src/Server/TLSHandler.cpp @@ -61,8 +61,8 @@ DB::TLSHandler::TLSHandler(const StreamSocket & socket, const LayeredConfigurati disabled_protocols |= Context::PROTO_TLSV1_2; } - extended_verification = config.getBool(prefix + SSLManager::CFG_EXTENDED_VERIFICATION, false); - prefer_server_ciphers = config.getBool(prefix + SSLManager::CFG_PREFER_SERVER_CIPHERS, false); + extended_verification = config.getBool(prefix + SSLManager::CFG_EXTENDED_VERIFICATION, false); + prefer_server_ciphers = config.getBool(prefix + SSLManager::CFG_PREFER_SERVER_CIPHERS, false); } #endif } diff --git a/src/Server/TLSHandler.h b/src/Server/TLSHandler.h index 901855fbb07..61f8c86d9ae 100644 --- a/src/Server/TLSHandler.h +++ b/src/Server/TLSHandler.h @@ -42,9 +42,9 @@ public: { ctx = new Context(usage, params); ctx->disableProtocols(disabled_protocols); - ctx->enableExtendedCertificateVerification(extended_verification); + ctx->enableExtendedCertificateVerification(extended_verification); if (prefer_server_ciphers) - ctx->preferServerCiphers(); + ctx->preferServerCiphers(); } socket() = SecureStreamSocket::attach(socket(), ctx); stack_data.socket = socket(); From 222dbe30d854ee811bc564763d65c7b1f630ff45 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Fri, 17 May 2024 15:06:50 +0200 Subject: [PATCH 054/318] Fix non-SSL build --- src/Server/TLSHandler.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Server/TLSHandler.cpp b/src/Server/TLSHandler.cpp index dc8008d1278..549384a7433 100644 --- a/src/Server/TLSHandler.cpp +++ b/src/Server/TLSHandler.cpp @@ -1,5 +1,10 @@ +#if USE_SSL + #include #include + +#endif + #include DB::TLSHandler::TLSHandler(const StreamSocket & socket, const LayeredConfiguration & config, const std::string & prefix, TCPProtocolStackData & stack_data_) From 72285615cba13e124c4b1bb524fa072063199ef7 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Fri, 17 May 2024 15:33:14 +0200 Subject: [PATCH 055/318] Fix non-SSL build 2 --- src/Server/TLSHandler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Server/TLSHandler.cpp b/src/Server/TLSHandler.cpp index 549384a7433..c2b9f3f7fa0 100644 --- a/src/Server/TLSHandler.cpp +++ b/src/Server/TLSHandler.cpp @@ -1,3 +1,5 @@ +#include + #if USE_SSL #include @@ -5,8 +7,6 @@ #endif -#include - DB::TLSHandler::TLSHandler(const StreamSocket & socket, const LayeredConfiguration & config, const std::string & prefix, TCPProtocolStackData & stack_data_) : Poco::Net::TCPServerConnection(socket) , stack_data(stack_data_) From f4afaf7f1841a57c35dd7a6b77cae52cfad97d36 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Fri, 17 May 2024 16:16:54 +0200 Subject: [PATCH 056/318] Fix non-SSL build 3 --- src/Server/TLSHandler.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Server/TLSHandler.h b/src/Server/TLSHandler.h index 61f8c86d9ae..0dcf29aa152 100644 --- a/src/Server/TLSHandler.h +++ b/src/Server/TLSHandler.h @@ -27,8 +27,8 @@ class TLSHandler : public Poco::Net::TCPServerConnection #if USE_SSL using SecureStreamSocket = Poco::Net::SecureStreamSocket; using SSLManager = Poco::Net::SSLManager; -#endif using Context = Poco::Net::Context; +#endif using StreamSocket = Poco::Net::StreamSocket; using LayeredConfiguration = Poco::Util::LayeredConfiguration; public: @@ -54,11 +54,13 @@ public: #endif } private: +#if USE_SSL Context::Params params [[maybe_unused]]; Context::Usage usage [[maybe_unused]]; int disabled_protocols = 0; bool extended_verification = false; bool prefer_server_ciphers = false; +#endif TCPProtocolStackData & stack_data [[maybe_unused]]; }; From f01386fc79059907eb47e7a70167ca4cb2c4ab41 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Fri, 17 May 2024 16:36:55 +0200 Subject: [PATCH 057/318] Fix non-SSL build 4 --- src/Server/TLSHandler.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Server/TLSHandler.cpp b/src/Server/TLSHandler.cpp index c2b9f3f7fa0..722f4c7c19e 100644 --- a/src/Server/TLSHandler.cpp +++ b/src/Server/TLSHandler.cpp @@ -7,7 +7,11 @@ #endif -DB::TLSHandler::TLSHandler(const StreamSocket & socket, const LayeredConfiguration & config, const std::string & prefix, TCPProtocolStackData & stack_data_) +DB::TLSHandler::TLSHandler( + const StreamSocket & socket, + [[maybe_unused]] const LayeredConfiguration & config, + [[maybe_unused]] const std::string & prefix, + TCPProtocolStackData & stack_data_) : Poco::Net::TCPServerConnection(socket) , stack_data(stack_data_) { From 8d235a4a399b3489ff3a8672134c8905511562a3 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 17 May 2024 15:11:21 +0000 Subject: [PATCH 058/318] 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 56c97d0305e..3660b4c1187 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 059/318] 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 584fb72cccb..abb3a0aad41 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 060/318] 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 3a6329997bd..47a0567dfec 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 1d0098f0cd9..4e703828554 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 8d28d68dc39..45add4332ff 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 40a78df96a44552c2548e67aef93601a14afec57 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Fri, 17 May 2024 17:10:48 +0000 Subject: [PATCH 061/318] small improvements --- src/Interpreters/Squashing.cpp | 46 +++++++------------ src/Interpreters/Squashing.h | 2 +- .../Transforms/PlanSquashingTransform.cpp | 1 - src/Storages/ProjectionsDescription.cpp | 2 +- 4 files changed, 19 insertions(+), 32 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 526af3db2e4..8f0964403b1 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -147,47 +147,35 @@ Block ApplySquashing::addImpl(Chunk && input_chunk) return Block(); const auto *info = getInfoFromChunk(input_chunk); - append(info->chunks); + for (auto & chunk : info->chunks) + append(chunk); Block to_return; std::swap(to_return, accumulated_block); return to_return; } -void ApplySquashing::append(const std::vector & input_chunks) +void ApplySquashing::append(Chunk & input_chunk) { - 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) + if (input_chunk.getNumColumns() == 0) + return; + if (!accumulated_block) { - if (!accumulated_block) + for (size_t i = 0; i < input_chunk.getNumColumns(); ++ i) { - 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); - } + ColumnWithTypeAndName col = ColumnWithTypeAndName(input_chunk.getColumns()[i], header.getDataTypes()[i], header.getNames()[i]); + accumulated_block.insert(accumulated_block.columns(), col); } + return; + } - if (input_chunk.getNumColumns() == 0) - continue; + for (size_t i = 0, size = accumulated_block.columns(); i < size; ++i) + { + const auto source_column = input_chunk.getColumns()[i]; - 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(); - } + 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); } } diff --git a/src/Interpreters/Squashing.h b/src/Interpreters/Squashing.h index a68b581d40a..8273ae8cc8e 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -70,7 +70,7 @@ private: const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); - void append(const std::vector & input_chunk); + void append(Chunk & input_chunk); bool isEnoughSize(const Block & block); bool isEnoughSize(size_t rows, size_t bytes) const; diff --git a/src/Processors/Transforms/PlanSquashingTransform.cpp b/src/Processors/Transforms/PlanSquashingTransform.cpp index f4106204462..51781b03853 100644 --- a/src/Processors/Transforms/PlanSquashingTransform.cpp +++ b/src/Processors/Transforms/PlanSquashingTransform.cpp @@ -1,6 +1,5 @@ #include #include -#include "Common/logger_useful.h" #include namespace DB diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 45add4332ff..070245ba1b4 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -318,7 +318,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) } else { - 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)); From 330bac788e64fd0a50f7f55d8a52231d640abb49 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 21 May 2024 12:05:22 +0000 Subject: [PATCH 062/318] remove setting --- src/Core/Settings.h | 1 - src/Interpreters/InterpreterInsertQuery.cpp | 54 +++++-------------- src/Interpreters/Squashing.cpp | 2 - .../Transforms/buildPushingToViewsChain.cpp | 29 +++------- src/Storages/ProjectionsDescription.cpp | 12 ++--- 5 files changed, 25 insertions(+), 73 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 393d8202d05..4a0de354a03 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -668,7 +668,6 @@ 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 47a0567dfec..c7729f3985b 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -620,29 +620,16 @@ BlockIO InterpreterInsertQuery::execute() { bool table_prefers_large_blocks = table->prefersLargeBlocks(); - 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.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 - { - return std::make_shared(in_header); - }); - } - else + 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); - }); - } + return std::make_shared(in_header); + }); } size_t num_select_threads = pipeline.getNumThreads(); @@ -694,30 +681,17 @@ BlockIO InterpreterInsertQuery::execute() { bool table_prefers_large_blocks = table->prefersLargeBlocks(); - if (settings.allow_insert_threads_reduction_optimizaion) - { - auto squashing = std::make_shared(chain.getInputHeader()); + auto squashing = std::make_shared(chain.getInputHeader()); - chain.addSource(std::move(squashing)); + 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, - presink_chains.size()); - - chain.addSource(std::move(balancing)); - } - else - { - auto squashing = 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); - - chain.addSource(std::move(squashing)); - } + table_prefers_large_blocks ? settings.min_insert_block_size_bytes : 0ULL, + presink_chains.size()); + chain.addSource(std::move(balancing)); } auto context_ptr = getContext(); diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 8f0964403b1..9e398febdca 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -1,8 +1,6 @@ #include #include #include -#include "Core/Block.h" -#include "Core/ColumnsWithTypeAndName.h" namespace DB diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 4e703828554..951f40dadb9 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -367,29 +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); - 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(); - if (settings.allow_insert_threads_reduction_optimizaion) - { - out.addSource(std::make_shared(out.getInputHeader())); + out.addSource(std::make_shared(out.getInputHeader())); - 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)); - } - } + 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 auto counting = std::make_shared(out.getInputHeader(), current_thread, insert_context->getQuota()); counting->setProcessListElement(insert_context->getProcessListElement()); diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 070245ba1b4..c88582a8a1a 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -311,15 +311,9 @@ 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. - 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())); - } - else - { - builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); - } + + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0, 1)); + builder.addTransform(std::make_shared(builder.getHeader())); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); From 40eef7f027b603a7313d23bea96769e149d17c02 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Tue, 21 May 2024 15:58:00 +0200 Subject: [PATCH 063/318] Fix tidy build --- src/Server/TLSHandler.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Server/TLSHandler.cpp b/src/Server/TLSHandler.cpp index 722f4c7c19e..b3ae3625b1c 100644 --- a/src/Server/TLSHandler.cpp +++ b/src/Server/TLSHandler.cpp @@ -56,17 +56,17 @@ DB::TLSHandler::TLSHandler( std::string disabled_protocols_list = config.getString(prefix + SSLManager::CFG_DISABLE_PROTOCOLS, ""); Poco::StringTokenizer dp_tok(disabled_protocols_list, ";,", Poco::StringTokenizer::TOK_TRIM | Poco::StringTokenizer::TOK_IGNORE_EMPTY); disabled_protocols = 0; - for (Poco::StringTokenizer::Iterator it = dp_tok.begin(); it != dp_tok.end(); ++it) + for (const auto & token : dp_tok) { - if (*it == "sslv2") + if (token == "sslv2") disabled_protocols |= Context::PROTO_SSLV2; - else if (*it == "sslv3") + else if (token == "sslv3") disabled_protocols |= Context::PROTO_SSLV3; - else if (*it == "tlsv1") + else if (token == "tlsv1") disabled_protocols |= Context::PROTO_TLSV1; - else if (*it == "tlsv1_1") + else if (token == "tlsv1_1") disabled_protocols |= Context::PROTO_TLSV1_1; - else if (*it == "tlsv1_2") + else if (token == "tlsv1_2") disabled_protocols |= Context::PROTO_TLSV1_2; } From ca45acd57f5e68bed85600eefd93f1b007f89511 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 22 May 2024 12:28:40 +0200 Subject: [PATCH 064/318] Use CertificateReloader in TLSHandler --- src/Server/CertificateReloader.cpp | 53 +++++++++++++++++++++--------- src/Server/CertificateReloader.h | 43 +++++++++++++++--------- src/Server/TLSHandler.cpp | 40 ++++++++++++++++++---- src/Server/TLSHandler.h | 27 +++------------ 4 files changed, 105 insertions(+), 58 deletions(-) diff --git a/src/Server/CertificateReloader.cpp b/src/Server/CertificateReloader.cpp index 98d7a362bd7..677eb5b9273 100644 --- a/src/Server/CertificateReloader.cpp +++ b/src/Server/CertificateReloader.cpp @@ -15,18 +15,23 @@ namespace DB namespace { + /// Call set process for certificate. -int callSetCertificate(SSL * ssl, [[maybe_unused]] void * arg) +int callSetCertificate(SSL * ssl, void * arg) { - return CertificateReloader::instance().setCertificate(ssl); + if (!arg) + return -1; + + const CertificateReloader::MultiData * pdata = reinterpret_cast(arg); + return CertificateReloader::instance().setCertificate(ssl, pdata); } } /// This is callback for OpenSSL. It will be called on every connection to obtain a certificate and private key. -int CertificateReloader::setCertificate(SSL * ssl) +int CertificateReloader::setCertificate(SSL * ssl, const CertificateReloader::MultiData * pdata) { - auto current = data.get(); + auto current = pdata->data.get(); if (!current) return -1; @@ -65,24 +70,30 @@ int CertificateReloader::setCertificate(SSL * ssl) } -void CertificateReloader::init() +void CertificateReloader::init(MultiData * pdata, SSL_CTX * ctx) { LOG_DEBUG(log, "Initializing certificate reloader."); /// Set a callback for OpenSSL to allow get the updated cert and key. - auto* ctx = Poco::Net::SSLManager::instance().defaultServerContext()->sslContext(); - SSL_CTX_set_cert_cb(ctx, callSetCertificate, nullptr); - init_was_not_made = false; + SSL_CTX_set_cert_cb(ctx, callSetCertificate, reinterpret_cast(pdata)); + pdata->init_was_not_made = false; } void CertificateReloader::tryLoad(const Poco::Util::AbstractConfiguration & config) +{ + auto* ctx = Poco::Net::SSLManager::instance().defaultServerContext()->sslContext(); + tryLoad(config, ctx, Poco::Net::SSLManager::CFG_SERVER_PREFIX); +} + + +void CertificateReloader::tryLoad(const Poco::Util::AbstractConfiguration & config, SSL_CTX * ctx, const std::string & prefix) { /// If at least one of the files is modified - recreate - std::string new_cert_path = config.getString("openSSL.server.certificateFile", ""); - std::string new_key_path = config.getString("openSSL.server.privateKeyFile", ""); + std::string new_cert_path = config.getString(prefix + "certificateFile", ""); + std::string new_key_path = config.getString(prefix + "privateKeyFile", ""); /// For empty paths (that means, that user doesn't want to use certificates) /// no processing required @@ -95,24 +106,36 @@ void CertificateReloader::tryLoad(const Poco::Util::AbstractConfiguration & conf { bool cert_file_changed = cert_file.changeIfModified(std::move(new_cert_path), log); bool key_file_changed = key_file.changeIfModified(std::move(new_key_path), log); - std::string pass_phrase = config.getString("openSSL.server.privateKeyPassphraseHandler.options.password", ""); + std::string pass_phrase = config.getString(prefix + "privateKeyPassphraseHandler.options.password", ""); + + std::unique_lock lock(data_mutex); + auto it = data.end(); + auto i = data_index.find(prefix); + if (i != data_index.end()) + it = i->second; + else + { + data.push_back(MultiData()); + --it; + data_index[prefix] = it; + } if (cert_file_changed || key_file_changed) { LOG_DEBUG(log, "Reloading certificate ({}) and key ({}).", cert_file.path, key_file.path); - data.set(std::make_unique(cert_file.path, key_file.path, pass_phrase)); + it->data.set(std::make_unique(cert_file.path, key_file.path, pass_phrase)); LOG_INFO(log, "Reloaded certificate ({}) and key ({}).", cert_file.path, key_file.path); } /// If callback is not set yet try { - if (init_was_not_made) - init(); + if (it->init_was_not_made) + init(&*it, ctx); } catch (...) { - init_was_not_made = true; + it->init_was_not_made = true; LOG_ERROR(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false)); } } diff --git a/src/Server/CertificateReloader.h b/src/Server/CertificateReloader.h index 5ab799037d5..1c6f6109b07 100644 --- a/src/Server/CertificateReloader.h +++ b/src/Server/CertificateReloader.h @@ -6,6 +6,9 @@ #include #include +#include +#include +#include #include #include @@ -31,6 +34,20 @@ class CertificateReloader public: using stat_t = struct stat; + struct Data + { + Poco::Crypto::X509Certificate::List certs_chain; + Poco::Crypto::EVPPKey key; + + Data(std::string cert_path, std::string key_path, std::string pass_phrase); + }; + + struct MultiData + { + MultiVersion data; + bool init_was_not_made = true; + }; + /// Singleton CertificateReloader(CertificateReloader const &) = delete; void operator=(CertificateReloader const &) = delete; @@ -40,18 +57,21 @@ public: return instance; } - /// Initialize the callback and perform the initial cert loading - void init(); - - /// Handle configuration reload + /// Handle configuration reload for default path void tryLoad(const Poco::Util::AbstractConfiguration & config); + /// Handle configuration reload + void tryLoad(const Poco::Util::AbstractConfiguration & config, SSL_CTX * ctx, const std::string & prefix); + /// A callback for OpenSSL - int setCertificate(SSL * ssl); + int setCertificate(SSL * ssl, const MultiData * pdata); private: CertificateReloader() = default; + /// Initialize the callback and perform the initial cert loading + void init(MultiData * pdata, SSL_CTX * ctx); + LoggerPtr log = getLogger("CertificateReloader"); struct File @@ -68,16 +88,9 @@ private: File cert_file{"certificate"}; File key_file{"key"}; - struct Data - { - Poco::Crypto::X509Certificate::List certs_chain; - Poco::Crypto::EVPPKey key; - - Data(std::string cert_path, std::string key_path, std::string pass_phrase); - }; - - MultiVersion data; - bool init_was_not_made = true; + std::mutex data_mutex; + std::list data; + std::unordered_map::iterator> data_index; }; } diff --git a/src/Server/TLSHandler.cpp b/src/Server/TLSHandler.cpp index b3ae3625b1c..9ec1df54c32 100644 --- a/src/Server/TLSHandler.cpp +++ b/src/Server/TLSHandler.cpp @@ -1,18 +1,24 @@ #include +#include +#include + #if USE_SSL - -#include -#include - +# include +# include +# include #endif DB::TLSHandler::TLSHandler( const StreamSocket & socket, - [[maybe_unused]] const LayeredConfiguration & config, - [[maybe_unused]] const std::string & prefix, + const LayeredConfiguration & config_, + const std::string & prefix_, TCPProtocolStackData & stack_data_) : Poco::Net::TCPServerConnection(socket) +#if USE_SSL + , config(config_) + , prefix(prefix_) +#endif , stack_data(stack_data_) { #if USE_SSL @@ -75,3 +81,25 @@ DB::TLSHandler::TLSHandler( } #endif } + + +void DB::TLSHandler::run() +{ +#if USE_SSL + auto ctx = SSLManager::instance().defaultServerContext(); + if (!params.privateKeyFile.empty() && !params.certificateFile.empty()) + { + ctx = new Context(usage, params); + ctx->disableProtocols(disabled_protocols); + ctx->enableExtendedCertificateVerification(extended_verification); + if (prefer_server_ciphers) + ctx->preferServerCiphers(); + CertificateReloader::instance().tryLoad(config, ctx->sslContext(), prefix); + } + socket() = SecureStreamSocket::attach(socket(), ctx); + stack_data.socket = socket(); + stack_data.certificate = params.certificateFile; +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); +#endif +} diff --git a/src/Server/TLSHandler.h b/src/Server/TLSHandler.h index 0dcf29aa152..63b220cec2b 100644 --- a/src/Server/TLSHandler.h +++ b/src/Server/TLSHandler.h @@ -1,8 +1,6 @@ #pragma once #include -#include -#include #include #include @@ -32,27 +30,10 @@ class TLSHandler : public Poco::Net::TCPServerConnection using StreamSocket = Poco::Net::StreamSocket; using LayeredConfiguration = Poco::Util::LayeredConfiguration; public: - explicit TLSHandler(const StreamSocket & socket, const LayeredConfiguration & config, const std::string & prefix, TCPProtocolStackData & stack_data_); + explicit TLSHandler(const StreamSocket & socket, const LayeredConfiguration & config_, const std::string & prefix_, TCPProtocolStackData & stack_data_); + + void run() override; - void run() override - { -#if USE_SSL - auto ctx = SSLManager::instance().defaultServerContext(); - if (!params.privateKeyFile.empty() && !params.certificateFile.empty()) - { - ctx = new Context(usage, params); - ctx->disableProtocols(disabled_protocols); - ctx->enableExtendedCertificateVerification(extended_verification); - if (prefer_server_ciphers) - ctx->preferServerCiphers(); - } - socket() = SecureStreamSocket::attach(socket(), ctx); - stack_data.socket = socket(); - stack_data.certificate = params.certificateFile; -#else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); -#endif - } private: #if USE_SSL Context::Params params [[maybe_unused]]; @@ -60,6 +41,8 @@ private: int disabled_protocols = 0; bool extended_verification = false; bool prefer_server_ciphers = false; + const LayeredConfiguration & config; + std::string prefix; #endif TCPProtocolStackData & stack_data [[maybe_unused]]; }; From 298b19c19daf5d12978764886c2da9213d49419b Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 22 May 2024 13:27:39 +0200 Subject: [PATCH 065/318] Fix style --- src/Server/TLSHandler.cpp | 11 +++++++++-- src/Server/TLSHandler.h | 9 ++------- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/src/Server/TLSHandler.cpp b/src/Server/TLSHandler.cpp index 9ec1df54c32..45ccad06529 100644 --- a/src/Server/TLSHandler.cpp +++ b/src/Server/TLSHandler.cpp @@ -9,10 +9,17 @@ # include #endif +#if !defined(USE_SSL) || USE_SSL == 0 +namespace ErrorCodes +{ + extern const int SUPPORT_IS_DISABLED; +} +#endif + DB::TLSHandler::TLSHandler( const StreamSocket & socket, - const LayeredConfiguration & config_, - const std::string & prefix_, + [[maybe_unused]] const LayeredConfiguration & config_, + [[maybe_unused]] const std::string & prefix_, TCPProtocolStackData & stack_data_) : Poco::Net::TCPServerConnection(socket) #if USE_SSL diff --git a/src/Server/TLSHandler.h b/src/Server/TLSHandler.h index 63b220cec2b..2bec7380b08 100644 --- a/src/Server/TLSHandler.h +++ b/src/Server/TLSHandler.h @@ -15,11 +15,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int SUPPORT_IS_DISABLED; -} - class TLSHandler : public Poco::Net::TCPServerConnection { #if USE_SSL @@ -41,8 +36,8 @@ private: int disabled_protocols = 0; bool extended_verification = false; bool prefer_server_ciphers = false; - const LayeredConfiguration & config; - std::string prefix; + const LayeredConfiguration & config [[maybe_unused]]; + std::string prefix [[maybe_unused]]; #endif TCPProtocolStackData & stack_data [[maybe_unused]]; }; From 319542f85bc5c36bbc7c810c0883b721956250be Mon Sep 17 00:00:00 2001 From: yariks5s Date: Wed, 22 May 2024 19:07:14 +0000 Subject: [PATCH 066/318] fix planning algorithm + switch matView to old squashing --- src/Interpreters/Squashing.cpp | 42 ++++++++++++++++--- src/Interpreters/Squashing.h | 2 +- .../Transforms/buildPushingToViewsChain.cpp | 13 ++---- 3 files changed, 41 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 9e398febdca..7ebe4a930c9 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(); - - if (input_chunk) chunks_to_merge_vec.push_back(std::move(input_chunk)); + return res_chunk; + } + /// Accumulated block is already enough. if (isEnoughSize(chunks_to_merge_vec)) { - Chunk res_chunk = convertToChunk(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(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 8273ae8cc8e..0e9f001762f 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -100,7 +100,7 @@ private: 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 951f40dadb9..cf407a75879 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 067/318] 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 51781b03853..51c637f745b 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 068/318] 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 51c637f745b..2cb0a19ecdb 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 983fa64e4fe1c275a5d7a426196635b5d5ff2471 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 22 May 2024 17:29:50 +0200 Subject: [PATCH 069/318] Fix CertificateReloader usage for custom protocols --- .../include/Poco/Net/SSLManager.h | 10 ++++ base/poco/NetSSL_OpenSSL/src/SSLManager.cpp | 17 +++++++ programs/server/Server.cpp | 18 ++++++-- src/Server/CertificateReloader.cpp | 46 +++++++++++++------ src/Server/CertificateReloader.h | 39 ++++++++++------ src/Server/TLSHandler.cpp | 18 +++++--- 6 files changed, 110 insertions(+), 38 deletions(-) diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h index 4269df95246..25dc133fb20 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h @@ -17,6 +17,7 @@ #ifndef NetSSL_SSLManager_INCLUDED #define NetSSL_SSLManager_INCLUDED +#include #include #include "Poco/BasicEvent.h" @@ -219,6 +220,13 @@ namespace Net /// Unless initializeClient() has been called, the first call to this method initializes the default Context /// from the application configuration. + Context::Ptr getCustomServerContext(const std::string & name); + /// Return custom Context used by the server. + + Context::Ptr setCustomServerContext(const std::string & name, Context::Ptr ctx); + /// Set custom Context used by the server. + /// Return pointer on inserted Context or on old Context if exists. + PrivateKeyPassphraseHandlerPtr serverPassphraseHandler(); /// Returns the configured passphrase handler of the server. If none is set, the method will create a default one /// from an application configuration. @@ -348,6 +356,8 @@ namespace Net InvalidCertificateHandlerPtr _ptrClientCertificateHandler; Poco::FastMutex _mutex; + std::unordered_map _mapPtrServerContexts; + friend class Poco::SingletonHolder; friend class Context; }; diff --git a/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp b/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp index 7f6cc9abcb2..53dc0094cac 100644 --- a/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp @@ -429,6 +429,23 @@ void SSLManager::initCertificateHandler(bool server) } +Context::Ptr SSLManager::getCustomServerContext(const std::string & name) +{ + Poco::FastMutex::ScopedLock lock(_mutex); + auto it = _mapPtrServerContexts.find(name); + if (it != _mapPtrServerContexts.end()) + return it->second; + return nullptr; +} + +Context::Ptr SSLManager::setCustomServerContext(const std::string & name, Context::Ptr ctx) +{ + Poco::FastMutex::ScopedLock lock(_mutex); + ctx = _mapPtrServerContexts.insert({name, ctx}).first->second; + return ctx; +} + + Poco::Util::AbstractConfiguration& SSLManager::appConfig() { try diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 9c9476d1aa7..06e7ea7d834 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1359,8 +1359,8 @@ try tryLogCurrentException(log, "Disabling cgroup memory observer because of an error during initialization"); } - const std::string cert_path = config().getString("openSSL.server.certificateFile", ""); - const std::string key_path = config().getString("openSSL.server.privateKeyFile", ""); + std::string cert_path = config().getString("openSSL.server.certificateFile", ""); + std::string key_path = config().getString("openSSL.server.privateKeyFile", ""); std::vector extra_paths = {include_from_path}; if (!cert_path.empty()) @@ -1368,6 +1368,18 @@ try if (!key_path.empty()) extra_paths.emplace_back(key_path); + Poco::Util::AbstractConfiguration::Keys protocols; + config().keys("protocols", protocols); + for (const auto & protocol : protocols) + { + cert_path = config().getString("protocols." + protocol + ".certificateFile", ""); + key_path = config().getString("protocols." + protocol + ".privateKeyFile", ""); + if (!cert_path.empty()) + extra_paths.emplace_back(cert_path); + if (!key_path.empty()) + extra_paths.emplace_back(key_path); + } + auto main_config_reloader = std::make_unique( config_path, extra_paths, @@ -1602,7 +1614,7 @@ try CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, "encryption_codecs"); #if USE_SSL - CertificateReloader::instance().tryLoad(*config); + CertificateReloader::instance().tryReloadAll(*config); #endif NamedCollectionUtils::reloadFromConfig(*config); diff --git a/src/Server/CertificateReloader.cpp b/src/Server/CertificateReloader.cpp index 677eb5b9273..a89634e05e4 100644 --- a/src/Server/CertificateReloader.cpp +++ b/src/Server/CertificateReloader.cpp @@ -70,25 +70,30 @@ int CertificateReloader::setCertificate(SSL * ssl, const CertificateReloader::Mu } -void CertificateReloader::init(MultiData * pdata, SSL_CTX * ctx) +void CertificateReloader::init(MultiData * pdata) { LOG_DEBUG(log, "Initializing certificate reloader."); /// Set a callback for OpenSSL to allow get the updated cert and key. - SSL_CTX_set_cert_cb(ctx, callSetCertificate, reinterpret_cast(pdata)); + SSL_CTX_set_cert_cb(pdata->ctx, callSetCertificate, reinterpret_cast(pdata)); pdata->init_was_not_made = false; } void CertificateReloader::tryLoad(const Poco::Util::AbstractConfiguration & config) { - auto* ctx = Poco::Net::SSLManager::instance().defaultServerContext()->sslContext(); - tryLoad(config, ctx, Poco::Net::SSLManager::CFG_SERVER_PREFIX); + tryLoad(config, nullptr, Poco::Net::SSLManager::CFG_SERVER_PREFIX); } void CertificateReloader::tryLoad(const Poco::Util::AbstractConfiguration & config, SSL_CTX * ctx, const std::string & prefix) +{ + std::unique_lock lock(data_mutex); + tryLoadImpl(config, ctx, prefix); +} + +void CertificateReloader::tryLoadImpl(const Poco::Util::AbstractConfiguration & config, SSL_CTX * ctx, const std::string & prefix) { /// If at least one of the files is modified - recreate @@ -104,34 +109,39 @@ void CertificateReloader::tryLoad(const Poco::Util::AbstractConfiguration & conf } else { - bool cert_file_changed = cert_file.changeIfModified(std::move(new_cert_path), log); - bool key_file_changed = key_file.changeIfModified(std::move(new_key_path), log); - std::string pass_phrase = config.getString(prefix + "privateKeyPassphraseHandler.options.password", ""); - - std::unique_lock lock(data_mutex); auto it = data.end(); auto i = data_index.find(prefix); if (i != data_index.end()) it = i->second; else { - data.push_back(MultiData()); + data.push_back(MultiData(ctx)); --it; data_index[prefix] = it; } + bool cert_file_changed = it->cert_file.changeIfModified(std::move(new_cert_path), log); + bool key_file_changed = it->key_file.changeIfModified(std::move(new_key_path), log); + std::string pass_phrase = config.getString(prefix + "privateKeyPassphraseHandler.options.password", ""); + if (cert_file_changed || key_file_changed) { - LOG_DEBUG(log, "Reloading certificate ({}) and key ({}).", cert_file.path, key_file.path); - it->data.set(std::make_unique(cert_file.path, key_file.path, pass_phrase)); - LOG_INFO(log, "Reloaded certificate ({}) and key ({}).", cert_file.path, key_file.path); + LOG_DEBUG(log, "Reloading certificate ({}) and key ({}).", it->cert_file.path, it->key_file.path); + it->data.set(std::make_unique(it->cert_file.path, it->key_file.path, pass_phrase)); + LOG_INFO(log, "Reloaded certificate ({}) and key ({}).", it->cert_file.path, it->key_file.path); } /// If callback is not set yet try { + if (!ctx) + { + ctx = Poco::Net::SSLManager::instance().defaultServerContext()->sslContext(); + it->ctx = ctx; + } + if (it->init_was_not_made) - init(&*it, ctx); + init(&*it); } catch (...) { @@ -142,6 +152,14 @@ void CertificateReloader::tryLoad(const Poco::Util::AbstractConfiguration & conf } +void CertificateReloader::tryReloadAll(const Poco::Util::AbstractConfiguration & config) +{ + std::unique_lock lock(data_mutex); + for (auto & item : data_index) + tryLoadImpl(config, item.second->ctx, item.first); +} + + CertificateReloader::Data::Data(std::string cert_path, std::string key_path, std::string pass_phrase) : certs_chain(Poco::Crypto::X509Certificate::readPEM(cert_path)), key(/* public key */ "", /* private key */ key_path, pass_phrase) { diff --git a/src/Server/CertificateReloader.h b/src/Server/CertificateReloader.h index 1c6f6109b07..40b81f7a06f 100644 --- a/src/Server/CertificateReloader.h +++ b/src/Server/CertificateReloader.h @@ -42,10 +42,27 @@ public: Data(std::string cert_path, std::string key_path, std::string pass_phrase); }; + struct File + { + const char * description; + explicit File(const char * description_) : description(description_) {} + + std::string path; + std::filesystem::file_time_type modification_time; + + bool changeIfModified(std::string new_path, LoggerPtr logger); + }; + struct MultiData { + SSL_CTX * ctx; MultiVersion data; bool init_was_not_made = true; + + File cert_file{"certificate"}; + File key_file{"key"}; + + explicit MultiData(SSL_CTX * ctx_) : ctx(ctx_) {} }; /// Singleton @@ -63,6 +80,9 @@ public: /// Handle configuration reload void tryLoad(const Poco::Util::AbstractConfiguration & config, SSL_CTX * ctx, const std::string & prefix); + /// Handle configuration reload for all contexts + void tryReloadAll(const Poco::Util::AbstractConfiguration & config); + /// A callback for OpenSSL int setCertificate(SSL * ssl, const MultiData * pdata); @@ -70,24 +90,13 @@ private: CertificateReloader() = default; /// Initialize the callback and perform the initial cert loading - void init(MultiData * pdata, SSL_CTX * ctx); + void init(MultiData * pdata); + + /// Unsafe implementation + void tryLoadImpl(const Poco::Util::AbstractConfiguration & config, SSL_CTX * ctx, const std::string & prefix); LoggerPtr log = getLogger("CertificateReloader"); - struct File - { - const char * description; - explicit File(const char * description_) : description(description_) {} - - std::string path; - std::filesystem::file_time_type modification_time; - - bool changeIfModified(std::string new_path, LoggerPtr logger); - }; - - File cert_file{"certificate"}; - File key_file{"key"}; - std::mutex data_mutex; std::list data; std::unordered_map::iterator> data_index; diff --git a/src/Server/TLSHandler.cpp b/src/Server/TLSHandler.cpp index 45ccad06529..b0ed342c251 100644 --- a/src/Server/TLSHandler.cpp +++ b/src/Server/TLSHandler.cpp @@ -3,6 +3,7 @@ #include #include + #if USE_SSL # include # include @@ -96,12 +97,17 @@ void DB::TLSHandler::run() auto ctx = SSLManager::instance().defaultServerContext(); if (!params.privateKeyFile.empty() && !params.certificateFile.empty()) { - ctx = new Context(usage, params); - ctx->disableProtocols(disabled_protocols); - ctx->enableExtendedCertificateVerification(extended_verification); - if (prefer_server_ciphers) - ctx->preferServerCiphers(); - CertificateReloader::instance().tryLoad(config, ctx->sslContext(), prefix); + ctx = SSLManager::instance().getCustomServerContext(prefix); + if (!ctx) + { + ctx = new Context(usage, params); + ctx->disableProtocols(disabled_protocols); + ctx->enableExtendedCertificateVerification(extended_verification); + if (prefer_server_ciphers) + ctx->preferServerCiphers(); + CertificateReloader::instance().tryLoad(config, ctx->sslContext(), prefix); + ctx = SSLManager::instance().setCustomServerContext(prefix, ctx); + } } socket() = SecureStreamSocket::attach(socket(), ctx); stack_data.socket = socket(); From f632636f210c34841a7634790e32ba2153633ebf Mon Sep 17 00:00:00 2001 From: yariks5s Date: Thu, 23 May 2024 19:12:02 +0000 Subject: [PATCH 070/318] 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 7ebe4a930c9..46e21635a30 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 0e9f001762f..d116ff1eddd 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 abb3a0aad41..e63691fcc6a 100644 --- a/src/Processors/Transforms/ApplySquashingTransform.h +++ b/src/Processors/Transforms/ApplySquashingTransform.h @@ -37,8 +37,8 @@ public: 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 @@ protected: } 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 dc5b6d669b1..7afc942a7f2 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 8f7f6488d3e..a516811bf45 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 c5b727ac6ec..b5b3c6616d2 100644 --- a/src/Processors/Transforms/SquashingTransform.h +++ b/src/Processors/Transforms/SquashingTransform.h @@ -24,7 +24,8 @@ protected: void onFinish() override; private: - Squashing squashing; + PlanSquashing planSquashing; + ApplySquashing applySquashing; Chunk cur_chunk; Chunk finish_chunk; }; @@ -43,7 +44,8 @@ protected: 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 071/318] 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 46e21635a30..12dcac7eb96 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; + accumulated_size.rows += rows; + accumulated_size.bytes += bytes; +} - for (const Chunk & chunk : chunks) - { - rows += chunk.getNumRows(); - bytes += chunk.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 d116ff1eddd..0e844c4912b 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -61,14 +61,12 @@ public: 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 @@ public: } 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 2cb0a19ecdb..1384f760d48 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 a516811bf45..67358316d48 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 3f8e43d81dd..9f14facdf8f 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 8ebb2173f6c..0f7d65fc7c9 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 072/318] 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 12dcac7eb96..e2abcd00be3 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 073/318] 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 e2abcd00be3..855bf32abe9 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 074/318] 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 855bf32abe9..3872c2ba6b9 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 0e844c4912b..d9d430c1835 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 @@ private: 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 @@ private: 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 1384f760d48..96f41e37d2f 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 67358316d48..6f7c877b2f3 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 9f14facdf8f..476c4dd372b 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 8052ee8f630..f7a4651f6fd 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 @@ private: 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 075/318] 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 3872c2ba6b9..82d80114a85 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 076/318] 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 82d80114a85..9ecd92f732c 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 d9d430c1835..a2928e0eeb6 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -48,7 +48,7 @@ private: 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 @@ private: 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 96f41e37d2f..7945bd97e04 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 6f7c877b2f3..a516811bf45 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 476c4dd372b..dfe2d909b43 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 f7a4651f6fd..5e14d4c5b38 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1267,7 +1267,7 @@ private: 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 @@ private: 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)); + 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()) + { + 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])); - projection_block = Block(cols); - } - - if (projection_block) - { 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 01183902a667d94d71ed9faabeffdc60cdcf95cd Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 28 May 2024 12:07:20 +0000 Subject: [PATCH 077/318] 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 dfe2d909b43..d0e9dc5f3ee 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 5e14d4c5b38..0e272fc8eb9 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1282,8 +1282,6 @@ private: 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 45f6c19c9df5c3f62b1ed4933321053ef6f77c91 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 28 May 2024 15:36:19 +0000 Subject: [PATCH 078/318] 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 0e272fc8eb9..8c4e0c6e654 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 079/318] 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 d0e9dc5f3ee..b95face57e1 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 080/318] 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 a2928e0eeb6..05259bbc0c3 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -32,10 +32,10 @@ public: 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 8c4e0c6e654..0a3a217d943 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 081/318] 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 ff1be9323f5..177d45650dd 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 082/318] 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 b95face57e1..af184940c7e 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 0a3a217d943..3469b609f6b 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 083/318] 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 ff1be9323f5..177d45650dd 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 b95face57e1..3cbaffe857a 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 0a3a217d943..af36b7bb3e8 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 084/318] 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 05259bbc0c3..84e67e5d4c1 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -32,10 +32,19 @@ public: 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 3cbaffe857a..77f84dba6e4 --- 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 af36b7bb3e8..ff1d7c0b7c2 --- 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 085/318] 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 05259bbc0c3..84e67e5d4c1 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -32,10 +32,19 @@ public: 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 3cbaffe857a..77f84dba6e4 --- 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 af36b7bb3e8..ff1d7c0b7c2 --- 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 086/318] 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 087/318] 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 177d45650dd..ff1be9323f5 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 088/318] 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 ff1be9323f5..177d45650dd 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 089/318] 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 9ecd92f732c..47add495421 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 ff1d7c0b7c2..2269b16b443 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 090/318] 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 47add495421..6706399a3d2 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 84e67e5d4c1..802e77847e9 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 7945bd97e04..1384f760d48 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 a516811bf45..67358316d48 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 77f84dba6e4..6973808078c 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 2269b16b443..5267143bf65 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 091/318] 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 82d80114a85..a15abf968d2 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 d9d430c1835..77191e63050 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -33,9 +33,10 @@ public: Chunk add(Chunk && input_chunk); + Block header; + private: Chunk accumulated_chunk; - const Block header; const ChunksToSquash * getInfoFromChunk(const Chunk & chunk); @@ -48,7 +49,8 @@ private: 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 @@ private: 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 96f41e37d2f..7945bd97e04 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 6f7c877b2f3..a516811bf45 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 ff1be9323f5..177d45650dd 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 476c4dd372b..06f20fef613 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 f7a4651f6fd..c24a643c374 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 092/318] 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 06f20fef613..070073f414d 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 0d1fc46ec76..e5285a970dd 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 093/318] 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 177d45650dd..ff1be9323f5 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 070073f414d..1dd99796754 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 e5285a970dd..0d1fc46ec76 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 094/318] [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 aa18817f4e6..af8244c06b8 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 095/318] 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 00000000000..0adde3a0d1f --- /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 bd8642b9f66..667f31b36b5 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 6e248dd664a..c2700a52be1 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 @@ private: 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 096/318] 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 0adde3a0d1f..fe8c6bc035b 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 097/318] 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 1ff97031acb..a74519a7608 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 098/318] 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 fe8c6bc035b..9a6ebe9fe81 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 099/318] 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 c2700a52be1..d1962ebfe65 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -327,7 +327,8 @@ private: 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 100/318] 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 d1962ebfe65..3a921987c49 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -327,7 +327,7 @@ private: 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 101/318] 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 a5cb5972bd8..7cd538e78b3 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 00000000000..4905c7f8a71 --- /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 00000000000..1301135b4cb --- /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 102/318] 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 7cd538e78b3..6ace1b3b5ce 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 1b9755a74d5..f43c83c2c95 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 77ef213b36d..4a4e898c5bd 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 9bb0c022752..70bcd7f255b 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 103/318] 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 ff1be9323f5..177d45650dd 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 104/318] 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 9d6395f73ac..21fdbc0b1a5 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 105/318] 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 8c5839a9803..b0ba4c3d40c 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 106/318] 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 86a36a9392c..9bd6ca234da 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 ca61a8f2d57..4f010ebadfd 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 107/318] 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 b0ba4c3d40c..0bf49b426e2 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 108/318] 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 c04bf30c7ae..9c3f35ff3dd 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 109/318] 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 4c678a1228b..f625b6a1820 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 00000000000..e2fa0e43e0c --- /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 00000000000..5c0751b7a35 --- /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 8a2e1870c757f3018698efc82f589cfc33c1c84a Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 7 Jun 2024 02:20:03 -0400 Subject: [PATCH 110/318] Fix alter query clone with SQL security --- src/Parsers/ASTAlterQuery.cpp | 3 +++ src/Parsers/ASTSQLSecurity.cpp | 2 +- ..._create_view_with_sql_security_option.reference | 3 +++ .../02884_create_view_with_sql_security_option.sh | 14 ++++++++++++++ 4 files changed, 21 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 90b63d2ce6f..58eeb7c4cbf 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -60,6 +60,8 @@ ASTPtr ASTAlterCommand::clone() const res->settings_resets = res->children.emplace_back(settings_resets->clone()).get(); if (select) res->select = res->children.emplace_back(select->clone()).get(); + if (sql_security) + res->sql_security = res->children.emplace_back(sql_security->clone()).get(); if (rename_to) res->rename_to = res->children.emplace_back(rename_to->clone()).get(); @@ -522,6 +524,7 @@ void ASTAlterCommand::forEachPointerToChild(std::function f) f(reinterpret_cast(&settings_changes)); f(reinterpret_cast(&settings_resets)); f(reinterpret_cast(&select)); + f(reinterpret_cast(&sql_security)); f(reinterpret_cast(&rename_to)); } diff --git a/src/Parsers/ASTSQLSecurity.cpp b/src/Parsers/ASTSQLSecurity.cpp index d6f1c21d035..74408747290 100644 --- a/src/Parsers/ASTSQLSecurity.cpp +++ b/src/Parsers/ASTSQLSecurity.cpp @@ -7,7 +7,7 @@ namespace DB void ASTSQLSecurity::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - if (!type.has_value()) + if (!type) return; if (definer || is_definer_current_user) diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference index 0589fdeef04..a03343c8cb3 100644 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference @@ -35,5 +35,8 @@ OK 2 2 6 6 9 9 +===== TestInsertChain ===== 1000 1000 +===== TestOnCluster ===== +1 diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh index f32aee44bee..dd869cd9988 100755 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh @@ -260,6 +260,8 @@ EOF ${CLICKHOUSE_CLIENT} --user $user2 --query "SELECT * FROM $db.test_mv_row_2" +echo "===== TestInsertChain =====" + ${CLICKHOUSE_CLIENT} --multiquery < Date: Thu, 6 Jun 2024 21:19:28 +0000 Subject: [PATCH 111/318] 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 ffaf53085c4..a0ee2ef0399 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 0323ae728a9..24557db4ee9 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 27ce54c03a7..2f85fb71e6f 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 814c08c5705..cda036c22ea 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 f1e47ea1158..8ac010deafc 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -207,7 +207,7 @@ public: 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 4a2d502a31a..801727e9eb9 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 @@ public: 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 @@ public: 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 @@ public: 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 @@ public: 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 @@ public: 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 @@ public: 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 @@ public: 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 @@ public: } -REGISTER_FUNCTION(DateTimeToSnowflake) +REGISTER_FUNCTION(LegacySnowflakeConversion) { - 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) -{ - 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 00000000000..abaf09b165b --- /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 837b0b4be20..e39b618b670 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 1090179bb67..0386717c933 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 00000000000..ab4e6770123 --- /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 00000000000..d4ea1d7efd0 --- /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 00000000000..9ed8c1dd3e5 --- /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 00000000000..b0e244ef814 --- /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 f1a50dd370d..1729a50ae44 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 49f43615c7e..0025214762e 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 112/318] 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 df8d86f379c..37a85666f3a 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 @@ public: } /// 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 @@ public: 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 @@ public: 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 @@ public: std::unique_lock lock{mutex}; while (true) { - if (!queue.empty()) + if (!events.empty() || !activations.empty()) { processQueue(std::move(lock)); return; @@ -269,141 +405,63 @@ private: 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 00000000000..0e281607bc2 --- /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 9703227ccfc..6cfccb252fa 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 651ca815420..339a59e88a5 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 113/318] 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 ab4e6770123..aaf5a2e3543 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 d4ea1d7efd0..ae8f7376697 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 114/318] 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 aaf5a2e3543..5dcd0c9dfcd 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 ae8f7376697..33bac8aaa35 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 115/318] 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 c87719b63a4..3178306363c 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 116/318] 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 4f1f0d337c0..443c39ec88b 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 117/318] 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 118/318] 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 b91596a555d..6f2ea51ef0f 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 119/318] 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 6f2ea51ef0f..98b3c9f7002 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 d53660dfd4ed70819643b1d3335b5c1eb333e62c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 7 Jun 2024 17:56:24 +0200 Subject: [PATCH 120/318] Fix crash on destroying AccessControl: add explicit shutdown. --- src/Access/AccessControl.cpp | 19 ++++++++++++++++++- src/Access/AccessControl.h | 3 +++ src/Access/DiskAccessStorage.cpp | 15 ++++++++++++--- src/Access/DiskAccessStorage.h | 2 ++ src/Access/IAccessStorage.h | 5 +++++ src/Access/MultipleAccessStorage.cpp | 26 ++++++++++++++++++++++++-- src/Access/MultipleAccessStorage.h | 3 +++ src/Access/ReplicatedAccessStorage.cpp | 12 ++++++++++++ src/Access/ReplicatedAccessStorage.h | 2 ++ src/Interpreters/Context.cpp | 3 +++ 10 files changed, 84 insertions(+), 6 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index c3bb42160ad..58ca5cdc435 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -261,7 +261,24 @@ AccessControl::AccessControl() } -AccessControl::~AccessControl() = default; +AccessControl::~AccessControl() +{ + try + { + shutdown(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + + +void AccessControl::shutdown() +{ + MultipleAccessStorage::shutdown(); + removeAllStorages(); +} void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index d1537219a06..bfaf256ad48 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -53,6 +53,9 @@ public: AccessControl(); ~AccessControl() override; + /// Shutdown the access control and stops all background activity. + void shutdown() override; + /// Initializes access storage (user directories). void setUpFromMainConfig(const Poco::Util::AbstractConfiguration & config_, const String & config_path_, const zkutil::GetZooKeeper & get_zookeeper_function_); diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index fe698b32816..c1633ee950f 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -194,11 +194,9 @@ DiskAccessStorage::DiskAccessStorage(const String & storage_name_, const String DiskAccessStorage::~DiskAccessStorage() { - stopListsWritingThread(); - try { - writeLists(); + shutdown(); } catch (...) { @@ -207,6 +205,17 @@ DiskAccessStorage::~DiskAccessStorage() } +void DiskAccessStorage::shutdown() +{ + stopListsWritingThread(); + + { + std::lock_guard lock{mutex}; + writeLists(); + } +} + + String DiskAccessStorage::getStorageParamsJSON() const { std::lock_guard lock{mutex}; diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 5d94008b34f..38172b26970 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -18,6 +18,8 @@ public: DiskAccessStorage(const String & storage_name_, const String & directory_path_, AccessChangesNotifier & changes_notifier_, bool readonly_, bool allow_backup_); ~DiskAccessStorage() override; + void shutdown() override; + const char * getStorageType() const override { return STORAGE_TYPE; } String getStorageParamsJSON() const override; diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index 4f980bf9212..e88b1601f32 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -44,6 +44,11 @@ public: explicit IAccessStorage(const String & storage_name_) : storage_name(storage_name_) {} virtual ~IAccessStorage() = default; + /// If the AccessStorage has to do some complicated work when destroying - do it in advance. + /// For example, if the AccessStorage contains any threads for background work - ask them to complete and wait for completion. + /// By default, does nothing. + virtual void shutdown() {} + /// Returns the name of this storage. const String & getStorageName() const { return storage_name; } virtual const char * getStorageType() const = 0; diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index a8b508202b5..35b94de38c7 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -34,11 +34,23 @@ MultipleAccessStorage::MultipleAccessStorage(const String & storage_name_) MultipleAccessStorage::~MultipleAccessStorage() { - /// It's better to remove the storages in the reverse order because they could depend on each other somehow. + try + { + shutdown(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +void MultipleAccessStorage::shutdown() +{ + /// It's better to shutdown the storages in the reverse order because they could depend on each other somehow. const auto storages = getStoragesPtr(); for (const auto & storage : *storages | boost::adaptors::reversed) { - removeStorage(storage); + storage->shutdown(); } } @@ -72,6 +84,16 @@ void MultipleAccessStorage::removeStorage(const StoragePtr & storage_to_remove) ids_cache.clear(); } +void MultipleAccessStorage::removeAllStorages() +{ + /// It's better to remove the storages in the reverse order because they could depend on each other somehow. + const auto storages = getStoragesPtr(); + for (const auto & storage : *storages | boost::adaptors::reversed) + { + removeStorage(storage); + } +} + std::vector MultipleAccessStorage::getStorages() { return *getStoragesPtr(); diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index 005e6e2b9cd..e1543c59b67 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -21,6 +21,8 @@ public: explicit MultipleAccessStorage(const String & storage_name_ = STORAGE_TYPE); ~MultipleAccessStorage() override; + void shutdown() override; + const char * getStorageType() const override { return STORAGE_TYPE; } bool isReadOnly() const override; bool isReadOnly(const UUID & id) const override; @@ -32,6 +34,7 @@ public: void setStorages(const std::vector & storages); void addStorage(const StoragePtr & new_storage); void removeStorage(const StoragePtr & storage_to_remove); + void removeAllStorages(); std::vector getStorages(); std::vector getStorages() const; std::shared_ptr> getStoragesPtr(); diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index cd9a86a1bd2..415b757483e 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -66,6 +66,18 @@ ReplicatedAccessStorage::ReplicatedAccessStorage( } ReplicatedAccessStorage::~ReplicatedAccessStorage() +{ + try + { + shutdown(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +void ReplicatedAccessStorage::shutdown() { stopWatchingThread(); } diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index cddb20860f7..f8518226997 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -23,6 +23,8 @@ public: ReplicatedAccessStorage(const String & storage_name, const String & zookeeper_path, zkutil::GetZooKeeper get_zookeeper, AccessChangesNotifier & changes_notifier_, bool allow_backup); ~ReplicatedAccessStorage() override; + void shutdown() override; + const char * getStorageType() const override { return STORAGE_TYPE; } void startPeriodicReloading() override { startWatchingThread(); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 5c9ae4716b9..df86f156a4e 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -674,6 +674,9 @@ struct ContextSharedPart : boost::noncopyable } } + LOG_TRACE(log, "Shutting down AccessControl"); + access_control->shutdown(); + { std::lock_guard lock(mutex); From a7acc79fe5eb8def7509369e7d70bb6c8fcef295 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Mon, 10 Jun 2024 10:52:35 -0700 Subject: [PATCH 121/318] Fix broken multi-column aggregation on s390x --- src/Interpreters/AggregationCommon.h | 5 +---- src/Interpreters/AggregationMethod.cpp | 5 +---- 2 files changed, 2 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/AggregationCommon.h b/src/Interpreters/AggregationCommon.h index ab078d1c5e5..43c80d361d1 100644 --- a/src/Interpreters/AggregationCommon.h +++ b/src/Interpreters/AggregationCommon.h @@ -90,10 +90,7 @@ void fillFixedBatch(size_t keys_size, const ColumnRawPtrs & key_columns, const S /// Note: here we violate strict aliasing. /// It should be ok as log as we do not reffer to any value from `out` before filling. const char * source = static_cast(column)->getRawDataBegin(); - size_t offset_to = offset; - if constexpr (std::endian::native == std::endian::big) - offset_to = sizeof(Key) - sizeof(T) - offset; - T * dest = reinterpret_cast(reinterpret_cast(out.data()) + offset_to); + T * dest = reinterpret_cast(reinterpret_cast(out.data()) + offset); fillFixedBatch(num_rows, reinterpret_cast(source), dest); /// NOLINT(bugprone-sizeof-expression) offset += sizeof(T); } diff --git a/src/Interpreters/AggregationMethod.cpp b/src/Interpreters/AggregationMethod.cpp index 3ff4f0cae43..0fc789528b8 100644 --- a/src/Interpreters/AggregationMethod.cpp +++ b/src/Interpreters/AggregationMethod.cpp @@ -160,10 +160,7 @@ void AggregationMethodKeysFixedinsertData(reinterpret_cast(&key) + offset_to, size); + observed_column->insertData(reinterpret_cast(&key) + pos, size); pos += size; } } From 68e1d8701c973def035f0b9ea17f6ca3d224d73c Mon Sep 17 00:00:00 2001 From: yariks5s Date: Mon, 10 Jun 2024 18:09:07 +0000 Subject: [PATCH 122/318] 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 e632886778f..d735fb8a55c 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 47add495421..a05c5853ce3 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 ApplySquashing::add(Chunk && input_chunk) +Chunk Squashing::flush() +{ + return convertToChunk(std::move(chunks_to_merge_vec)); +} + +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 77191e63050..760b7d7475f 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 @@ private: 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 e63691fcc6a..7bf1f32340b 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 @@ public: 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 @@ protected: } 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 7945bd97e04..f8d5143493f 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 7afc942a7f2..a9152d9dbe9 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 @@ public: 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 a516811bf45..c1f8a9f2513 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 b5b3c6616d2..c5b727ac6ec 100644 --- a/src/Processors/Transforms/SquashingTransform.h +++ b/src/Processors/Transforms/SquashingTransform.h @@ -24,8 +24,7 @@ protected: void onFinish() override; private: - PlanSquashing planSquashing; - ApplySquashing applySquashing; + Squashing squashing; Chunk cur_chunk; Chunk finish_chunk; }; @@ -44,8 +43,7 @@ protected: 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 1dd99796754..2be4e8d5665 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 0d1fc46ec76..fad195d6a36 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1267,8 +1267,7 @@ private: 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 c88582a8a1a..37ea3f274b6 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 123/318] 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 a05c5853ce3..05d99d89b15 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 124/318] 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 a05c5853ce3..588b75d33de 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 760b7d7475f..15b543be236 100644 --- a/src/Interpreters/Squashing.h +++ b/src/Interpreters/Squashing.h @@ -55,11 +55,10 @@ private: 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 f8d5143493f..5ae605cc51a 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 125/318] 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 2c0e7859d44..7964379a35d 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 15b543be236..ea991d6dc85 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 @@ public: return !chunks_to_merge_vec.empty(); } - Block header; - private: struct CurrentSize { @@ -56,9 +54,9 @@ private: 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 4f753798eaa..3c0952f7aed 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -89,7 +89,7 @@ public: 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 7bf1f32340b..2de614b8dbd 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 @@ public: 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 @@ protected: } 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 5ae605cc51a..71e4ee15ecc 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 c1f8a9f2513..34c5b1a7202 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 cfabc9cef34..940fa94e182 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 1d0c24c0f08..6fcc8f4ad92 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 126/318] 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 54db8980e12..271f8f7474e 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 150fa84762f..ae2cb4129d5 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -103,13 +103,11 @@ private: 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 @@ public: /// 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 0bdd4c089f1..b41c4509bd3 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -486,16 +486,12 @@ public: } }; -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 046c7387ee8..a0064637939 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 @@ public: 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 @@ public: 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 04f29f35c3c..3ba448a0da0 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 cb467004d29..b8ac10fd80e 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 d80d5cd5b93..530bace3b1e 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 941194e69ff..43608cab412 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -117,7 +117,7 @@ public: /// 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 @@ public: 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 @@ protected: /// 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 @@ public: /// 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 @@ private: */ /// 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 @@ private: 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 c26e4517c9a..15004189f77 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 127/318] 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 7964379a35d..f8b6a6542cc 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 ea991d6dc85..d76cca60e41 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 @@ public: 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 2de614b8dbd..965a084bb13 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 @@ protected: } 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 71e4ee15ecc..5ae605cc51a 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 34c5b1a7202..34b733cde5e 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 940fa94e182..f9b24139d86 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 6fcc8f4ad92..4ea0e9ece4f 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 128/318] 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 f8b6a6542cc..7964379a35d 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 d76cca60e41..ea991d6dc85 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 @@ public: 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 965a084bb13..2de614b8dbd 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 @@ protected: } 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 5ae605cc51a..71e4ee15ecc 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 34b733cde5e..34c5b1a7202 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 f9b24139d86..940fa94e182 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 4ea0e9ece4f..6fcc8f4ad92 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 129/318] 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 3c0952f7aed..4f753798eaa 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -89,7 +89,7 @@ public: 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 130/318] 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 1cef43530e0..aab7a39534d 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 131/318] 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 7964379a35d..f8b6a6542cc 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 ea991d6dc85..d76cca60e41 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 @@ public: 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 2de614b8dbd..965a084bb13 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 @@ protected: } 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 71e4ee15ecc..5ae605cc51a 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 34c5b1a7202..34b733cde5e 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 940fa94e182..f9b24139d86 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 6fcc8f4ad92..4ea0e9ece4f 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 132/318] 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 4c8f1240cf5..f3a54092a75 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 17760a93dfc29c78861799b1d5632db49b2eb5ac Mon Sep 17 00:00:00 2001 From: 0x01f Date: Tue, 28 May 2024 13:07:23 +0800 Subject: [PATCH 133/318] Add SelectQueriesWithPrimaryKeyUsage in ProfileEvents --- src/Common/ProfileEvents.cpp | 1 + .../QueryPlan/ReadFromMergeTree.cpp | 15 +++++--- ...t_queries_with_primary_key_usage.reference | 24 +++++++++++++ ..._select_queries_with_primary_key_usage.sql | 35 +++++++++++++++++++ 4 files changed, 71 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.reference create mode 100644 tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.sql diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 8c8e2163aad..10c08b60dde 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -14,6 +14,7 @@ M(QueriesWithSubqueries, "Count queries with all subqueries") \ M(SelectQueriesWithSubqueries, "Count SELECT queries with all subqueries") \ M(InsertQueriesWithSubqueries, "Count INSERT queries with all subqueries") \ + M(SelectQueriesWithPrimaryKeyUsage, "Count SELECT queries with primary key as a WHERE condition.") \ M(AsyncInsertQuery, "Same as InsertQuery, but only for asynchronous INSERT queries.") \ M(AsyncInsertBytes, "Data size in bytes of asynchronous INSERT queries.") \ M(AsyncInsertRows, "Number of rows inserted by asynchronous INSERT queries.") \ diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6f0fa55c349..8b14034c63a 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -120,6 +120,7 @@ namespace ProfileEvents extern const Event SelectedParts; extern const Event SelectedRanges; extern const Event SelectedMarks; + extern const Event SelectQueriesWithPrimaryKeyUsage; } namespace DB @@ -1617,11 +1618,17 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( if (indexes->part_values && indexes->part_values->empty()) return std::make_shared(std::move(result)); - if (settings.force_primary_key && indexes->key_condition.alwaysUnknownOrTrue()) + if (indexes->key_condition.alwaysUnknownOrTrue()) { - throw Exception(ErrorCodes::INDEX_NOT_USED, - "Primary key ({}) is not used and setting 'force_primary_key' is set", - fmt::join(primary_key_column_names, ", ")); + if (settings.force_primary_key) + { + throw Exception(ErrorCodes::INDEX_NOT_USED, + "Primary key ({}) is not used and setting 'force_primary_key' is set", + fmt::join(primary_key_column_names, ", ")); + } + } else + { + ProfileEvents::increment(ProfileEvents::SelectQueriesWithPrimaryKeyUsage); } LOG_DEBUG(log, "Key condition: {}", indexes->key_condition.toString()); diff --git a/tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.reference b/tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.reference new file mode 100644 index 00000000000..d5563921583 --- /dev/null +++ b/tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.reference @@ -0,0 +1,24 @@ +1 +100 +100 +100 +100 +Row 1: +────── +queries_with_primary_key_usage: 1 + +Row 2: +────── +queries_with_primary_key_usage: 1 + +Row 3: +────── +queries_with_primary_key_usage: 1 + +Row 4: +────── +queries_with_primary_key_usage: 0 + +Row 5: +────── +queries_with_primary_key_usage: 0 diff --git a/tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.sql b/tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.sql new file mode 100644 index 00000000000..6b7aa6fb3ee --- /dev/null +++ b/tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.sql @@ -0,0 +1,35 @@ +-- Tests for ProfileEvents "SelectedMarksByPrimaryKeyUsage" +set log_queries=1; +set log_queries_min_type='QUERY_FINISH'; + +DROP TABLE IF EXISTS 03164_select_queries_with_primary_key_usage; +CREATE TABLE 03164_select_queries_with_primary_key_usage +( + a Int64, b Int64, c Int64 +) ENGINE = MergeTree ORDER BY a; + +ALTER TABLE 03164_select_queries_with_primary_key_usage ADD PROJECTION b_projection ( + SELECT * ORDER BY b +); + +INSERT INTO 03164_select_queries_with_primary_key_usage SELECT number, number + 100, number + 1000 FROM numbers(100); + +SELECT count(*) FROM 03164_select_queries_with_primary_key_usage WHERE a >= 0 and b <= 100; +SELECT count(*) FROM 03164_select_queries_with_primary_key_usage WHERE a >= 0; +SELECT count(*) FROM 03164_select_queries_with_primary_key_usage WHERE b >= 100; +SELECT count(*) FROM 03164_select_queries_with_primary_key_usage WHERE c >= 1000; +SELECT count(*) FROM 03164_select_queries_with_primary_key_usage; + +SYSTEM FLUSH LOGS; + +SELECT + IF (ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] > 0, 1, 0) AS queries_with_primary_key_usage +FROM + system.query_log +WHERE + current_database = currentDatabase() + AND query LIKE 'SELECT count(*) FROM 03164_select_queries_with_primary_key_usage%' +ORDER BY query +FORMAT Vertical; + +DROP TABLE IF EXISTS 03164_select_queries_with_primary_key_usage; From 6e9285791d7c8b3812f038114618bf1abf694997 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 12 Jun 2024 13:34:03 +0200 Subject: [PATCH 134/318] Revert "Revert "Small fix for 02340_parts_refcnt_mergetree"" --- tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index b100f96befa..e7d95d8db72 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -58,7 +58,7 @@ function check_refcnt_for_table() $CLICKHOUSE_CLIENT -q "select table, name, refcount>=6 from system.parts where database = '$CLICKHOUSE_DATABASE' and table = '$table' and refcount >= 3" # Kill the query gracefully. - kill -INT $PID + kill -INT $PID ||: wait $PID grep -F Exception "$log_file" | grep -v -F QUERY_WAS_CANCELLED rm -f "${log_file:?}" From 089f5bfecdf26d18a5e3dda3e3f8f2c4a86deac5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 12 Jun 2024 15:09:38 +0200 Subject: [PATCH 135/318] 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 4f8ecf6a813..cae819fa982 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 136/318] 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 e36d548e0b8..876cee60baa 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 137/318] 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 271f8f7474e..da53f197fd8 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 ae2cb4129d5..d251f66a129 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -166,9 +166,12 @@ public: /// 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 b41c4509bd3..b292d02ccaa 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 a0064637939..e13bd82be06 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 3ba448a0da0..d25d5b53226 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 b8ac10fd80e..7c6af41c04d 100644 --- a/src/Interpreters/ExpressionActions.h +++ b/src/Interpreters/ExpressionActions.h @@ -79,11 +79,13 @@ private: 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 530bace3b1e..c53cdd0d2ed 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 43608cab412..6fa50bb70c9 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -115,10 +115,10 @@ public: /// 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 927bafe4bfb..91bfa863bd9 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 b72399df2c1..fac3c0637b5 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 e89a1e5febf..c4012180b0f 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -174,13 +174,13 @@ private: /// 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 @@ private: 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 ba33b70b59c..309ab2691a9 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 15004189f77..c26e4517c9a 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 c5438b5d2d4..85d50429bab 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 4907fdbad87..ab5823b99d7 100644 --- a/src/Planner/ActionsChain.h +++ b/src/Planner/ActionsChain.h @@ -48,18 +48,18 @@ public: * 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 @@ public: 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 27b5909c13b..d5e39a9f123 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -90,7 +90,7 @@ public: ActionsDAGPtr alias_column_actions_dag = std::make_shared(); 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 b40e23a9553..b117a0cd34b 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 @@ public: }; 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 837307ba2ca..f5e71acee38 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 8506c309171..6bb32047327 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -37,7 +37,7 @@ public: * 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 f0a2845c3e8..b2223d9c51d 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 0773272e49a..3a9ed903bbc 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 efc449402b9..18c3744f864 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 c410b04f209..84efdd21336 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 4a74bf413d3..18a6d297838 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 4706f552c9d..3172847f053 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 c175cd516ac..537555afa2a 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 4017670ad14..7eca8ed74f9 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 887a95da60d..750bb5600b3 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 e6c2bcec2c8..7945b3999c1 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 cd706dab9ae..ea1d9cd8131 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 735f51e1f32..9625c13197c 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 4c678a1228b..bb27715ddd1 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 7b089e9334595578abd25b868347c007f3880505 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Wed, 12 Jun 2024 19:47:17 -0700 Subject: [PATCH 138/318] Dummy commit to trigger a build From 2de54fa4d304ea1db42e7c4cc608bbc236b39d93 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 13 Jun 2024 12:21:49 +0100 Subject: [PATCH 139/318] incorporate suggestion from azat --- tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index e7d95d8db72..5720a8a69e9 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -16,7 +16,7 @@ function check_refcnt_for_table() -- queue may hold the parts lock for awhile as well system stop pulling replication log $table; " - $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into $table select number, number%4 from numbers(200)" + $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into $table select number, number%4 from numbers(2000)" local query_id query_id="$table-$(random_str 10)" @@ -35,7 +35,7 @@ function check_refcnt_for_table() ) # Notes: - # - query may sleep 0.1*(200/4)=5 seconds maximum, it is enough to check system.parts + # - query may sleep 0.1*(2000/4)=5 seconds maximum, it is enough to check system.parts # - "part = 1" condition should prune all parts except first # - max_block_size=1 with index_granularity=1 will allow to cancel the query earlier $CLICKHOUSE_CLIENT "${args[@]}" -q "select sleepEachRow(0.1) from $table where part = 1" & From cb9ea78810e1ccb894fae0831c299a7e5958f3af Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 6 Jun 2024 08:13:51 +0000 Subject: [PATCH 140/318] 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 00000000000..3ee56295b2e --- /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 00000000000..97651d1b0fd --- /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 141/318] 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 3ee56295b2e..00000000000 --- 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 97651d1b0fd..00000000000 --- 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 142/318] 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 443c39ec88b..55cdc9c4aff 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 143/318] 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 79d0ca4f151..76c59a7be50 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 00000000000..a223737eae6 --- /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 00000000000..207f54587c9 --- /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 144/318] 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 5ae605cc51a..8598a29278a 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 145/318] 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 fb4d9a6bdcc..8c993f906e0 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 146/318] 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 da53f197fd8..92d8e3ace9e 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 c53cdd0d2ed..b4fa852916e 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 147/318] 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 dfb41a33fb1..324a0170bdd 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 b9506cc5b7f..e2187bfeb66 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 @@ public: /// 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 @@ private: 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 6335ee82b47..f57c81e4128 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 @@ private: 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 @@ public: ThreadPool* pool() { FastMutex::ScopedLock lock(_mutex); - + if (!_pPool) { _pPool = new ThreadPool("default"); @@ -490,7 +515,7 @@ public: } return _pPool; } - + private: ThreadPool* _pPool; FastMutex _mutex; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c3a5be706b4..8a5f6173c49 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 02d0959ff50..d6afa571e71 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 0d79aaa227b..45c28babe3a 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 6709cd298e5..4612e2e9fa8 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 6456f6d24ff..9471509ad4b 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 473d681ddb2..8ba8421e6f0 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 e3a820340ad..b60339e9fd8 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 191617f1905..75e36836b63 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 148/318] 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 2e597a7c466..738b8c1e9d5 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 149/318] 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 b292d02ccaa..89de76d2815 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 e13bd82be06..7e3b7fcb6bb 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 d25d5b53226..e19c4b0cd20 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 91bfa863bd9..927bafe4bfb 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 acea74eff4e2b830208d3e3081250ec9b3a82558 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Thu, 13 Jun 2024 14:03:55 -0700 Subject: [PATCH 150/318] Another dummy commit to trigger a build From fcc16b9f5229dde2207cf3789fe800a03548b682 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Fri, 14 Jun 2024 02:08:17 +0300 Subject: [PATCH 151/318] 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 9a6ebe9fe81..a6437019bd3 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 55cdc9c4aff..7e4b1db4c89 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 7f92206dd24..6e5956706aa 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 207f54587c9..590f979efb2 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 152/318] 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 f50dae0f1a2..6c4a9fa39d6 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 29993ed33e4..b1b8e2367a4 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 309becdd78f..8261db6ce8e 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 e72834eddbe..feb7113e6f9 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 66936dc25d7..b0351a004a4 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 00000000000..e0d61993b11 --- /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 00000000000..e0d61993b11 --- /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 00000000000..127153956f9 --- /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 00000000000..386f3d2fd86 --- /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 00000000000..365fad5e99d --- /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 153/318] 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 feb7113e6f9..233db07cd68 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 fd38a31da5c..8c509a1dd53 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 154/318] 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 8c509a1dd53..c590486d10e 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 365fad5e99d..f665c7ce4bf 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 e0d61993b11..00000000000 --- 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 155/318] 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 b0351a004a4..30d9ddb93f5 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 156/318] 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 f665c7ce4bf..440173393d0 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 157/318] 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 30d9ddb93f5..1904dd58d9f 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 440173393d0..c0b0639dec4 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 158/318] 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 00000000000..9a573b158fe --- /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 00000000000..d5bd2228799 --- /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 e0d61993b11..00000000000 --- 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 127153956f9..00000000000 --- 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 386f3d2fd86..00000000000 --- 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 c0b0639dec4..00000000000 --- 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 159/318] 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 00000000000..e69de29bb2d diff --git a/tests/integration/test_table_db_num_limit/test.py b/tests/integration/test_table_db_num_limit/test.py index d5bd2228799..71bb494e04e 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 = cluster.add_instance( "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 160/318] 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 37125d9900c..ac9c01e5dd7 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -129,6 +129,7 @@ public: 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 1904dd58d9f..697e8136ee1 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 71bb494e04e..bde9a956d70 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 161/318] 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 697e8136ee1..c33f97dc80d 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 162/318] trigger CI From c63ceb33dbd269d113e806103257f31342304d5a Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Thu, 13 Jun 2024 17:37:54 -0700 Subject: [PATCH 163/318] One more dummy commit to trigger a build From 08b481af3d8d59088216df5bc182871ef73284ff Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Fri, 14 Jun 2024 09:44:08 +0800 Subject: [PATCH 164/318] 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 165/318] 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 a223737eae6..6e8a5df9145 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 590f979efb2..2d566e52c94 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 166/318] 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 167/318] 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 51d602b0bc8..68967f331f0 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 8e8183c0762..5496a7e4257 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 fc3cd16f8ed..43b223172e6 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 9d416da6010db16d0cde24d0c3fba1fe4dab97a1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 Jun 2024 10:23:26 +0000 Subject: [PATCH 168/318] 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 92d8e3ace9e..c01142ff01a 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 8ee3654067c..5f94367b46d 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 169/318] 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 b4fa852916e..2e7bc69402f 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 68967f331f0..a1ad10ff623 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 fc6f2aa59ad015821253744e8a45d8cbe23ee833 Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Fri, 14 Jun 2024 14:31:58 +0200 Subject: [PATCH 170/318] Small refactoring of CertificateReloader --- src/Server/CertificateReloader.cpp | 42 ++++++++++++++++-------------- src/Server/CertificateReloader.h | 12 +++++---- 2 files changed, 30 insertions(+), 24 deletions(-) diff --git a/src/Server/CertificateReloader.cpp b/src/Server/CertificateReloader.cpp index a89634e05e4..989a06876f7 100644 --- a/src/Server/CertificateReloader.cpp +++ b/src/Server/CertificateReloader.cpp @@ -89,10 +89,29 @@ void CertificateReloader::tryLoad(const Poco::Util::AbstractConfiguration & conf void CertificateReloader::tryLoad(const Poco::Util::AbstractConfiguration & config, SSL_CTX * ctx, const std::string & prefix) { - std::unique_lock lock(data_mutex); + std::lock_guard lock{data_mutex}; tryLoadImpl(config, ctx, prefix); } + +std::list::iterator CertificateReloader::findOrInsert(SSL_CTX * ctx, const std::string & prefix) +{ + auto it = data.end(); + auto i = data_index.find(prefix); + if (i != data_index.end()) + it = i->second; + else + { + if (!ctx) + ctx = Poco::Net::SSLManager::instance().defaultServerContext()->sslContext(); + data.push_back(MultiData(ctx)); + --it; + data_index[prefix] = it; + } + return it; +} + + void CertificateReloader::tryLoadImpl(const Poco::Util::AbstractConfiguration & config, SSL_CTX * ctx, const std::string & prefix) { /// If at least one of the files is modified - recreate @@ -109,24 +128,15 @@ void CertificateReloader::tryLoadImpl(const Poco::Util::AbstractConfiguration & } else { - auto it = data.end(); - auto i = data_index.find(prefix); - if (i != data_index.end()) - it = i->second; - else - { - data.push_back(MultiData(ctx)); - --it; - data_index[prefix] = it; - } + auto it = findOrInsert(ctx, prefix); bool cert_file_changed = it->cert_file.changeIfModified(std::move(new_cert_path), log); bool key_file_changed = it->key_file.changeIfModified(std::move(new_key_path), log); - std::string pass_phrase = config.getString(prefix + "privateKeyPassphraseHandler.options.password", ""); if (cert_file_changed || key_file_changed) { LOG_DEBUG(log, "Reloading certificate ({}) and key ({}).", it->cert_file.path, it->key_file.path); + std::string pass_phrase = config.getString(prefix + "privateKeyPassphraseHandler.options.password", ""); it->data.set(std::make_unique(it->cert_file.path, it->key_file.path, pass_phrase)); LOG_INFO(log, "Reloaded certificate ({}) and key ({}).", it->cert_file.path, it->key_file.path); } @@ -134,12 +144,6 @@ void CertificateReloader::tryLoadImpl(const Poco::Util::AbstractConfiguration & /// If callback is not set yet try { - if (!ctx) - { - ctx = Poco::Net::SSLManager::instance().defaultServerContext()->sslContext(); - it->ctx = ctx; - } - if (it->init_was_not_made) init(&*it); } @@ -154,7 +158,7 @@ void CertificateReloader::tryLoadImpl(const Poco::Util::AbstractConfiguration & void CertificateReloader::tryReloadAll(const Poco::Util::AbstractConfiguration & config) { - std::unique_lock lock(data_mutex); + std::lock_guard lock{data_mutex}; for (auto & item : data_index) tryLoadImpl(config, item.second->ctx, item.first); } diff --git a/src/Server/CertificateReloader.h b/src/Server/CertificateReloader.h index 40b81f7a06f..f58e2964f57 100644 --- a/src/Server/CertificateReloader.h +++ b/src/Server/CertificateReloader.h @@ -90,16 +90,18 @@ private: CertificateReloader() = default; /// Initialize the callback and perform the initial cert loading - void init(MultiData * pdata); + void init(MultiData * pdata) TSA_REQUIRES(data_mutex); /// Unsafe implementation - void tryLoadImpl(const Poco::Util::AbstractConfiguration & config, SSL_CTX * ctx, const std::string & prefix); + void tryLoadImpl(const Poco::Util::AbstractConfiguration & config, SSL_CTX * ctx, const std::string & prefix) TSA_REQUIRES(data_mutex); + + std::list::iterator findOrInsert(SSL_CTX * ctx, const std::string & prefix) TSA_REQUIRES(data_mutex); LoggerPtr log = getLogger("CertificateReloader"); - std::mutex data_mutex; - std::list data; - std::unordered_map::iterator> data_index; + std::list data TSA_GUARDED_BY(data_mutex); + std::unordered_map::iterator> data_index TSA_GUARDED_BY(data_mutex); + mutable std::mutex data_mutex; }; } From b43240068600c806679a061c2359c728e62dd22f Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Jun 2024 13:10:04 +0000 Subject: [PATCH 171/318] 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 37a85666f3a..3f699c881fb 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 @@ public: 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 @@ public: 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 172/318] 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 c590486d10e..6426123bb4f 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 173/318] 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 bde9a956d70..f2080ec4738 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 3bc720e7a7824e81137504e9eb684e268d7654e1 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 14 Jun 2024 15:33:19 +0200 Subject: [PATCH 174/318] Fix nonsense docs --- .../reference/stddevpop.md | 2 +- .../aggregate-functions/reference/varpop.md | 81 +++---------- .../reference/varpopstable.md | 52 +++++++++ .../aggregate-functions/reference/varsamp.md | 106 ++++-------------- .../reference/varsampstable.md | 63 +++++++++++ 5 files changed, 156 insertions(+), 148 deletions(-) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/varpopstable.md create mode 100644 docs/en/sql-reference/aggregate-functions/reference/varsampstable.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/stddevpop.md b/docs/en/sql-reference/aggregate-functions/reference/stddevpop.md index d2406197ecc..ece9f722c45 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/stddevpop.md +++ b/docs/en/sql-reference/aggregate-functions/reference/stddevpop.md @@ -25,7 +25,7 @@ stddevPop(x) **Returned value** -Square root of standard deviation of `x`. [Float64](../../data-types/float.md). +- Square root of standard deviation of `x`. [Float64](../../data-types/float.md). **Example** diff --git a/docs/en/sql-reference/aggregate-functions/reference/varpop.md b/docs/en/sql-reference/aggregate-functions/reference/varpop.md index fcabeb4c6a8..be57115f146 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varpop.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varpop.md @@ -4,30 +4,29 @@ slug: "/en/sql-reference/aggregate-functions/reference/varpop" sidebar_position: 32 --- -This page covers the `varPop` and `varPopStable` functions available in ClickHouse. - ## varPop -Calculates the population covariance between two data columns. The population covariance measures the degree to which two variables vary together. Calculates the amount `Σ((x - x̅)^2) / n`, where `n` is the sample size and `x̅`is the average value of `x`. +Calculates the population variance. + +:::note +This function uses a numerically unstable algorithm. If you need [numerical stability](https://en.wikipedia.org/wiki/Numerical_stability) in calculations, use the [`varPopStable`](../reference/varpopstable.md) function. It works slower but provides a lower computational error. +::: **Syntax** ```sql -covarPop(x, y) +varPop(x) ``` +Alias: `VAR_POP`. + **Parameters** -- `x`: The first data column. [Numeric](../../../native-protocol/columns.md) -- `y`: The second data column. [Numeric](../../../native-protocol/columns.md) +- `x`: Population of values to find the population variance of. [(U)Int*](../../data-types/int-uint.md), [Float*](../../data-types/float.md), [Decimal*](../../data-types/decimal.md). **Returned value** -Returns an integer of type `Float64`. - -**Implementation details** - -This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varPopStable` function](#varPopStable). +- Returns the population variance of `x`. [`Float64`](../../data-types/float.md). **Example** @@ -37,69 +36,21 @@ Query: DROP TABLE IF EXISTS test_data; CREATE TABLE test_data ( - x Int32, - y Int32 + x UInt8, ) ENGINE = Memory; -INSERT INTO test_data VALUES (1, 2), (2, 3), (3, 5), (4, 6), (5, 8); +INSERT INTO test_data VALUES (3), (3), (3), (4), (4), (5), (5), (7), (11), (15); SELECT - covarPop(x, y) AS covar_pop + varPop(x) AS var_pop FROM test_data; ``` Result: ```response -3 -``` - -## varPopStable - -Calculates population covariance between two data columns using a stable, numerically accurate method to calculate the variance. This function is designed to provide reliable results even with large datasets or values that might cause numerical instability in other implementations. - -**Syntax** - -```sql -covarPopStable(x, y) -``` - -**Parameters** - -- `x`: The first data column. [String literal](../../syntax#syntax-string-literal) -- `y`: The second data column. [Expression](../../syntax#syntax-expressions) - -**Returned value** - -Returns an integer of type `Float64`. - -**Implementation details** - -Unlike [`varPop()`](#varPop), this function uses a stable, numerically accurate algorithm to calculate the population variance to avoid issues like catastrophic cancellation or loss of precision. This function also handles `NaN` and `Inf` values correctly, excluding them from calculations. - -**Example** - -Query: - -```sql -DROP TABLE IF EXISTS test_data; -CREATE TABLE test_data -( - x Int32, - y Int32 -) -ENGINE = Memory; - -INSERT INTO test_data VALUES (1, 2), (2, 9), (9, 5), (4, 6), (5, 8); - -SELECT - covarPopStable(x, y) AS covar_pop_stable -FROM test_data; -``` - -Result: - -```response -0.5999999999999999 +┌─var_pop─┐ +│ 14.4 │ +└─────────┘ ``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/varpopstable.md b/docs/en/sql-reference/aggregate-functions/reference/varpopstable.md new file mode 100644 index 00000000000..811506c2fa8 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/varpopstable.md @@ -0,0 +1,52 @@ +--- +title: "varPopStable" +slug: "/en/sql-reference/aggregate-functions/reference/varpopstable" +sidebar_position: 32 +--- + +## varPopStable + +Returns the population variance. Unlike [`varPop`](../reference/varpop.md), this function uses a [numerically stable](https://en.wikipedia.org/wiki/Numerical_stability) algorithm. It works slower but provides a lower computational error. + +**Syntax** + +```sql +varPopStable(x) +``` + +Alias: `VAR_POP_STABLE`. + +**Parameters** + +- `x`: Population of values to find the population variance of. [(U)Int*](../../data-types/int-uint.md), [Float*](../../data-types/float.md), [Decimal*](../../data-types/decimal.md). + +**Returned value** + +- Returns the population variance of `x`. [Float64](../../data-types/float.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test_data; +CREATE TABLE test_data +( + x UInt8, +) +ENGINE = Memory; + +INSERT INTO test_data VALUES (3),(3),(3),(4),(4),(5),(5),(7),(11),(15); + +SELECT + varPopStable(x) AS var_pop_stable +FROM test_data; +``` + +Result: + +```response +┌─var_pop_stable─┐ +│ 14.4 │ +└────────────────┘ +``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/varsamp.md b/docs/en/sql-reference/aggregate-functions/reference/varsamp.md index be669a16ae8..4182265cff2 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varsamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varsamp.md @@ -4,33 +4,37 @@ slug: /en/sql-reference/aggregate-functions/reference/varsamp sidebar_position: 33 --- -This page contains information on the `varSamp` and `varSampStable` ClickHouse functions. - ## varSamp Calculate the sample variance of a data set. +:::note +This function uses a numerically unstable algorithm. If you need [numerical stability](https://en.wikipedia.org/wiki/Numerical_stability) in calculations, use the [`varSampStable`](../reference/varsampstable.md) function. It works slower but provides a lower computational error. +::: + **Syntax** ```sql -varSamp(expr) +varSamp(x) ``` +Alias: `VAR_SAMP`. + **Parameters** -- `expr`: An expression representing the data set for which you want to calculate the sample variance. [Expression](../../syntax#syntax-expressions) +- `x`: The population for which you want to calculate the sample variance. [(U)Int*](../../data-types/int-uint.md), [Float*](../../data-types/float.md), [Decimal*](../../data-types/decimal.md). **Returned value** -Returns a Float64 value representing the sample variance of the input data set. +- Returns the sample variance of the input data set `x`. [Float64](../../data-types/float.md). **Implementation details** -The `varSamp()` function calculates the sample variance using the following formula: +The `varSamp` function calculates the sample variance using the following formula: -```plaintext -∑(x - mean(x))^2 / (n - 1) -``` +$$ +\sum\frac{(x - \text{mean}(x))^2}{(n - 1)} +$$ Where: @@ -38,91 +42,29 @@ Where: - `mean(x)` is the arithmetic mean of the data set. - `n` is the number of data points in the data set. -The function assumes that the input data set represents a sample from a larger population. If you want to calculate the variance of the entire population (when you have the complete data set), you should use the [`varPop()` function](./varpop#varpop) instead. - -This function uses a numerically unstable algorithm. If you need numerical stability in calculations, use the slower but more stable [`varSampStable` function](#varSampStable). +The function assumes that the input data set represents a sample from a larger population. If you want to calculate the variance of the entire population (when you have the complete data set), you should use [`varPop`](../reference/varpop.md) instead. **Example** Query: ```sql -CREATE TABLE example_table +DROP TABLE IF EXISTS test_data; +CREATE TABLE test_data ( - id UInt64, - value Float64 + x Float64 ) -ENGINE = MergeTree -ORDER BY id; +ENGINE = Memory; -INSERT INTO example_table VALUES (1, 10.5), (2, 12.3), (3, 9.8), (4, 11.2), (5, 10.7); +INSERT INTO test_data VALUES (10.5), (12.3), (9.8), (11.2), (10.7); -SELECT varSamp(value) FROM example_table; +SELECT round(varSamp(x),3) AS var_samp FROM test_data; ``` Response: ```response -0.8650000000000091 -``` - -## varSampStable - -Calculate the sample variance of a data set using a numerically stable algorithm. - -**Syntax** - -```sql -varSampStable(expr) -``` - -**Parameters** - -- `expr`: An expression representing the data set for which you want to calculate the sample variance. [Expression](../../syntax#syntax-expressions) - -**Returned value** - -The `varSampStable()` function returns a Float64 value representing the sample variance of the input data set. - -**Implementation details** - -The `varSampStable()` function calculates the sample variance using the same formula as the [`varSamp()`](#varSamp function): - -```plaintext -∑(x - mean(x))^2 / (n - 1) -``` - -Where: -- `x` is each individual data point in the data set. -- `mean(x)` is the arithmetic mean of the data set. -- `n` is the number of data points in the data set. - -The difference between `varSampStable()` and `varSamp()` is that `varSampStable()` is designed to provide a more deterministic and stable result when dealing with floating-point arithmetic. It uses an algorithm that minimizes the accumulation of rounding errors, which can be particularly important when dealing with large data sets or data with a wide range of values. - -Like `varSamp()`, the `varSampStable()` function assumes that the input data set represents a sample from a larger population. If you want to calculate the variance of the entire population (when you have the complete data set), you should use the [`varPopStable()` function](./varpop#varpopstable) instead. - -**Example** - -Query: - -```sql -CREATE TABLE example_table -( - id UInt64, - value Float64 -) -ENGINE = MergeTree -ORDER BY id; - -INSERT INTO example_table VALUES (1, 10.5), (2, 12.3), (3, 9.8), (4, 11.2), (5, 10.7); - -SELECT varSampStable(value) FROM example_table; -``` - -Response: - -```response -0.865 -``` - -This query calculates the sample variance of the `value` column in the `example_table` using the `varSampStable()` function. The result shows that the sample variance of the values `[10.5, 12.3, 9.8, 11.2, 10.7]` is approximately 0.865, which may differ slightly from the result of `varSamp()` due to the more precise handling of floating-point arithmetic. +┌─var_samp─┐ +│ 0.865 │ +└──────────┘ +``` \ No newline at end of file diff --git a/docs/en/sql-reference/aggregate-functions/reference/varsampstable.md b/docs/en/sql-reference/aggregate-functions/reference/varsampstable.md new file mode 100644 index 00000000000..04650b50666 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/varsampstable.md @@ -0,0 +1,63 @@ +--- +title: "varSampStable" +slug: /en/sql-reference/aggregate-functions/reference/varsampstable +sidebar_position: 33 +--- + +## varSampStable + +Calculate the sample variance of a data set. Unlike [`varSamp`](../reference/varsamp.md), this function uses a numerically stable algorithm. It works slower but provides a lower computational error. + +**Syntax** + +```sql +varSampStable(x) +``` + +Alias: `VAR_SAMP_STABLE` + +**Parameters** + +- `x`: The population for which you want to calculate the sample variance. [(U)Int*](../../data-types/int-uint.md), [Float*](../../data-types/float.md), [Decimal*](../../data-types/decimal.md). + +**Returned value** + +- Returns the sample variance of the input data set. [Float64](../../data-types/float.md). + +**Implementation details** + +The `varSampStable` function calculates the sample variance using the same formula as the [`varSamp`](../reference/varsamp.md): + +$$ +\sum\frac{(x - \text{mean}(x))^2}{(n - 1)} +$$ + +Where: +- `x` is each individual data point in the data set. +- `mean(x)` is the arithmetic mean of the data set. +- `n` is the number of data points in the data set. + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test_data; +CREATE TABLE test_data +( + x Float64 +) +ENGINE = Memory; + +INSERT INTO test_data VALUES (10.5), (12.3), (9.8), (11.2), (10.7); + +SELECT round(varSampStable(x),3) AS var_samp_stable FROM test_data; +``` + +Response: + +```response +┌─var_samp_stable─┐ +│ 0.865 │ +└─────────────────┘ +``` From 5f3535309be83cebab9fae967d6b4ab739aca9dc Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 14 Jun 2024 15:38:09 +0200 Subject: [PATCH 175/318] 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 00000000000..e69de29bb2d 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 00000000000..b541090bd98 --- /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 00000000000..9ad0a772860 --- /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 00000000000..ff31f7cdbf4 --- /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 00000000000..bb24f13d9c2 --- /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 176/318] 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 bb24f13d9c2..796d7798747 100644 --- a/tests/integration/test_keeper_profiler/test.py +++ b/tests/integration/test_keeper_profiler/test.py @@ -24,6 +24,7 @@ node3 = cluster.add_instance( with_minio=True, ) + @pytest.fixture(scope="module", autouse=True) def started_cluster(): try: From 398b3cc22f9a56cfdb2c97d277626b4755418a9e Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 14 Jun 2024 15:49:39 +0200 Subject: [PATCH 177/318] Update tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh Co-authored-by: Azat Khuzhin --- tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index 5720a8a69e9..1e268fa53d3 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -35,7 +35,7 @@ function check_refcnt_for_table() ) # Notes: - # - query may sleep 0.1*(2000/4)=5 seconds maximum, it is enough to check system.parts + # - query may sleep 0.1*(2000/4)=50 seconds maximum, it is enough to check system.parts # - "part = 1" condition should prune all parts except first # - max_block_size=1 with index_granularity=1 will allow to cancel the query earlier $CLICKHOUSE_CLIENT "${args[@]}" -q "select sleepEachRow(0.1) from $table where part = 1" & From c503f9b51ec78ec8f23be3ffc8721060981b390f Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 14 Jun 2024 15:50:20 +0200 Subject: [PATCH 178/318] Update tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh --- tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index 1e268fa53d3..bd018018789 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -58,7 +58,7 @@ function check_refcnt_for_table() $CLICKHOUSE_CLIENT -q "select table, name, refcount>=6 from system.parts where database = '$CLICKHOUSE_DATABASE' and table = '$table' and refcount >= 3" # Kill the query gracefully. - kill -INT $PID ||: + kill -INT $PID wait $PID grep -F Exception "$log_file" | grep -v -F QUERY_WAS_CANCELLED rm -f "${log_file:?}" From 5a577915aba1f927124d340d0b27da39a9db5bad Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 Jun 2024 15:43:43 +0000 Subject: [PATCH 179/318] 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 c01142ff01a..6e11dc96769 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 2e7bc69402f..d0f037ad9e1 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 9625c13197c..316f398b476 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 dd107065380..d391c365ea7 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 c74a212428b..9ffbe64f85e 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 02d76f60823b71e6b1de5f7562e749ed45b25312 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 Jun 2024 17:05:03 +0000 Subject: [PATCH 180/318] 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 cae819fa982..419e944f456 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 e2fa0e43e0c..8e759648871 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 5c0751b7a35..1b4881d4e7b 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 459631261d028c30e8b13fcffa654fc764561344 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 14 Jun 2024 17:07:11 +0000 Subject: [PATCH 181/318] Update version_date.tsv and changelogs after v24.4.3.25-stable --- docs/changelogs/v24.4.3.25-stable.md | 30 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 31 insertions(+) create mode 100644 docs/changelogs/v24.4.3.25-stable.md diff --git a/docs/changelogs/v24.4.3.25-stable.md b/docs/changelogs/v24.4.3.25-stable.md new file mode 100644 index 00000000000..9582753c731 --- /dev/null +++ b/docs/changelogs/v24.4.3.25-stable.md @@ -0,0 +1,30 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v24.4.3.25-stable (a915dd4eda4) FIXME as compared to v24.4.2.141-stable (9e23d27bd11) + +#### Build/Testing/Packaging Improvement +* Backported in [#65130](https://github.com/ClickHouse/ClickHouse/issues/65130): Decrease the `unit-test` image a few times. [#65102](https://github.com/ClickHouse/ClickHouse/pull/65102) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Backported in [#64982](https://github.com/ClickHouse/ClickHouse/issues/64982): Fix the `Block structure mismatch` error for queries reading with `PREWHERE` from the materialized view when the materialized view has columns of different types than the source table. Fixes [#64611](https://github.com/ClickHouse/ClickHouse/issues/64611). [#64855](https://github.com/ClickHouse/ClickHouse/pull/64855) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#64974](https://github.com/ClickHouse/ClickHouse/issues/64974): Fix rare crash when table has TTL with subquery + database replicated + parallel replicas + analyzer. It's really rare, but please don't use TTLs with subqueries. [#64858](https://github.com/ClickHouse/ClickHouse/pull/64858) ([alesapin](https://github.com/alesapin)). +* Backported in [#65072](https://github.com/ClickHouse/ClickHouse/issues/65072): Fix `ALTER MODIFY COMMENT` query that was broken for parameterized VIEWs in https://github.com/ClickHouse/ClickHouse/pull/54211. [#65031](https://github.com/ClickHouse/ClickHouse/pull/65031) ([Nikolay Degterinsky](https://github.com/evillique)). +* Backported in [#65177](https://github.com/ClickHouse/ClickHouse/issues/65177): Fix the `Unknown expression identifier` error for remote queries with `INTERPOLATE (alias)` (new analyzer). Fixes [#64636](https://github.com/ClickHouse/ClickHouse/issues/64636). [#65090](https://github.com/ClickHouse/ClickHouse/pull/65090) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Backported in [#65263](https://github.com/ClickHouse/ClickHouse/issues/65263): Fix the bug in Hashed and Hashed_Array dictionary short circuit evaluation, which may read uninitialized number, leading to various errors. [#65256](https://github.com/ClickHouse/ClickHouse/pull/65256) ([jsc0218](https://github.com/jsc0218)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) + +* Backported in [#65285](https://github.com/ClickHouse/ClickHouse/issues/65285): Fix crash with UniqInjectiveFunctionsEliminationPass and uniqCombined. [#65188](https://github.com/ClickHouse/ClickHouse/pull/65188) ([Raúl Marín](https://github.com/Algunenano)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#65114](https://github.com/ClickHouse/ClickHouse/issues/65114): Adjust the `version_helper` and script to a new release scheme. [#64759](https://github.com/ClickHouse/ClickHouse/pull/64759) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#65225](https://github.com/ClickHouse/ClickHouse/issues/65225): Capture weak_ptr of ContextAccess for safety. [#65051](https://github.com/ClickHouse/ClickHouse/pull/65051) ([Alexander Gololobov](https://github.com/davenger)). +* Backported in [#65217](https://github.com/ClickHouse/ClickHouse/issues/65217): Fix false positives leaky memory warnings in OpenSSL. [#65125](https://github.com/ClickHouse/ClickHouse/pull/65125) ([Robert Schulze](https://github.com/rschu1ze)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 719c25bdc95..1bff4cb0b09 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -41,6 +41,7 @@ v23.9.4.11-stable 2023-11-08 v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 +v23.8.15.35-lts 2024-06-14 v23.8.14.6-lts 2024-05-02 v23.8.13.25-lts 2024-04-26 v23.8.12.13-lts 2024-03-26 From f4b01dd132156bb48a267c2122eaf9960237be12 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 15 Jun 2024 08:42:07 +0200 Subject: [PATCH 182/318] Update aspell-dict.txt --- 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 64ff3e8e2cb..2191d877d75 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2847,7 +2847,9 @@ variantElement variantType varint varpop +varpopstable varsamp +varsampstable vectorized vectorscan vendoring 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 183/318] 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 ceda264b5a6..9a267bfa149 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 0bdd4c089f1..081e7e4fc2c 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 9363e3d83eb..57d4e18010b 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 7f49ea5479d..044b0c9b1ce 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 379885fb1ab..94607ffa924 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 297acc4ef26..bc365523be1 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 38f01bd831aa36efb76b43bf2f53357cd10ef92b Mon Sep 17 00:00:00 2001 From: chloro <13125187405@163.com> Date: Sun, 16 Jun 2024 14:31:49 +0800 Subject: [PATCH 184/318] 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 9a267bfa149..71297169b5c 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 081e7e4fc2c..83142718073 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 57d4e18010b..184c263dbdb 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 044b0c9b1ce..cb7903f587a 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 185/318] 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 796d7798747..c0a70829107 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 186/318] 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 187/318] 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 c0a70829107..01c3680f89d 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 188/318] 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 71297169b5c..9d898aea6ae 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 83142718073..e5a433b4bcd 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 cb7903f587a..4aa09f8619e 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 189/318] 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 a6437019bd3..4f0a1d5dddf 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 190/318] 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 4f0a1d5dddf..f0f8360de90 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 191/318] 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 207ddadb8bf..0d18ee5439e 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 @@ public: 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 @@ public: { 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 df9664060b3..a917722c92c 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 192/318] 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 bea1a627129..440d0c73803 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 a917722c92c..fab5dac8f87 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 b266679b06a..75a81c5dbd3 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 7bed33012db0047db31e302c42193887138281f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jun 2024 07:51:58 +0200 Subject: [PATCH 193/318] 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 d8b52ecf5e4..a5eacbe1b6e 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 ec289a5ec0a..bc1b01f47d0 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 396562189e0..9dd686290db 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 14f6f806acd..fc41c78e666 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -102,8 +102,7 @@ public: 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 adb94cae0c2..dd6af8b2a19 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 bea1a627129..fa60ce77835 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 016a680d309d6bace02712325f0fa765926444c5 Mon Sep 17 00:00:00 2001 From: Artem Mustafin Date: Mon, 17 Jun 2024 12:07:03 +0300 Subject: [PATCH 194/318] reload-ci From b845a242cc1f616eca3919a9d509afabc63b4f7f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 17 Jun 2024 11:06:22 +0000 Subject: [PATCH 195/318] 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 3de39ae6bfa..28e83306c53 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 196/318] 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 01c3680f89d..f9a90b9033e 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 197/318] 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 f9a90b9033e..28dae69bd5f 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 e3818a97944bb10d56646b4145696a60a7aa0edd Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 11 Jun 2024 15:55:42 +0000 Subject: [PATCH 198/318] 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 7f6cc9abcb2..d404aed4d13 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 7954e439977..c1874a07977 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 1aced61cb42..e838ab87c1d 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 199/318] 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 c1874a07977..f75577e6ddf 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 e838ab87c1d..af7f348a6a2 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 200/318] 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 4cf61460252..3805f2a95e9 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 af7f348a6a2..6b3bcc58868 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 201/318] 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 67c0b63e578..e0d6ae2bf93 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 202/318] 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 74d81548e98..59dd92f0fcd 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 064faa228ae..182fe67f135 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 203/318] 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 5048ef4788e..1e013ff9a0c 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 419e944f456..0e75282f499 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 204/318] 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 28dae69bd5f..f5095221286 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 @@ select cnt from ( '\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 205/318] 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 89de76d2815..9e56d740e5e 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 7e3b7fcb6bb..46d2d60e461 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 7c6af41c04d..c762abc1149 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 d0f037ad9e1..ff65475d9af 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 206/318] 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 8e072779b53..38ffd40b6cd 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 00000000000..c2f595d8c4b --- /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 00000000000..59f0153cd36 --- /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 207/318] reload-ci From 8d072ade18b61b6eafef0899f5ce551030b6662d Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 17 Jun 2024 22:35:49 +0200 Subject: [PATCH 208/318] 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 f5095221286..98738890ad8 100644 --- a/tests/integration/test_keeper_profiler/test.py +++ b/tests/integration/test_keeper_profiler/test.py @@ -69,7 +69,7 @@ from system.trace_log where trace_type = ‘Real’ and trace ilike '%KeeperTCPH """ 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 209/318] 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 f94621ba092..4434a5338a7 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 210/318] 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 9d898aea6ae..0ecb3545225 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 e5a433b4bcd..3f4afff56e8 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 f2a162a4a784af00fc788084eab8bf6763f06f73 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 18 Jun 2024 11:09:13 +0800 Subject: [PATCH 211/318] 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 f2080ec4738..aa8030b077c 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 212/318] 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 c33f97dc80d..9230cba2a29 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 213/318] 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 f6ca69813ae..bbb596d0a50 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 214/318] 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 3213b8cd387..f74459489a5 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 d7970bd2eb1..466ff9002e9 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 215/318] 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 c535b82d710..a258456345e 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 6cb23bbea9f..48f4aaf4e09 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 e8cce2017d9..71b15bc8753 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 fddbf41f0e5..482996e1448 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 64ff3e8e2cb..c8fc6754502 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 216/318] 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 59f0153cd36..d8cb2bad2ea 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 a2f3bdbe83e59d3bbb11fb994cd6905994b46385 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 17 Jun 2024 10:09:21 +0000 Subject: [PATCH 217/318] Remove unused headers --- src/Common/SystemLogBase.h | 2 -- src/Interpreters/AsynchronousMetricLog.h | 2 -- 2 files changed, 4 deletions(-) diff --git a/src/Common/SystemLogBase.h b/src/Common/SystemLogBase.h index 95906c63349..9441578c9bc 100644 --- a/src/Common/SystemLogBase.h +++ b/src/Common/SystemLogBase.h @@ -1,9 +1,7 @@ #pragma once -#include #include #include -#include #include #include diff --git a/src/Interpreters/AsynchronousMetricLog.h b/src/Interpreters/AsynchronousMetricLog.h index 739b2aa5b56..2ce1d929592 100644 --- a/src/Interpreters/AsynchronousMetricLog.h +++ b/src/Interpreters/AsynchronousMetricLog.h @@ -8,8 +8,6 @@ #include #include -#include -#include #include From 0ae0ca1a95756e9d784b5a8e4b2d5d93a0e1a0cb Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Mon, 17 Jun 2024 15:02:50 +0000 Subject: [PATCH 218/318] Add system.error_log that flushes periodically system.errors to disk --- programs/server/config.xml | 12 ++++ src/Common/ErrorCodes.cpp | 1 - src/Common/ErrorCodes.h | 3 +- src/Common/SystemLogBase.cpp | 1 + src/Common/SystemLogBase.h | 3 +- src/Interpreters/ErrorLog.cpp | 121 +++++++++++++++++++++++++++++++++ src/Interpreters/ErrorLog.h | 60 ++++++++++++++++ src/Interpreters/SystemLog.cpp | 12 ++++ src/Interpreters/SystemLog.h | 2 + 9 files changed, 212 insertions(+), 3 deletions(-) create mode 100644 src/Interpreters/ErrorLog.cpp create mode 100644 src/Interpreters/ErrorLog.h diff --git a/programs/server/config.xml b/programs/server/config.xml index b7a4b8dd0e9..e3228813482 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -1155,6 +1155,18 @@ false + + + system + error_log
+ 7500 + 1048576 + 8192 + 524288 + 1000 + false +
+ + system error_log
diff --git a/programs/server/config.yaml.example b/programs/server/config.yaml.example index 9fc188e97aa..acc21fff5bb 100644 --- a/programs/server/config.yaml.example +++ b/programs/server/config.yaml.example @@ -726,6 +726,13 @@ metric_log: flush_interval_milliseconds: 7500 collect_interval_milliseconds: 1000 +# Error log contains rows with current values of errors collected with "collect_interval_milliseconds" interval. +error_log: + database: system + table: error_log + flush_interval_milliseconds: 7500 + collect_interval_milliseconds: 1000 + # Asynchronous metric log contains values of metrics from # system.asynchronous_metrics. asynchronous_metric_log: diff --git a/tests/integration/test_MemoryTracking/configs/no_system_log.xml b/tests/integration/test_MemoryTracking/configs/no_system_log.xml index 3218dae4dc7..7d80c7fbf78 100644 --- a/tests/integration/test_MemoryTracking/configs/no_system_log.xml +++ b/tests/integration/test_MemoryTracking/configs/no_system_log.xml @@ -5,6 +5,7 @@ + diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index 8363d42f9af..d8662fad011 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -1485,6 +1485,7 @@ def test_backup_all(exclude_system_log_tables): "processors_profile_log", "asynchronous_insert_log", "backup_log", + "error_log", ] exclude_from_backup += ["system." + table_name for table_name in log_tables] diff --git a/tests/integration/test_config_xml_full/configs/config.d/error_log.xml b/tests/integration/test_config_xml_full/configs/config.d/error_log.xml new file mode 100644 index 00000000000..903d8699f5c --- /dev/null +++ b/tests/integration/test_config_xml_full/configs/config.d/error_log.xml @@ -0,0 +1,8 @@ + + + system + error_log
+ 7500 + 1000 +
+
diff --git a/tests/integration/test_config_xml_full/configs/config.xml b/tests/integration/test_config_xml_full/configs/config.xml index 628e1432350..61aa0a5c724 100644 --- a/tests/integration/test_config_xml_full/configs/config.xml +++ b/tests/integration/test_config_xml_full/configs/config.xml @@ -756,6 +756,14 @@ 1000
+ + + system + error_log
+ 7500 + 1000 +
+ 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 70b71da8fa5..b19d246e1d0 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 135a4c91c56..4e34e6b6135 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 60c6a60af1a..bef43083a35 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -545,7 +545,7 @@ class CI: 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 @@ class CI: 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 fdd5dc7a671..94555158811 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 a36fcf953ae..7b2dd12c310 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 @@ class CiSettings: 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 7a51a65b5d5..47247b91858 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -201,6 +201,37 @@ class TestCIConfig(unittest.TestCase): 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 fc21c7dda4e..ee256f73abc 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -19,6 +19,7 @@ _TEST_BODY_1 = """ #### 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 @@ class TestCIOptions(unittest.TestCase): 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 @@ class TestCIOptions(unittest.TestCase): 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 7b47032d370fcd4ce4fa561b2c50e28c1f64df00 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 18 Jun 2024 12:00:12 +0000 Subject: [PATCH 236/318] Fix style check --- src/Interpreters/ErrorLog.cpp | 4 ++-- src/Interpreters/ErrorLog.h | 2 +- utils/check-style/check-style | 1 + 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ErrorLog.cpp b/src/Interpreters/ErrorLog.cpp index c8048caff5b..dc6b58a2322 100644 --- a/src/Interpreters/ErrorLog.cpp +++ b/src/Interpreters/ErrorLog.cpp @@ -49,7 +49,7 @@ void ErrorLog::startCollectError(size_t collect_interval_milliseconds_) } -void ErrorLog::stopCollect() +void ErrorLog::stopCollectError() { bool old_val = false; if (!is_shutdown_error_thread.compare_exchange_strong(old_val, true)) @@ -61,7 +61,7 @@ void ErrorLog::stopCollect() void ErrorLog::shutdown() { - stopCollect(); + stopCollectError(); stopFlushThread(); } diff --git a/src/Interpreters/ErrorLog.h b/src/Interpreters/ErrorLog.h index 28e9bebbdc5..f0af30cc4ff 100644 --- a/src/Interpreters/ErrorLog.h +++ b/src/Interpreters/ErrorLog.h @@ -42,7 +42,7 @@ public: void startCollectError(size_t collect_interval_milliseconds_); /// Stop background thread. Call before shutdown. - void stopCollect(); + void stopCollectError(); private: struct ValuePair { diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 7f25ca4607c..722dfbcad16 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -93,6 +93,7 @@ EXTERN_TYPES_EXCLUDES=( ErrorCodes::values ErrorCodes::values[i] ErrorCodes::getErrorCodeByName + ErrorCodes::Value ) for extern_type in ${!EXTERN_TYPES[@]}; do type_of_extern=${EXTERN_TYPES[$extern_type]} From b1d6a31af4f2a5af58ce7f265ae9e94c7af3c662 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 18 Jun 2024 12:04:16 +0000 Subject: [PATCH 237/318] 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 6901b6cb5ff..646d99326e3 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 1e013ff9a0c..849fa5dbe0b 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 0e75282f499..87d7d21874a 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 238/318] 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 8709bf955a9..c9974fd849c 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 c7472f46a85c571c570e853bb9862bd2baef565c Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 18 Jun 2024 13:51:15 +0000 Subject: [PATCH 239/318] Move ValuePair to the internal implementation Attempt #3 to fix CI style check. The good thing is that there was no need to expose ValuePair to the outside. --- src/Interpreters/ErrorLog.cpp | 6 ++++++ src/Interpreters/ErrorLog.h | 5 ----- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ErrorLog.cpp b/src/Interpreters/ErrorLog.cpp index dc6b58a2322..195d61e1bab 100644 --- a/src/Interpreters/ErrorLog.cpp +++ b/src/Interpreters/ErrorLog.cpp @@ -15,6 +15,12 @@ namespace DB { +struct ValuePair { + UInt64 local = 0; + UInt64 remote = 0; +}; + + ColumnsDescription ErrorLogElement::getColumnsDescription() { return ColumnsDescription { diff --git a/src/Interpreters/ErrorLog.h b/src/Interpreters/ErrorLog.h index f0af30cc4ff..97d7a7cefe5 100644 --- a/src/Interpreters/ErrorLog.h +++ b/src/Interpreters/ErrorLog.h @@ -45,11 +45,6 @@ public: void stopCollectError(); private: - struct ValuePair { - UInt64 local = 0; - UInt64 remote = 0; - }; - void threadFunction(); std::unique_ptr flush_thread; From ec855651f3dff3ac370d4cafcb917b9b4dbc2c84 Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Tue, 18 Jun 2024 13:54:45 +0000 Subject: [PATCH 240/318] 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 e7fee827320..bb0c717160e 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -196,7 +196,7 @@ def main(): # 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 d20239d49e126b68da1a0b0b3efd7d300bc2350a Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 18 Jun 2024 14:29:47 +0000 Subject: [PATCH 241/318] Fix for real the CI style check. This time is personal --- src/Interpreters/ErrorLog.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ErrorLog.cpp b/src/Interpreters/ErrorLog.cpp index 195d61e1bab..051db08e790 100644 --- a/src/Interpreters/ErrorLog.cpp +++ b/src/Interpreters/ErrorLog.cpp @@ -15,12 +15,6 @@ namespace DB { -struct ValuePair { - UInt64 local = 0; - UInt64 remote = 0; -}; - - ColumnsDescription ErrorLogElement::getColumnsDescription() { return ColumnsDescription { @@ -71,6 +65,12 @@ void ErrorLog::shutdown() stopFlushThread(); } +struct ValuePair +{ + UInt64 local = 0; + UInt64 remote = 0; +}; + void ErrorLog::threadFunction() { auto desired_timepoint = std::chrono::system_clock::now(); 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 242/318] 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 69aaaeb61c8..848929df086 100644 --- a/tests/integration/test_keeper_profiler/test.py +++ b/tests/integration/test_keeper_profiler/test.py @@ -59,7 +59,7 @@ select cnt from ( 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 4c38f9fb342aa014b7cce3a3d2f1704bbc59f497 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 18 Jun 2024 15:29:27 +0000 Subject: [PATCH 243/318] Improve error_log test to ensure errors are propagated --- .../03172_error_log_table_not_empty.reference | 5 ++++ .../03172_error_log_table_not_empty.sql | 26 ++++++++++++++++--- 2 files changed, 28 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03172_error_log_table_not_empty.reference b/tests/queries/0_stateless/03172_error_log_table_not_empty.reference index d00491fd7e5..a9e2f17562a 100644 --- a/tests/queries/0_stateless/03172_error_log_table_not_empty.reference +++ b/tests/queries/0_stateless/03172_error_log_table_not_empty.reference @@ -1 +1,6 @@ 1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/03172_error_log_table_not_empty.sql b/tests/queries/0_stateless/03172_error_log_table_not_empty.sql index 77d969622da..8c8d6108c0c 100644 --- a/tests/queries/0_stateless/03172_error_log_table_not_empty.sql +++ b/tests/queries/0_stateless/03172_error_log_table_not_empty.sql @@ -1,5 +1,25 @@ -select sleep(2) format Null; --More than collect_interval_milliseconds +-- Throw three random errors: 111, 222 and 333 +SELECT throwIf(true, 'test', toInt16(111)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 111 } +SELECT throwIf(true, 'test', toInt16(222)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 222 } +SELECT throwIf(true, 'test', toInt16(333)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 333 } -system flush logs; +-- Wait for more than collect_interval_milliseconds to ensure system.error_log is flushed +SELECT sleep(2) FORMAT NULL; +SYSTEM FLUSH LOGS; -select count()>0 from system.error_log +-- Check that the three random errors are propagated +SELECT sum(value) > 0 FROM system.error_log WHERE code = 111 AND event_time < now() - INTERVAL 1 MINUTE; +SELECT sum(value) > 0 FROM system.error_log WHERE code = 222 AND event_time < now() - INTERVAL 1 MINUTE; +SELECT sum(value) > 0 FROM system.error_log WHERE code = 333 AND event_time < now() - INTERVAL 1 MINUTE; + +-- Ensure that if we throw them again, they're still propagated +SELECT throwIf(true, 'test', toInt16(111)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 111 } +SELECT throwIf(true, 'test', toInt16(222)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 222 } +SELECT throwIf(true, 'test', toInt16(333)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 333 } + +SELECT sleep(2) FORMAT NULL; +SYSTEM FLUSH LOGS; + +SELECT sum(value) > 1 FROM system.error_log WHERE code = 111 AND event_time < now() - INTERVAL 1 MINUTE; +SELECT sum(value) > 1 FROM system.error_log WHERE code = 222 AND event_time < now() - INTERVAL 1 MINUTE; +SELECT sum(value) > 1 FROM system.error_log WHERE code = 333 AND event_time < now() - INTERVAL 1 MINUTE; \ No newline at end of file From ca0da7a481325157b1436d1a997078db7ab2fa77 Mon Sep 17 00:00:00 2001 From: Alex Katsman Date: Wed, 29 May 2024 16:09:03 +0000 Subject: [PATCH 244/318] 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 | 168 +++++++------ src/Access/ContextAccess.h | 228 +++++++++++++----- .../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, 377 insertions(+), 151 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 aa8ef6073d3..0d629e7b77a 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 e87c28dd823..aaeea6ceddc 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 28a825de6cf..a2807ecc5ea 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::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 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); } +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 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::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 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); } +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 237c423d261..465932af1d3 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 @@ public: /// 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 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 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 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 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 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 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 @@ private: 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 @@ private: 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 6a46ac9c330..ac3b549a576 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 4b0ebc008ea..2807807b294 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 f3073ccc09c..b3ade94ccdc 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 @@ public: 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 @@ public: 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 @@ public: 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 92f8ddae141..e63a2ae31aa 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 5072d220160..bbaa7179757 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 396562189e0..d73593b7f1c 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 9c5436517ab..9f33cbf1c27 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 49da1eba9ec..8dd8d3b6154 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -298,7 +298,7 @@ private: 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 00000000000..e3ac97f9945 --- /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 00000000000..9abc635a874 --- /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 245/318] 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 75b855966a3..47094eec3f0 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 6a95f1995859562753e2d6908da56e4397fe484d Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Tue, 18 Jun 2024 15:42:52 +0000 Subject: [PATCH 246/318] Minor changes found reviewing to remove the draft status --- docs/en/operations/system-tables/error_log.md | 2 +- src/Common/ErrorCodes.cpp | 1 + src/Common/ErrorCodes.h | 3 +-- src/Interpreters/ErrorLog.h | 2 +- src/Interpreters/MetricLog.h | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/system-tables/error_log.md b/docs/en/operations/system-tables/error_log.md index bfd6a165462..15edef58662 100644 --- a/docs/en/operations/system-tables/error_log.md +++ b/docs/en/operations/system-tables/error_log.md @@ -3,7 +3,7 @@ slug: /en/operations/system-tables/error_log --- # error_log -Contains history of error values from tables `system.errors`, periodically flushed to disk. +Contains history of error values from table `system.errors`, periodically flushed to disk. Columns: - `hostname` ([LowCardinality(String)](../../sql-reference/data-types/string.md)) — Hostname of the server executing the query. diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 538052ba6e6..29993ed33e4 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -626,6 +626,7 @@ namespace ErrorCodes APPLY_FOR_ERROR_CODES(M) #undef M + constexpr ErrorCode END = 1002; ErrorPairHolder values[END + 1]{}; struct ErrorCodesNames diff --git a/src/Common/ErrorCodes.h b/src/Common/ErrorCodes.h index 1480b675e17..11a163becbe 100644 --- a/src/Common/ErrorCodes.h +++ b/src/Common/ErrorCodes.h @@ -18,7 +18,6 @@ namespace ErrorCodes { /// ErrorCode identifier (index in array). using ErrorCode = int; - constexpr ErrorCode END = 1002; using Value = size_t; using FramePointers = std::vector; @@ -34,7 +33,7 @@ namespace ErrorCodes struct Error { - /// Number of times Exception with this ErrorCode had been thrown. + /// Number of times Exception with this ErrorCode has been thrown. Value count = 0; /// Time of the last error. UInt64 error_time_ms = 0; diff --git a/src/Interpreters/ErrorLog.h b/src/Interpreters/ErrorLog.h index 97d7a7cefe5..6fb5ead559b 100644 --- a/src/Interpreters/ErrorLog.h +++ b/src/Interpreters/ErrorLog.h @@ -41,7 +41,7 @@ public: /// Launches a background thread to collect errors with interval void startCollectError(size_t collect_interval_milliseconds_); - /// Stop background thread. Call before shutdown. + /// Stop background thread void stopCollectError(); private: diff --git a/src/Interpreters/MetricLog.h b/src/Interpreters/MetricLog.h index 4f1e8fafc11..3a079792bc8 100644 --- a/src/Interpreters/MetricLog.h +++ b/src/Interpreters/MetricLog.h @@ -44,7 +44,7 @@ public: /// Launches a background thread to collect metrics with interval void startCollectMetric(size_t collect_interval_milliseconds_); - /// Stop background thread. Call before shutdown. + /// Stop background thread void stopCollectMetric(); private: From f658456d02c528d005b3f91ea8cf432f7273cc31 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 18 Jun 2024 15:55:36 +0000 Subject: [PATCH 247/318] Try to fix build --- src/Access/AccessControl.cpp | 2 +- src/Access/DiskAccessStorage.cpp | 2 +- src/Access/MultipleAccessStorage.cpp | 2 +- src/Access/ReplicatedAccessStorage.cpp | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 58ca5cdc435..353358fac65 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -265,7 +265,7 @@ AccessControl::~AccessControl() { try { - shutdown(); + AccessControl::shutdown(); } catch (...) { diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index c1633ee950f..ee422f7d8ff 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -196,7 +196,7 @@ DiskAccessStorage::~DiskAccessStorage() { try { - shutdown(); + DiskAccessStorage::shutdown(); } catch (...) { diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index 35b94de38c7..fda6601e4c6 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -36,7 +36,7 @@ MultipleAccessStorage::~MultipleAccessStorage() { try { - shutdown(); + MultipleAccessStorage::shutdown(); } catch (...) { diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index 415b757483e..ed114327041 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -69,7 +69,7 @@ ReplicatedAccessStorage::~ReplicatedAccessStorage() { try { - shutdown(); + ReplicatedAccessStorage::shutdown(); } catch (...) { From 11e844c953380bbfda3b61dcd2dfda60b809f5e2 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 18 Jun 2024 19:12:58 +0000 Subject: [PATCH 248/318] 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 3f699c881fb..81b491b0eda 100644 --- a/src/Common/Scheduler/ISchedulerNode.h +++ b/src/Common/Scheduler/ISchedulerNode.h @@ -406,13 +406,19 @@ private: 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 249/318] 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 88bc50a729d..384bf6825f9 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 250/318] 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 ac9a72c2641..250e766f2b7 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 251/318] 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 23447760b47..31798e8dd7e 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 9229342b8c1..55441cfb86b 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 252/318] 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 e0d6ae2bf93..277de2ba202 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 253/318] 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 0958680dc56..97ca79501a7 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 254/318] 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 70b71da8fa5..c9e7ae4450d 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 03bda34c4372ef10d9306abbdf5abe172f389303 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Wed, 19 Jun 2024 10:54:52 +0000 Subject: [PATCH 255/318] Fix test. Classic mistake :') --- .../03172_error_log_table_not_empty.sql | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/03172_error_log_table_not_empty.sql b/tests/queries/0_stateless/03172_error_log_table_not_empty.sql index 8c8d6108c0c..71814edc5d8 100644 --- a/tests/queries/0_stateless/03172_error_log_table_not_empty.sql +++ b/tests/queries/0_stateless/03172_error_log_table_not_empty.sql @@ -1,25 +1,25 @@ -- Throw three random errors: 111, 222 and 333 -SELECT throwIf(true, 'test', toInt16(111)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 111 } -SELECT throwIf(true, 'test', toInt16(222)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 222 } -SELECT throwIf(true, 'test', toInt16(333)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 333 } +SELECT throwIf(true, 'error_log', toInt16(111)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 111 } +SELECT throwIf(true, 'error_log', toInt16(222)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 222 } +SELECT throwIf(true, 'error_log', toInt16(333)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 333 } -- Wait for more than collect_interval_milliseconds to ensure system.error_log is flushed SELECT sleep(2) FORMAT NULL; SYSTEM FLUSH LOGS; -- Check that the three random errors are propagated -SELECT sum(value) > 0 FROM system.error_log WHERE code = 111 AND event_time < now() - INTERVAL 1 MINUTE; -SELECT sum(value) > 0 FROM system.error_log WHERE code = 222 AND event_time < now() - INTERVAL 1 MINUTE; -SELECT sum(value) > 0 FROM system.error_log WHERE code = 333 AND event_time < now() - INTERVAL 1 MINUTE; +SELECT sum(value) > 0 FROM system.error_log WHERE code = 111 AND event_time > now() - INTERVAL 1 MINUTE; +SELECT sum(value) > 0 FROM system.error_log WHERE code = 222 AND event_time > now() - INTERVAL 1 MINUTE; +SELECT sum(value) > 0 FROM system.error_log WHERE code = 333 AND event_time > now() - INTERVAL 1 MINUTE; -- Ensure that if we throw them again, they're still propagated -SELECT throwIf(true, 'test', toInt16(111)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 111 } -SELECT throwIf(true, 'test', toInt16(222)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 222 } -SELECT throwIf(true, 'test', toInt16(333)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 333 } +SELECT throwIf(true, 'error_log', toInt16(111)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 111 } +SELECT throwIf(true, 'error_log', toInt16(222)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 222 } +SELECT throwIf(true, 'error_log', toInt16(333)) SETTINGS allow_custom_error_code_in_throwif=1; -- { serverError 333 } SELECT sleep(2) FORMAT NULL; SYSTEM FLUSH LOGS; -SELECT sum(value) > 1 FROM system.error_log WHERE code = 111 AND event_time < now() - INTERVAL 1 MINUTE; -SELECT sum(value) > 1 FROM system.error_log WHERE code = 222 AND event_time < now() - INTERVAL 1 MINUTE; -SELECT sum(value) > 1 FROM system.error_log WHERE code = 333 AND event_time < now() - INTERVAL 1 MINUTE; \ No newline at end of file +SELECT sum(value) > 1 FROM system.error_log WHERE code = 111 AND event_time > now() - INTERVAL 1 MINUTE; +SELECT sum(value) > 1 FROM system.error_log WHERE code = 222 AND event_time > now() - INTERVAL 1 MINUTE; +SELECT sum(value) > 1 FROM system.error_log WHERE code = 333 AND event_time > now() - INTERVAL 1 MINUTE; \ No newline at end of file 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 256/318] 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 66631c51b03..276df7ff40a 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -877,7 +877,7 @@ def test_max_set_age(started_cluster): 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 257/318] 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 94555158811..414331829c9 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 d06867e886952f36a2691ac8144e9946ba0c193f Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Wed, 19 Jun 2024 17:21:25 +0200 Subject: [PATCH 258/318] Catch exception in tryLoadImpl --- src/Server/CertificateReloader.cpp | 42 ++++++++++++------------------ 1 file changed, 16 insertions(+), 26 deletions(-) diff --git a/src/Server/CertificateReloader.cpp b/src/Server/CertificateReloader.cpp index 1488c747fee..df7b6e7fbd7 100644 --- a/src/Server/CertificateReloader.cpp +++ b/src/Server/CertificateReloader.cpp @@ -103,16 +103,7 @@ std::list::iterator CertificateReloader::findOrI else { if (!ctx) - { - try - { - ctx = Poco::Net::SSLManager::instance().defaultServerContext()->sslContext(); - } - catch (...) - { - LOG_ERROR(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false)); - } - } + ctx = Poco::Net::SSLManager::instance().defaultServerContext()->sslContext(); data.push_back(MultiData(ctx)); --it; data_index[prefix] = it; @@ -137,28 +128,27 @@ void CertificateReloader::tryLoadImpl(const Poco::Util::AbstractConfiguration & } else { - auto it = findOrInsert(ctx, prefix); - - bool cert_file_changed = it->cert_file.changeIfModified(std::move(new_cert_path), log); - bool key_file_changed = it->key_file.changeIfModified(std::move(new_key_path), log); - - if (cert_file_changed || key_file_changed) - { - LOG_DEBUG(log, "Reloading certificate ({}) and key ({}).", it->cert_file.path, it->key_file.path); - std::string pass_phrase = config.getString(prefix + "privateKeyPassphraseHandler.options.password", ""); - it->data.set(std::make_unique(it->cert_file.path, it->key_file.path, pass_phrase)); - LOG_INFO(log, "Reloaded certificate ({}) and key ({}).", it->cert_file.path, it->key_file.path); - } - - /// If callback is not set yet try { - if (it->init_was_not_made && it->ctx != nullptr) + auto it = findOrInsert(ctx, prefix); + + bool cert_file_changed = it->cert_file.changeIfModified(std::move(new_cert_path), log); + bool key_file_changed = it->key_file.changeIfModified(std::move(new_key_path), log); + + if (cert_file_changed || key_file_changed) + { + LOG_DEBUG(log, "Reloading certificate ({}) and key ({}).", it->cert_file.path, it->key_file.path); + std::string pass_phrase = config.getString(prefix + "privateKeyPassphraseHandler.options.password", ""); + it->data.set(std::make_unique(it->cert_file.path, it->key_file.path, pass_phrase)); + LOG_INFO(log, "Reloaded certificate ({}) and key ({}).", it->cert_file.path, it->key_file.path); + } + + /// If callback is not set yet + if (it->init_was_not_made) init(&*it); } catch (...) { - it->init_was_not_made = true; LOG_ERROR(log, getCurrentExceptionMessageAndPattern(/* with_stacktrace */ false)); } } From 84d282c5738414b18b6f5a7cc269c018b75f3143 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 19 Jun 2024 17:29:24 +0200 Subject: [PATCH 259/318] 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 414331829c9..d60c67d318d 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 260/318] 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 ce5ab24ecb0..42820505406 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 261/318] 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 a0047a2925a..89aeb5b22e5 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 69f16a003a73f15bf0702d1b523b0d4d69c29bcb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 19 Jun 2024 16:53:17 +0000 Subject: [PATCH 262/318] Fixups --- src/Common/ProfileEvents.cpp | 2 +- ...t_queries_with_primary_key_usage.reference | 24 ----- ..._select_queries_with_primary_key_usage.sql | 35 ------ ...ects_with_pk_usage_profile_event.reference | 8 ++ ...164_selects_with_pk_usage_profile_event.sh | 101 ++++++++++++++++++ 5 files changed, 110 insertions(+), 60 deletions(-) delete mode 100644 tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.reference delete mode 100644 tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.sql create mode 100644 tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.reference create mode 100755 tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.sh diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index ddc5753e816..c0f535cc5d6 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -14,7 +14,7 @@ M(QueriesWithSubqueries, "Count queries with all subqueries") \ M(SelectQueriesWithSubqueries, "Count SELECT queries with all subqueries") \ M(InsertQueriesWithSubqueries, "Count INSERT queries with all subqueries") \ - M(SelectQueriesWithPrimaryKeyUsage, "Count SELECT queries with primary key as a WHERE condition.") \ + M(SelectQueriesWithPrimaryKeyUsage, "Count SELECT queries which use the primary key to evaluate the WHERE condition") \ M(AsyncInsertQuery, "Same as InsertQuery, but only for asynchronous INSERT queries.") \ M(AsyncInsertBytes, "Data size in bytes of asynchronous INSERT queries.") \ M(AsyncInsertRows, "Number of rows inserted by asynchronous INSERT queries.") \ diff --git a/tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.reference b/tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.reference deleted file mode 100644 index d5563921583..00000000000 --- a/tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.reference +++ /dev/null @@ -1,24 +0,0 @@ -1 -100 -100 -100 -100 -Row 1: -────── -queries_with_primary_key_usage: 1 - -Row 2: -────── -queries_with_primary_key_usage: 1 - -Row 3: -────── -queries_with_primary_key_usage: 1 - -Row 4: -────── -queries_with_primary_key_usage: 0 - -Row 5: -────── -queries_with_primary_key_usage: 0 diff --git a/tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.sql b/tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.sql deleted file mode 100644 index 6b7aa6fb3ee..00000000000 --- a/tests/queries/0_stateless/03164_select_queries_with_primary_key_usage.sql +++ /dev/null @@ -1,35 +0,0 @@ --- Tests for ProfileEvents "SelectedMarksByPrimaryKeyUsage" -set log_queries=1; -set log_queries_min_type='QUERY_FINISH'; - -DROP TABLE IF EXISTS 03164_select_queries_with_primary_key_usage; -CREATE TABLE 03164_select_queries_with_primary_key_usage -( - a Int64, b Int64, c Int64 -) ENGINE = MergeTree ORDER BY a; - -ALTER TABLE 03164_select_queries_with_primary_key_usage ADD PROJECTION b_projection ( - SELECT * ORDER BY b -); - -INSERT INTO 03164_select_queries_with_primary_key_usage SELECT number, number + 100, number + 1000 FROM numbers(100); - -SELECT count(*) FROM 03164_select_queries_with_primary_key_usage WHERE a >= 0 and b <= 100; -SELECT count(*) FROM 03164_select_queries_with_primary_key_usage WHERE a >= 0; -SELECT count(*) FROM 03164_select_queries_with_primary_key_usage WHERE b >= 100; -SELECT count(*) FROM 03164_select_queries_with_primary_key_usage WHERE c >= 1000; -SELECT count(*) FROM 03164_select_queries_with_primary_key_usage; - -SYSTEM FLUSH LOGS; - -SELECT - IF (ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] > 0, 1, 0) AS queries_with_primary_key_usage -FROM - system.query_log -WHERE - current_database = currentDatabase() - AND query LIKE 'SELECT count(*) FROM 03164_select_queries_with_primary_key_usage%' -ORDER BY query -FORMAT Vertical; - -DROP TABLE IF EXISTS 03164_select_queries_with_primary_key_usage; diff --git a/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.reference b/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.reference new file mode 100644 index 00000000000..1526555f6c8 --- /dev/null +++ b/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.reference @@ -0,0 +1,8 @@ +selects_with_pk_usage +0 +selects_with_pk_usage +0 +selects_with_pk_usage +1 +selects_with_pk_usage +1 diff --git a/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.sh b/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.sh new file mode 100755 index 00000000000..29d4c877909 --- /dev/null +++ b/tests/queries/0_stateless/03164_selects_with_pk_usage_profile_event.sh @@ -0,0 +1,101 @@ +#!/usr/bin/env bash + +# Tests profile event "SelectedMarksByPrimaryKeyUsage" + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +table_id="$(random_str 10)" + +$CLICKHOUSE_CLIENT -q " + DROP TABLE IF EXISTS table_$table_id;" + +$CLICKHOUSE_CLIENT -q " + CREATE TABLE table_$table_id ( + pk Int64, + col1 Int64, + col2 Int64, + INDEX idx(col2) TYPE minmax + ) ENGINE = MergeTree ORDER BY pk PARTITION BY (pk % 2);"; + +$CLICKHOUSE_CLIENT -q " + ALTER TABLE table_$table_id ADD PROJECTION proj (SELECT * ORDER BY col1);" + +# Populate two partitions with 50k rows each. Each partition has >1 granules. +# We want SelectQueriesWithPrimaryKeyUsage to increase by +1 in each query, not by +1 per partition or by +1 per granule. +$CLICKHOUSE_CLIENT -q " + INSERT INTO table_$table_id SELECT number, number, number FROM numbers(100000);" + +# Run SELECTs + +# -- No filter +query_id="$(random_str 10)" +$CLICKHOUSE_CLIENT --query_id "$query_id" -q " + SELECT count(*) FROM table_$table_id FORMAT Null;" +$CLICKHOUSE_CLIENT -mn -q " + SYSTEM FLUSH LOGS; + SELECT + ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] AS selects_with_pk_usage + FROM + system.query_log + WHERE + current_database = currentDatabase() + AND type = 'QueryFinish' + AND query_id = '$query_id' + FORMAT TSVWithNames; +" + +# -- Filter on non-PK column. However, it has a minmax-index defined. We expect the profile event to not increase. +query_id="$(random_str 10)" +$CLICKHOUSE_CLIENT --query_id "$query_id" -q " + SELECT count(*) FROM table_$table_id WHERE col2 >= 50000 FORMAT Null;" +$CLICKHOUSE_CLIENT -mn -q " + SYSTEM FLUSH LOGS; + SELECT + ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] AS selects_with_pk_usage + FROM + system.query_log + WHERE + current_database = currentDatabase() + AND type = 'QueryFinish' + AND query_id = '$query_id' + FORMAT TSVWithNames; +" + +# Filter on PK +query_id="$(random_str 10)" +$CLICKHOUSE_CLIENT --query_id "$query_id" -q " + SELECT count(*) FROM table_$table_id WHERE pk >= 50000 FORMAT Null;" +$CLICKHOUSE_CLIENT -mn -q " + SYSTEM FLUSH LOGS; + SELECT + ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] AS selects_with_pk_usage + FROM + system.query_log + WHERE + current_database = currentDatabase() + AND type = 'QueryFinish' + AND query_id = '$query_id' + FORMAT TSVWithNames; +" + +# Filter on PK in projection +query_id="$(random_str 10)" +$CLICKHOUSE_CLIENT --query_id "$query_id" -q " + SELECT count(*) FROM table_$table_id WHERE col1 >= 50000 FORMAT Null;" +$CLICKHOUSE_CLIENT -mn -q " + SYSTEM FLUSH LOGS; + SELECT + ProfileEvents['SelectQueriesWithPrimaryKeyUsage'] AS selects_with_pk_usage + FROM + system.query_log + WHERE + current_database = currentDatabase() + AND type = 'QueryFinish' + AND query_id = '$query_id' + FORMAT TSVWithNames; +" + +$CLICKHOUSE_CLIENT -q " + DROP TABLE table_$table_id;" From adba703ddc2ba9d54bf3c22cda303b2e5dd54f5b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 19 Jun 2024 23:13:20 +0000 Subject: [PATCH 263/318] fix AzureObjectStorage::exists --- .../AzureBlobStorage/AzureObjectStorage.cpp | 31 +++++++++---------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index e7ecf7cd515..86a035f3be7 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -127,25 +127,22 @@ bool AzureObjectStorage::exists(const StoredObject & object) const { auto client_ptr = client.get(); - /// What a shame, no Exists method... - Azure::Storage::Blobs::ListBlobsOptions options; - options.Prefix = object.remote_path; - options.PageSizeHint = 1; - - ProfileEvents::increment(ProfileEvents::AzureListObjects); + ProfileEvents::increment(ProfileEvents::AzureGetProperties); if (client_ptr->GetClickhouseOptions().IsClientForDisk) - ProfileEvents::increment(ProfileEvents::DiskAzureListObjects); + ProfileEvents::increment(ProfileEvents::DiskAzureGetProperties); - auto blobs_list_response = client_ptr->ListBlobs(options); - auto blobs_list = blobs_list_response.Blobs; - - for (const auto & blob : blobs_list) + try { - if (object.remote_path == blob.Name) - return true; + auto blob_client = client_ptr->GetBlobClient(object.remote_path); + blob_client.GetProperties(); + return true; + } + catch (const Azure::Storage::StorageException & e) + { + if (e.StatusCode == Azure::Core::Http::HttpStatusCode::NotFound) + return false; + throw; } - - return false; } ObjectStorageIteratorPtr AzureObjectStorage::iterate(const std::string & path_prefix, size_t max_keys) const @@ -160,7 +157,9 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith { auto client_ptr = client.get(); - /// What a shame, no Exists method... + /// NOTE: list doesn't work if endpoint contains non-empty prefix for blobs. + /// See AzureBlobStorageEndpoint and processAzureBlobStorageEndpoint for details. + Azure::Storage::Blobs::ListBlobsOptions options; options.Prefix = path; if (max_keys) 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 264/318] 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 072252de8c9..00000000000 --- 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 a2a2fcd967a..3967179f078 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 265/318] 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 7541d64af4a..00000000000 --- 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 4ac25e14ee6..58e657d3723 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 75b347be0c4..00000000000 --- 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 c14fde8943a..00000000000 --- 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 0b175336a6ad63454002c250b0855ca55a6a937e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 20 Jun 2024 09:37:53 +0200 Subject: [PATCH 266/318] Fix user specific auth with incremental backups --- src/Backups/BackupImpl.cpp | 6 +- src/Backups/BackupImpl.h | 1 + .../registerBackupEngineAzureBlobStorage.cpp | 1 + src/Backups/registerBackupEngineS3.cpp | 1 + .../registerBackupEnginesFileAndDisk.cpp | 1 + .../test_backup_restore_s3/test.py | 111 ++++++++++++++---- 6 files changed, 92 insertions(+), 29 deletions(-) diff --git a/src/Backups/BackupImpl.cpp b/src/Backups/BackupImpl.cpp index 8f32c918c61..3f972c36e47 100644 --- a/src/Backups/BackupImpl.cpp +++ b/src/Backups/BackupImpl.cpp @@ -24,8 +24,6 @@ #include #include -#include - namespace ProfileEvents { @@ -93,6 +91,7 @@ BackupImpl::BackupImpl( const std::optional & base_backup_info_, std::shared_ptr reader_, const ContextPtr & context_, + bool is_internal_backup_, bool use_same_s3_credentials_for_base_backup_) : backup_info(backup_info_) , backup_name_for_logging(backup_info.toStringForLogging()) @@ -101,7 +100,7 @@ BackupImpl::BackupImpl( , open_mode(OpenMode::READ) , reader(std::move(reader_)) , context(context_) - , is_internal_backup(false) + , is_internal_backup(is_internal_backup_) , version(INITIAL_BACKUP_VERSION) , base_backup_info(base_backup_info_) , use_same_s3_credentials_for_base_backup(use_same_s3_credentials_for_base_backup_) @@ -256,6 +255,7 @@ std::shared_ptr BackupImpl::getBaseBackupUnlocked() const params.backup_info = *base_backup_info; params.open_mode = OpenMode::READ; params.context = context; + params.is_internal_backup = is_internal_backup; /// use_same_s3_credentials_for_base_backup should be inherited for base backups params.use_same_s3_credentials_for_base_backup = use_same_s3_credentials_for_base_backup; diff --git a/src/Backups/BackupImpl.h b/src/Backups/BackupImpl.h index 6fed5fe758b..2b27e2ab090 100644 --- a/src/Backups/BackupImpl.h +++ b/src/Backups/BackupImpl.h @@ -40,6 +40,7 @@ public: const std::optional & base_backup_info_, std::shared_ptr reader_, const ContextPtr & context_, + bool is_internal_backup_, bool use_same_s3_credentials_for_base_backup_); BackupImpl( diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 81e3c104da1..03d156d1009 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -153,6 +153,7 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) params.base_backup_info, reader, params.context, + params.is_internal_backup, /* use_same_s3_credentials_for_base_backup*/ false); } else diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index c34dbe273f5..59ed9506af0 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -119,6 +119,7 @@ void registerBackupEngineS3(BackupFactory & factory) params.base_backup_info, reader, params.context, + params.is_internal_backup, params.use_same_s3_credentials_for_base_backup); } else diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp index c633ebb6a5a..35263d39cba 100644 --- a/src/Backups/registerBackupEnginesFileAndDisk.cpp +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -177,6 +177,7 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory) params.base_backup_info, reader, params.context, + params.is_internal_backup, params.use_same_s3_credentials_for_base_backup); } else diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 967ed6a221c..d53335000a6 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -627,67 +627,126 @@ def test_user_specific_auth(start_cluster): create_user("superuser2") create_user("regularuser") - node.query("CREATE TABLE specific_auth (col UInt64) ENGINE=Memory") + node.query("CREATE TABLE specific_auth (col UInt64) ENGINE=MergeTree ORDER BY col") + node.query("INSERT INTO specific_auth VALUES (1)") - assert "Access" in node.query_and_get_error( - "BACKUP TABLE specific_auth TO S3('http://minio1:9001/root/data/backups/limited/backup1.zip')" + def backup_restore(backup, user, should_fail, on_cluster=False, base_backup=None): + on_cluster_clause = "ON CLUSTER 'cluster'" if on_cluster else "" + base_backup = ( + f" SETTINGS base_backup = {base_backup}" if base_backup is not None else "" + ) + backup_query = ( + f"BACKUP TABLE specific_auth {on_cluster_clause} TO {backup} {base_backup}" + ) + restore_query = f"RESTORE TABLE specific_auth {on_cluster_clause} FROM {backup}" + + if should_fail: + assert "Access" in node.query_and_get_error(backup_query, user=user) + else: + node.query(backup_query, user=user) + node.query("DROP TABLE specific_auth SYNC") + node.query(restore_query, user=user) + + backup_restore( + "S3('http://minio1:9001/root/data/backups/limited/backup1/')", + user=None, + should_fail=True, ) - assert "Access" in node.query_and_get_error( - "BACKUP TABLE specific_auth TO S3('http://minio1:9001/root/data/backups/limited/backup1.zip')", + + backup_restore( + "S3('http://minio1:9001/root/data/backups/limited/backup1/')", user="regularuser", + should_fail=True, ) - node.query( - "BACKUP TABLE specific_auth TO S3('http://minio1:9001/root/data/backups/limited/backup1.zip')", - user="superuser1", - ) - node.query( - "RESTORE TABLE specific_auth FROM S3('http://minio1:9001/root/data/backups/limited/backup1.zip')", + backup_restore( + "S3('http://minio1:9001/root/data/backups/limited/backup1/')", user="superuser1", + should_fail=False, ) - node.query( - "BACKUP TABLE specific_auth TO S3('http://minio1:9001/root/data/backups/limited/backup2.zip')", - user="superuser2", - ) - node.query( - "RESTORE TABLE specific_auth FROM S3('http://minio1:9001/root/data/backups/limited/backup2.zip')", + backup_restore( + "S3('http://minio1:9001/root/data/backups/limited/backup2/')", user="superuser2", + should_fail=False, ) assert "Access" in node.query_and_get_error( - "RESTORE TABLE specific_auth FROM S3('http://minio1:9001/root/data/backups/limited/backup1.zip')", + "RESTORE TABLE specific_auth FROM S3('http://minio1:9001/root/data/backups/limited/backup1/')", user="regularuser", ) - assert "HTTP response code: 403" in node.query_and_get_error( - "SELECT * FROM s3('http://minio1:9001/root/data/backups/limited/backup1.zip', 'RawBLOB')", + node.query("INSERT INTO specific_auth VALUES (2)") + + backup_restore( + "S3('http://minio1:9001/root/data/backups/limited/backup1_inc/')", user="regularuser", + should_fail=True, + base_backup="S3('http://minio1:9001/root/data/backups/limited/backup1/')", ) - node.query( - "SELECT * FROM s3('http://minio1:9001/root/data/backups/limited/backup1.zip', 'RawBLOB')", + backup_restore( + "S3('http://minio1:9001/root/data/backups/limited/backup1_inc/')", user="superuser1", + should_fail=False, + base_backup="S3('http://minio1:9001/root/data/backups/limited/backup1/')", + ) + + assert "Access" in node.query_and_get_error( + "RESTORE TABLE specific_auth FROM S3('http://minio1:9001/root/data/backups/limited/backup1_inc/')", + user="regularuser", ) assert "Access Denied" in node.query_and_get_error( - "BACKUP TABLE specific_auth ON CLUSTER 'cluster' TO S3('http://minio1:9001/root/data/backups/limited/backup3/')", + "SELECT * FROM s3('http://minio1:9001/root/data/backups/limited/backup1/*', 'RawBLOB')", user="regularuser", ) node.query( - "BACKUP TABLE specific_auth ON CLUSTER 'cluster' TO S3('http://minio1:9001/root/data/backups/limited/backup3/')", + "SELECT * FROM s3('http://minio1:9001/root/data/backups/limited/backup1/*', 'RawBLOB')", user="superuser1", ) + backup_restore( + "S3('http://minio1:9001/root/data/backups/limited/backup3/')", + user="regularuser", + should_fail=True, + on_cluster=True, + ) + + backup_restore( + "S3('http://minio1:9001/root/data/backups/limited/backup3/')", + user="superuser1", + should_fail=False, + on_cluster=True, + ) + assert "Access Denied" in node.query_and_get_error( "RESTORE TABLE specific_auth ON CLUSTER 'cluster' FROM S3('http://minio1:9001/root/data/backups/limited/backup3/')", user="regularuser", ) - node.query( - "RESTORE TABLE specific_auth ON CLUSTER 'cluster' FROM S3('http://minio1:9001/root/data/backups/limited/backup3/')", + node.query("INSERT INTO specific_auth VALUES (3)") + + backup_restore( + "S3('http://minio1:9001/root/data/backups/limited/backup3_inc/')", + user="regularuser", + should_fail=True, + on_cluster=True, + base_backup="S3('http://minio1:9001/root/data/backups/limited/backup3/')", + ) + + backup_restore( + "S3('http://minio1:9001/root/data/backups/limited/backup3_inc/')", user="superuser1", + should_fail=False, + on_cluster=True, + base_backup="S3('http://minio1:9001/root/data/backups/limited/backup3/')", + ) + + assert "Access Denied" in node.query_and_get_error( + "RESTORE TABLE specific_auth ON CLUSTER 'cluster' FROM S3('http://minio1:9001/root/data/backups/limited/backup3_inc/')", + user="regularuser", ) assert "Access Denied" in node.query_and_get_error( 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 267/318] 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 dfbbb66a1e9..bb776fa9506 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 b42b070d518..9201eef185f 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 209407cf73e3deded3baac08793ad6b3439054c5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 20 Jun 2024 08:39:36 +0000 Subject: [PATCH 268/318] Generate 24.6 changelog --- CHANGELOG.md | 398 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 398 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4891b79e4c7..3965014707f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v24.6, 2024-06-27](#246)**
**[ClickHouse release v24.5, 2024-05-30](#245)**
**[ClickHouse release v24.4, 2024-04-30](#244)**
**[ClickHouse release v24.3 LTS, 2024-03-26](#243)**
@@ -8,6 +9,403 @@ # 2024 Changelog +### ClickHouse release 24.6, 2024-06-27 + +#### Backward Incompatible Change +* Some invalid queries will fail earlier during parsing. Note: disabled the support for inline KQL expressions (the experimental Kusto language) when they are put into a `kql` table function without a string literal, e.g. `kql(garbage | trash)` instead of `kql('garbage | trash')` or `kql($$garbage | trash$$)`. This feature was introduced unintentionally and should not exist. [#61500](https://github.com/ClickHouse/ClickHouse/pull/61500) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Rework parallel processing in `Ordered` mode of storage `S3Queue`. This PR is backward incompatible for Ordered mode if you used settings `s3queue_processing_threads_num` or `s3queue_total_shards_num`. Setting `s3queue_total_shards_num` is deleted, previously it was allowed to use only under `s3queue_allow_experimental_sharded_mode`, which is not deprecated. A new setting is added - `s3queue_buckets`. [#64349](https://github.com/ClickHouse/ClickHouse/pull/64349) ([Kseniia Sumarokova](https://github.com/kssenii)). +* New functions `snowflakeIDToDateTime`, `snowflakeIDToDateTime64`, `dateTimeToSnowflakeID`, and `dateTime64ToSnowflakeID` were added. Unlike the existing functions `snowflakeToDateTime`, `snowflakeToDateTime64`, `dateTimeToSnowflake`, and `dateTime64ToSnowflake`, the new functions are compatible with function `generateSnowflakeID`, i.e. they accept the snowflake IDs generated by `generateSnowflakeID` and produce snowflake IDs of the same type as `generateSnowflakeID` (i.e. `UInt64`). Furthermore, the new functions default to the UNIX epoch (aka. 1970-01-01), just like `generateSnowflakeID`. If necessary, a different epoch, e.g. Twitter's/X's epoch 2010-11-04 aka. 1288834974657 msec since UNIX epoch, can be passed. The old conversion functions are deprecated and will be removed after a transition period: to use them regardless, disable setting `uniform_snowflake_conversion_functions` (default: `true`). [#64948](https://github.com/ClickHouse/ClickHouse/pull/64948) ([Robert Schulze](https://github.com/rschu1ze)). + +#### New Feature +* Introduce statistics of type "number of distinct values". [#59357](https://github.com/ClickHouse/ClickHouse/pull/59357) ([Han Fei](https://github.com/hanfei1991)). +* Adds Hilbert Curve encode and decode functions - https://en.wikipedia.org/wiki/Hilbert_curve. Closes [#55520](https://github.com/ClickHouse/ClickHouse/issues/55520). [#60156](https://github.com/ClickHouse/ClickHouse/pull/60156) ([Artem Mustafin](https://github.com/Artemmm91)). +* Added support for reading LINESTRING geometry in WKT format using function `readWKTLineString`. [#62519](https://github.com/ClickHouse/ClickHouse/pull/62519) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Implement Dynamic data type that allows to store values of any type inside it without knowing all of them in advance. Dynamic type is available under a setting `allow_experimental_dynamic_type`. Reference: [#54864](https://github.com/ClickHouse/ClickHouse/issues/54864). [#63058](https://github.com/ClickHouse/ClickHouse/pull/63058) ([Kruglov Pavel](https://github.com/Avogar)). +* Allow to attach parts from a different disk * attach partition from the table on other disks using copy instead of hard link (such as instant table) * attach partition using copy when the hard link fails even on the same disk. [#63087](https://github.com/ClickHouse/ClickHouse/pull/63087) ([Unalian](https://github.com/Unalian)). +* Allow proxy to be bypassed for hosts specified in `no_proxy` env variable and ClickHouse proxy configuration. [#63314](https://github.com/ClickHouse/ClickHouse/pull/63314) ([Arthur Passos](https://github.com/arthurpassos)). +* Added a new table function `loop` to support returning query results in an infinite loop. [#63452](https://github.com/ClickHouse/ClickHouse/pull/63452) ([Sariel](https://github.com/sarielwxm)). +* Added new SQL functions `generateSnowflakeID` for generating Twitter-style Snowflake IDs. [#63577](https://github.com/ClickHouse/ClickHouse/pull/63577) ([Danila Puzov](https://github.com/kazalika)). +* Added `merge_workload` and `mutation_workload` settings to regulate how resources are utilized and shared between merges, mutations and other workloads. [#64061](https://github.com/ClickHouse/ClickHouse/pull/64061) ([Sergei Trifonov](https://github.com/serxa)). +* Equality between IPv4 and IPv6 values can now be evaluated using the `=` operator. [#64292](https://github.com/ClickHouse/ClickHouse/pull/64292) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). +* Allow to store named collections in zookeeper. To enable - add `named_collections_storage` section to server configuration file, where add `type` equal to `zookeeper` and `path` equal to whatever zookeeper path you want to use to store named collections there. [#64574](https://github.com/ClickHouse/ClickHouse/pull/64574) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Decimals are supported as arguments for binary math functions (pow(), atan2(), max2, min2(), hypot(). [#64582](https://github.com/ClickHouse/ClickHouse/pull/64582) ([Mikhail Gorshkov](https://github.com/mgorshkov)). +* Adds index analysis for Hilbert curve. If a table has hilbertEncode (aka Hilbert curve) in its index, e.g. `ORDER BY hilbertEncode(x, y)` then queries that use both `x` and `y` can be processed efficient, for example `WHERE 1 <= x AND x <= 2 AND 3 <= y AND y <= 4`. Works with the setting `analyze_index_with_space_filling_curves`. [#64662](https://github.com/ClickHouse/ClickHouse/pull/64662) ([Artem Mustafin](https://github.com/Artemmm91)). +* Added SQL functions `parseReadableSize` (along with `OrNull` and `OrZero` variants). These functions perform the opposite operation of function `formatReadableSize` and `formatReadableDecimalSize`, i.e. the given human-readable byte size, they return the number of bytes. Example: `SELECT formatReadableSize('3.0 MiB')` returns `3145728`. [#64742](https://github.com/ClickHouse/ClickHouse/pull/64742) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). +* Add 2 server_setting to limit numebr of table/db max_table_num_to_throw max_database_num_to_throw. [#64781](https://github.com/ClickHouse/ClickHouse/pull/64781) ([Xu Jia](https://github.com/XuJia0210)). +* Add _time virtual column to file alike storages (s3/file/hdfs/url/azureBlobStorage). [#64947](https://github.com/ClickHouse/ClickHouse/pull/64947) ([Ilya Golshtein](https://github.com/ilejn)). +* Introduced new functions `base64UrlEncode`, `base64UrlDecode` and `tryBase64UrlDecode`. [#64991](https://github.com/ClickHouse/ClickHouse/pull/64991) ([Mikhail Gorshkov](https://github.com/mgorshkov)). +* Add new function `editDistanceUTF8`, which calculates the [edit distance](https://en.wikipedia.org/wiki/Edit_distance) between two UTF8 strings. [#65269](https://github.com/ClickHouse/ClickHouse/pull/65269) ([LiuNeng](https://github.com/liuneng1994)). + +#### Performance Improvement +* A native parquet reader, which can read parquet binary to ClickHouse Columns directly. Now this feature can be activated by setting `input_format_parquet_use_native_reader` to true. [#60361](https://github.com/ClickHouse/ClickHouse/pull/60361) ([ZhiHong Zhang](https://github.com/copperybean)). +* Reduce useless virtual func call in ColumnNullable of size() func. [#60556](https://github.com/ClickHouse/ClickHouse/pull/60556) ([HappenLee](https://github.com/HappenLee)). +* Reduce lock contention for MergeTree tables (by renaming parts without holding lock). [#61973](https://github.com/ClickHouse/ClickHouse/pull/61973) ([Azat Khuzhin](https://github.com/azat)). +* Function `splitByRegexp` is now faster when the regular expression argument is a single-character, trivial regular expression (in this case, it now falls back internally to `splitByChar`). [#62696](https://github.com/ClickHouse/ClickHouse/pull/62696) ([Robert Schulze](https://github.com/rschu1ze)). +* Add min/max in fixedHashTable to limit the array index and reduce the `isZero()` loop in iterator++. [#62746](https://github.com/ClickHouse/ClickHouse/pull/62746) ([Jiebin Sun](https://github.com/jiebinn)). +* This pull request is the basic solution for [Feature: best effort sorting to optimize compression [#4413](https://github.com/ClickHouse/ClickHouse/issues/4413)](https://github.com/ClickHouse/ClickHouse/issues/4413). [#63578](https://github.com/ClickHouse/ClickHouse/pull/63578) ([Igor Markelov](https://github.com/ElderlyPassionFruit)). +* The queries of 3.3, 3.4, 4.1 and 4.2 from **Star Schema Benchmark** share a common pattern of predicate: `col = a or col = b` where `col` is a `LowCardinality(String)`. And during syntax optimization, this predicate is then transformed equivalently to `col in (a, b)`, which is executed by `FunctionIn` with `col` and a constant tuple `(a, b)` as its arguments. [#64060](https://github.com/ClickHouse/ClickHouse/pull/64060) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Initialisation and destruction of hash tables inside `ConcurrentHashJoin` is executed on a thread pool now. [#64241](https://github.com/ClickHouse/ClickHouse/pull/64241) ([Nikita Taranov](https://github.com/nickitat)). +* Optimized vertical merges in tables with sparse columns. [#64311](https://github.com/ClickHouse/ClickHouse/pull/64311) ([Anton Popov](https://github.com/CurtizJ)). +* Enabled prefetches of data from remote filesystem during vertical merges. It improves latency of vertical merges in tables with data stored on remote filesystem. [#64314](https://github.com/ClickHouse/ClickHouse/pull/64314) ([Anton Popov](https://github.com/CurtizJ)). +* ... Reduce the redundant `isDefault()` of `ColumnSparse::filter` to improve performance. [#64426](https://github.com/ClickHouse/ClickHouse/pull/64426) ([Jiebin Sun](https://github.com/jiebinn)). +* Speedup `find_super_nodes` and `find_big_family` keeper-client commands by making multiple asynchronous getChildren requests. [#64628](https://github.com/ClickHouse/ClickHouse/pull/64628) ([Alexander Gololobov](https://github.com/davenger)). +* Allow merging two consequent `FilterSteps` of a query plan. This improves filter-push-down optimization if the filter condition can be pushed down from the parent step. [#64760](https://github.com/ClickHouse/ClickHouse/pull/64760) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Remove bad optimization in vertical final implementation and re-enable vertical final algorithm by default. [#64783](https://github.com/ClickHouse/ClickHouse/pull/64783) ([Duc Canh Le](https://github.com/canhld94)). +* Remove ALIAS nodes from the filter expression. This slightly improves performance for queries with `PREWHERE` (with new analyzer). [#64793](https://github.com/ClickHouse/ClickHouse/pull/64793) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### Improvement +* Now, empty tuples can be stored, queried, and inserted just like other columns. The implementation is different from [#55021](https://github.com/ClickHouse/ClickHouse/issues/55021). It specializes all related methods of `ColumnTuple`, `DataTypeTuple`, and `SerializationTuple` for empty tuples, which is more complicated but fundamentally correct. [#55061](https://github.com/ClickHouse/ClickHouse/pull/55061) ([Amos Bird](https://github.com/amosbird)). +* Enable asynchronous load of databases and tables by default. See the `async_load_databases` in config.xml. [#57695](https://github.com/ClickHouse/ClickHouse/pull/57695) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Hot reload storage policy for distributed tables when adding a new disk. [#58285](https://github.com/ClickHouse/ClickHouse/pull/58285) ([Duc Canh Le](https://github.com/canhld94)). +* ... Use scheduleOrThrow in MergeTree data selection and initialization to avoid deadlocks. [#59427](https://github.com/ClickHouse/ClickHouse/pull/59427) ([Sean Haynes](https://github.com/seandhaynes)). +* Support partial trivial count optimization when query filter is able to select exact ranges from merge tree tables. This implements a more general version of [#36732](https://github.com/ClickHouse/ClickHouse/issues/36732). [#60463](https://github.com/ClickHouse/ClickHouse/pull/60463) ([Amos Bird](https://github.com/amosbird)). +* This implements new transform `PlanSquashingTransform` which provides improvements for insert process and availability to check memory during inserts. Ref [#59112](https://github.com/ClickHouse/ClickHouse/issues/59112). [#61047](https://github.com/ClickHouse/ClickHouse/pull/61047) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Fixed memory allocation by avoiding last extra buffer allocation. [#63160](https://github.com/ClickHouse/ClickHouse/pull/63160) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Few corner case fixes to proxy support & tunneling. Adjust tests accordingly.. [#63427](https://github.com/ClickHouse/ClickHouse/pull/63427) ([Arthur Passos](https://github.com/arthurpassos)). +* For HTTP Handlers rules add `http_response_headers` setting to support custom response headers. [#63562](https://github.com/ClickHouse/ClickHouse/pull/63562) ([Grigorii](https://github.com/GSokol)). +* Improve io_uring resubmits visibility. Rename profile event `IOUringSQEsResubmits` -> `IOUringSQEsResubmitsAsync` and add a new one `IOUringSQEsResubmitsSync`. [#63699](https://github.com/ClickHouse/ClickHouse/pull/63699) ([Tomer Shafir](https://github.com/tomershafir)). +* Introduce assertions to verify all functions are called with columns of the right size. [#63723](https://github.com/ClickHouse/ClickHouse/pull/63723) ([Raúl Marín](https://github.com/Algunenano)). +* `SHOW CREATE TABLE` executed on top of system tables will now show the super handy comment unique for each table which will explain why this table is needed. [#63788](https://github.com/ClickHouse/ClickHouse/pull/63788) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Added knob `metadata_storage_type` to keep free space on metadata storage disk. [#64128](https://github.com/ClickHouse/ClickHouse/pull/64128) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Metrics to track the number of directories created and removed by the plain_rewritable metadata storage, and the number of entries in the local-to-remote in-memory map. [#64175](https://github.com/ClickHouse/ClickHouse/pull/64175) ([Julia Kartseva](https://github.com/jkartseva)). +* The query cache now considers identical queries with different settings as different. This increases robustness in cases where different settings (e.g. `limit` or `additional_table_filters`) would affect the query result. [#64205](https://github.com/ClickHouse/ClickHouse/pull/64205) ([Robert Schulze](https://github.com/rschu1ze)). +* Better Exception Message in Delete Table with Projection, users can understand the error and the steps should be taken. [#64212](https://github.com/ClickHouse/ClickHouse/pull/64212) ([jsc0218](https://github.com/jsc0218)). +* Test that non standard error code `QpsLimitExceeded` is supported and it is retryable error. [#64225](https://github.com/ClickHouse/ClickHouse/pull/64225) ([Sema Checherinda](https://github.com/CheSema)). +* It is forbidden to convert MergeTree to replicated if the zookeeper path for this table already exists. [#64244](https://github.com/ClickHouse/ClickHouse/pull/64244) ([Kirill](https://github.com/kirillgarbar)). +* If "replica group" is configured for a `Replicated` database, automatically create a cluster that includes replicas from all groups. Closes [#62725](https://github.com/ClickHouse/ClickHouse/issues/62725). [#64312](https://github.com/ClickHouse/ClickHouse/pull/64312) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Added settings to disable materialization of skip indexes and statistics on inserts (`materialize_skip_indexes_on_insert` and `materialize_statistics_on_insert`). [#64391](https://github.com/ClickHouse/ClickHouse/pull/64391) ([Anton Popov](https://github.com/CurtizJ)). +* Use the allocated memory size to calculate the row group size and reduce the peak memory of the parquet writer in single-threaded mode. [#64424](https://github.com/ClickHouse/ClickHouse/pull/64424) ([LiuNeng](https://github.com/liuneng1994)). +* Added new configuration input_format_parquet_prefer_block_bytes to control the average output block bytes, and modified the default value of input_format_parquet_max_block_size to 65409. [#64427](https://github.com/ClickHouse/ClickHouse/pull/64427) ([LiuNeng](https://github.com/liuneng1994)). +* Always start Keeper with sufficient amount of threads in global thread pool. [#64444](https://github.com/ClickHouse/ClickHouse/pull/64444) ([Duc Canh Le](https://github.com/canhld94)). +* Settings from user config doesn't affect merges and mutations for MergeTree on top of object storage. [#64456](https://github.com/ClickHouse/ClickHouse/pull/64456) ([alesapin](https://github.com/alesapin)). +* Setting `replace_long_file_name_to_hash` is enabled by default for `MergeTree` tables. [#64457](https://github.com/ClickHouse/ClickHouse/pull/64457) ([Anton Popov](https://github.com/CurtizJ)). +* Improve the iterator of sparse column to reduce call of size(). [#64497](https://github.com/ClickHouse/ClickHouse/pull/64497) ([Jiebin Sun](https://github.com/jiebinn)). +* Update condition to use copy for azure blob storage. [#64518](https://github.com/ClickHouse/ClickHouse/pull/64518) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Test that totalqpslimitexceeded is retriable s3 error. [#64520](https://github.com/ClickHouse/ClickHouse/pull/64520) ([Sema Checherinda](https://github.com/CheSema)). +* Optimized memory usage of vertical merges for tables with high number of skip indexes. [#64580](https://github.com/ClickHouse/ClickHouse/pull/64580) ([Anton Popov](https://github.com/CurtizJ)). +* 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. [#64597](https://github.com/ClickHouse/ClickHouse/pull/64597) ([Alexey Katsman](https://github.com/alexkats)). +* Add settings `parallel_replicas_custom_key_range_lower` and `parallel_replicas_custom_key_range_upper` to control how parallel replicas with dynamic shards parallelizes queries when using a range filter. [#64604](https://github.com/ClickHouse/ClickHouse/pull/64604) ([josh-hildred](https://github.com/josh-hildred)). +* Updated Advanced Dashboard for both open-source and ClickHouse Cloud versions to include a chart for 'Maximum concurrent network connections'. [#64610](https://github.com/ClickHouse/ClickHouse/pull/64610) ([Thom O'Connor](https://github.com/thomoco)). +* If the arguments is numberic but nullable, and has 2 arguments, it would use `FunctionLeastGreatestGeneric` to do the comparsion, which is not efficient. and use `GreatestBaseImpl/LeastBaseImpl` for `FunctionBinaryArithmetic` would have better performance. So here we improve these two function by allow this. [#64668](https://github.com/ClickHouse/ClickHouse/pull/64668) ([KevinyhZou](https://github.com/KevinyhZou)). +* The second argument (scale) of functions `round()`, `roundBankers()`, `floor()`, `ceil()` and `trunc()` can now be non-const. [#64798](https://github.com/ClickHouse/ClickHouse/pull/64798) ([Mikhail Gorshkov](https://github.com/mgorshkov)). +* Improve progress report on zeros_mt and generateRandom. [#64804](https://github.com/ClickHouse/ClickHouse/pull/64804) ([Raúl Marín](https://github.com/Algunenano)). +* Add an asynchronous metric jemalloc.profile.active to show whether sampling is currently active. This is an activation mechanism in addition to prof.active; both must be active for the calling thread to sample. [#64842](https://github.com/ClickHouse/ClickHouse/pull/64842) ([Unalian](https://github.com/Unalian)). +* Make statistics work on replicated merge tree. [#64934](https://github.com/ClickHouse/ClickHouse/pull/64934) ([Han Fei](https://github.com/hanfei1991)). +* The setting `allow_experimental_join_condition` was accidentally marked as important which may prevent distributed queries in a mixed versions cluster from being executed successfully. [#65008](https://github.com/ClickHouse/ClickHouse/pull/65008) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* All HTTP buffers are now 1MB instead of 8KB to avoid too frequent send syscalls. [#65028](https://github.com/ClickHouse/ClickHouse/pull/65028) ([Sergei Trifonov](https://github.com/serxa)). +* Added server Asynchronous metrics `DiskGetObjectThrottler*` and `DiskGetObjectThrottler*` reflecting request per second rate limit defined with `s3_max_get_rps` and `s3_max_put_rps` disk settings and currently available number of requests that could be sent without hitting throttling limit on the disk. Metrics are defined for every disk that has a configured limit. [#65050](https://github.com/ClickHouse/ClickHouse/pull/65050) ([Sergei Trifonov](https://github.com/serxa)). +* Added a setting `output_format_pretty_display_footer_column_names` which when enabled displays column names at the end of the table for long tables (50 rows by default), with the threshold value for minimum number of rows controlled by `output_format_pretty_display_footer_column_names_min_rows`. [#65144](https://github.com/ClickHouse/ClickHouse/pull/65144) ([Shaun Struwig](https://github.com/Blargian)). +* Returned back the behaviour of how ClickHouse works and interprets Tuples in CSV format. This change effectively reverts https://github.com/ClickHouse/ClickHouse/pull/60994 and makes it available only under a few settings: `output_format_csv_serialize_tuple_into_separate_columns`, `input_format_csv_deserialize_separate_columns_into_tuple` and `input_format_csv_try_infer_strings_from_quoted_tuples`. [#65170](https://github.com/ClickHouse/ClickHouse/pull/65170) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Initialize global trace collector for Poco::ThreadPool (needed for keeper, etc). [#65239](https://github.com/ClickHouse/ClickHouse/pull/65239) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add validation when creating a user with bcrypt_hash. [#65242](https://github.com/ClickHouse/ClickHouse/pull/65242) ([Raúl Marín](https://github.com/Algunenano)). +* Fix performance regression introduced in [#60459](https://github.com/ClickHouse/ClickHouse/issues/60459). [#65243](https://github.com/ClickHouse/ClickHouse/pull/65243) ([Nikita Taranov](https://github.com/nickitat)). + +#### Build/Testing/Packaging Improvement +* Fix typo in test_hdfsCluster_unset_skip_unavailable_shards. The test writes data to unskip_unavailable_shards, but uses skip_unavailable_shards from the previous test. [#64243](https://github.com/ClickHouse/ClickHouse/pull/64243) ([Mikhail Artemenko](https://github.com/Michicosun)). +* Reduce the size of some slow tests. [#64387](https://github.com/ClickHouse/ClickHouse/pull/64387) ([Raúl Marín](https://github.com/Algunenano)). +* Reduce the size of some slow tests. [#64452](https://github.com/ClickHouse/ClickHouse/pull/64452) ([Raúl Marín](https://github.com/Algunenano)). +* Fix test_lost_part_other_replica. [#64512](https://github.com/ClickHouse/ClickHouse/pull/64512) ([Raúl Marín](https://github.com/Algunenano)). +* Add tests for experimental unequal joins and randomize new settings in clickhouse-test. [#64535](https://github.com/ClickHouse/ClickHouse/pull/64535) ([Nikita Fomichev](https://github.com/fm4v)). +* Upgrade tests: Update config and work with release candidates. [#64542](https://github.com/ClickHouse/ClickHouse/pull/64542) ([Raúl Marín](https://github.com/Algunenano)). +* Add support for LLVM XRay. [#64592](https://github.com/ClickHouse/ClickHouse/pull/64592) ([Tomer Shafir](https://github.com/tomershafir)). +* Speed up 02995_forget_partition. [#64761](https://github.com/ClickHouse/ClickHouse/pull/64761) ([Raúl Marín](https://github.com/Algunenano)). +* Fix 02790_async_queries_in_query_log. [#64764](https://github.com/ClickHouse/ClickHouse/pull/64764) ([Raúl Marín](https://github.com/Algunenano)). +* Support LLVM XRay on Linux amd64 only. [#64837](https://github.com/ClickHouse/ClickHouse/pull/64837) ([Tomer Shafir](https://github.com/tomershafir)). +* Get rid of custom code in `tests/ci/download_release_packages.py` and `tests/ci/get_previous_release_tag.py` to avoid issues after the https://github.com/ClickHouse/ClickHouse/pull/64759 is merged. [#64848](https://github.com/ClickHouse/ClickHouse/pull/64848) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Decrease the `unit-test` image a few times. [#65102](https://github.com/ClickHouse/ClickHouse/pull/65102) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fixed 'set' skip index not working with IN and indexHint(). [#62083](https://github.com/ClickHouse/ClickHouse/pull/62083) ([Michael Kolupaev](https://github.com/al13n321)). +* Support executing function during assignment of parameterized view value. [#63502](https://github.com/ClickHouse/ClickHouse/pull/63502) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fixed parquet memory tracking. [#63584](https://github.com/ClickHouse/ClickHouse/pull/63584) ([Michael Kolupaev](https://github.com/al13n321)). +* Fixed reading of columns of type `Tuple(Map(LowCardinality(String), String), ...)`. [#63956](https://github.com/ClickHouse/ClickHouse/pull/63956) ([Anton Popov](https://github.com/CurtizJ)). +* Fix an `Cyclic aliases` error for cyclic aliases of different type (expression and function). Fixes [#63205](https://github.com/ClickHouse/ClickHouse/issues/63205). [#63993](https://github.com/ClickHouse/ClickHouse/pull/63993) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* This fix will use a proper redefined context with the correct definer for each individual view in the query pipeline Closes [#63777](https://github.com/ClickHouse/ClickHouse/issues/63777). [#64079](https://github.com/ClickHouse/ClickHouse/pull/64079) ([pufit](https://github.com/pufit)). +* Fix analyzer: "Not found column" error is fixed when using INTERPOLATE. [#64096](https://github.com/ClickHouse/ClickHouse/pull/64096) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix creating backups to S3 buckets with different credentials from the disk containing the file. [#64153](https://github.com/ClickHouse/ClickHouse/pull/64153) ([Antonio Andelic](https://github.com/antonio2368)). +* The query cache now considers two identical queries against different databases as different. The previous behavior could be used to bypass missing privileges to read from a table. [#64199](https://github.com/ClickHouse/ClickHouse/pull/64199) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix `duplicate alias` error for distributed queries with `ARRAY JOIN`. Fixes [#62043](https://github.com/ClickHouse/ClickHouse/issues/62043). [#64226](https://github.com/ClickHouse/ClickHouse/pull/64226) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix unexpected accurateCast from string to integer ``` SELECT accurateCast('1234', 'UInt8') Before: ┌─accurateCast('1234', 'UInt8')─┐ │ 210 │ └───────────────────────────────┘. [#64255](https://github.com/ClickHouse/ClickHouse/pull/64255) ([wudidapaopao](https://github.com/wudidapaopao)). +* Fixed CNF simplification, in case any OR group contains mutually exclusive atoms. [#64256](https://github.com/ClickHouse/ClickHouse/pull/64256) ([Eduard Karacharov](https://github.com/korowa)). +* Fix Query Tree size validation. Closes [#63701](https://github.com/ClickHouse/ClickHouse/issues/63701). [#64377](https://github.com/ClickHouse/ClickHouse/pull/64377) ([Dmitry Novik](https://github.com/novikd)). +* Fix `Logical error: Bad cast` for `Buffer` table with `PREWHERE`. Fixes [#64172](https://github.com/ClickHouse/ClickHouse/issues/64172). [#64388](https://github.com/ClickHouse/ClickHouse/pull/64388) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Prevent recursive logging in `blob_storage_log` when it's stored on object storage. [#64393](https://github.com/ClickHouse/ClickHouse/pull/64393) ([vdimir](https://github.com/vdimir)). +* Fixed `CREATE TABLE AS` queries for tables with default expressions. [#64455](https://github.com/ClickHouse/ClickHouse/pull/64455) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed `optimize_read_in_order` behaviour for ORDER BY ... NULLS FIRST / LAST on tables with nullable keys. [#64483](https://github.com/ClickHouse/ClickHouse/pull/64483) ([Eduard Karacharov](https://github.com/korowa)). +* Fix the `Expression nodes list expected 1 projection names` and `Unknown expression or identifier` errors for queries with aliases to `GLOBAL IN.` Fixes [#64445](https://github.com/ClickHouse/ClickHouse/issues/64445). [#64517](https://github.com/ClickHouse/ClickHouse/pull/64517) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix an error `Cannot find column` in distributed queries with constant CTE in the `GROUP BY` key. [#64519](https://github.com/ClickHouse/ClickHouse/pull/64519) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed ORC statistics calculation, when writing, for unsigned types on all platforms and Int8 on ARM. [#64563](https://github.com/ClickHouse/ClickHouse/pull/64563) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix the crash loop when restoring from backup is blocked by creating an MV with a definer that hasn't been restored yet. [#64595](https://github.com/ClickHouse/ClickHouse/pull/64595) ([pufit](https://github.com/pufit)). +* Fix the output of function `formatDateTimeInJodaSyntax` when a formatter generates an uneven number of characters and the last character is `0`. For example, `SELECT formatDateTimeInJodaSyntax(toDate('2012-05-29'), 'D')` now correctly returns `150` instead of previously `15`. [#64614](https://github.com/ClickHouse/ClickHouse/pull/64614) ([LiuNeng](https://github.com/liuneng1994)). +* Fix [#64612](https://github.com/ClickHouse/ClickHouse/issues/64612). Do not rewrite aggregation if `-If` combinator is already used. [#64638](https://github.com/ClickHouse/ClickHouse/pull/64638) ([Dmitry Novik](https://github.com/novikd)). +* Fix type inference for float (in case of small buffer, i.e. `--max_read_buffer_size 1`). [#64641](https://github.com/ClickHouse/ClickHouse/pull/64641) ([Azat Khuzhin](https://github.com/azat)). +* Fix bug which could lead to non-working TTLs with expressions. Fixes [#63700](https://github.com/ClickHouse/ClickHouse/issues/63700). [#64694](https://github.com/ClickHouse/ClickHouse/pull/64694) ([alesapin](https://github.com/alesapin)). +* Fix removing the `WHERE` and `PREWHERE` expressions, which are always true (for the new analyzer). Fixes [#64575](https://github.com/ClickHouse/ClickHouse/issues/64575). [#64695](https://github.com/ClickHouse/ClickHouse/pull/64695) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix `loop()` table function crashing on empty table name. [#64716](https://github.com/ClickHouse/ClickHouse/pull/64716) ([Michael Kolupaev](https://github.com/al13n321)). +* Fixed excessive part elimination by token-based text indexes (`ngrambf` , `full_text`) when filtering by result of `startsWith`, `endsWith`, `match`, `multiSearchAny`. [#64720](https://github.com/ClickHouse/ClickHouse/pull/64720) ([Eduard Karacharov](https://github.com/korowa)). +* Fixes incorrect behaviour of ANSI CSI escaping in the `UTF8::computeWidth` function. [#64756](https://github.com/ClickHouse/ClickHouse/pull/64756) ([Shaun Struwig](https://github.com/Blargian)). +* Fix OrderByLimitByDuplicateEliminationVisitor across subqueries. [#64766](https://github.com/ClickHouse/ClickHouse/pull/64766) ([Raúl Marín](https://github.com/Algunenano)). +* Fixed issue [#64513](https://github.com/ClickHouse/ClickHouse/issues/64513). Build subqueries for sets which are in the mixed join conditions. [#64775](https://github.com/ClickHouse/ClickHouse/pull/64775) ([lgbo](https://github.com/lgbo-ustc)). +* Fix crash in a local cache over `plain_rewritable` disk. [#64778](https://github.com/ClickHouse/ClickHouse/pull/64778) ([Julia Kartseva](https://github.com/jkartseva)). +* Keeper fix: return correct value for `zk_latest_snapshot_size` in `mntr` command. [#64784](https://github.com/ClickHouse/ClickHouse/pull/64784) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix `Cannot find column` in distributed query with `ARRAY JOIN` by `Nested` column. Fixes [#64755](https://github.com/ClickHouse/ClickHouse/issues/64755). [#64801](https://github.com/ClickHouse/ClickHouse/pull/64801) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix memory leak in slru cache policy. Closes [#64791](https://github.com/ClickHouse/ClickHouse/issues/64791). [#64803](https://github.com/ClickHouse/ClickHouse/pull/64803) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed memory possible incorrect memory tracking in several kinds of queries: queries that read any data from S3, queries via http protocol, asynchronous inserts. [#64844](https://github.com/ClickHouse/ClickHouse/pull/64844) ([Anton Popov](https://github.com/CurtizJ)). +* Fix the `Block structure mismatch` error for queries reading with `PREWHERE` from the materialized view when the materialized view has columns of different types than the source table. Fixes [#64611](https://github.com/ClickHouse/ClickHouse/issues/64611). [#64855](https://github.com/ClickHouse/ClickHouse/pull/64855) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix rare crash when table has TTL with subquery + database replicated + parallel replicas + analyzer. It's really rare, but please don't use TTLs with subqueries. [#64858](https://github.com/ClickHouse/ClickHouse/pull/64858) ([alesapin](https://github.com/alesapin)). +* Fix duplicating `Delete` events in `blob_storage_log` in case of large batch to delete. [#64924](https://github.com/ClickHouse/ClickHouse/pull/64924) ([vdimir](https://github.com/vdimir)). +* Fixed `Session moved to another server` error from [Zoo]Keeper that might happen after server startup when the config has includes from [Zoo]Keeper. [#64986](https://github.com/ClickHouse/ClickHouse/pull/64986) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `ALTER MODIFY COMMENT` query that was broken for parameterized VIEWs in https://github.com/ClickHouse/ClickHouse/pull/54211. [#65031](https://github.com/ClickHouse/ClickHouse/pull/65031) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix `host_id` in DatabaseReplicated when `cluster_secure_connection` parameter is enabled. Previously all the connections within the cluster created by DatabaseReplicated were not secure, even if the parameter was enabled. Closes [#64216](https://github.com/ClickHouse/ClickHouse/issues/64216). [#65054](https://github.com/ClickHouse/ClickHouse/pull/65054) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fixing the `Not-ready Set` error after the `PREWHERE` optimization for StorageMerge. [#65057](https://github.com/ClickHouse/ClickHouse/pull/65057) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Avoid writing to finalized buffer in File-like storages. Closes [#62962](https://github.com/ClickHouse/ClickHouse/issues/62962). [#65063](https://github.com/ClickHouse/ClickHouse/pull/65063) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix possible infinite query duration in case of cyclic aliases. Fixes [#64849](https://github.com/ClickHouse/ClickHouse/issues/64849). [#65081](https://github.com/ClickHouse/ClickHouse/pull/65081) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix the `Unknown expression identifier` error for remote queries with `INTERPOLATE (alias)` (new analyzer). Fixes [#64636](https://github.com/ClickHouse/ClickHouse/issues/64636). [#65090](https://github.com/ClickHouse/ClickHouse/pull/65090) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix pushing arithmetic operations out of aggregation. In the new analyzer, optimization was applied only once. Part of [#62245](https://github.com/ClickHouse/ClickHouse/issues/62245). [#65104](https://github.com/ClickHouse/ClickHouse/pull/65104) ([Dmitry Novik](https://github.com/novikd)). +* Fix aggregate function name rewriting in the new analyzer. Part of [#62245](https://github.com/ClickHouse/ClickHouse/issues/62245). [#65110](https://github.com/ClickHouse/ClickHouse/pull/65110) ([Dmitry Novik](https://github.com/novikd)). +* Respond with 5xx instead of 200 OK in case of receive timeout while reading (parts of) the request body from the client socket. [#65118](https://github.com/ClickHouse/ClickHouse/pull/65118) ([Julian Maicher](https://github.com/jmaicher)). +* Fix possible crash for hedged requests. [#65206](https://github.com/ClickHouse/ClickHouse/pull/65206) ([Azat Khuzhin](https://github.com/azat)). +* Fix the bug in Hashed and Hashed_Array dictionary short circuit evaluation, which may read uninitialized number, leading to various errors. [#65256](https://github.com/ClickHouse/ClickHouse/pull/65256) ([jsc0218](https://github.com/jsc0218)). +* This PR ensures that the type of the constant(IN operator's second parameter) is always visible during the IN operator's type conversion process. Otherwise, losing type information may cause some conversions to fail, such as the conversion from DateTime to Date. fix ([#64487](https://github.com/ClickHouse/ClickHouse/issues/64487)). [#65315](https://github.com/ClickHouse/ClickHouse/pull/65315) ([pn](https://github.com/chloro-pn)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) + +* Disabled `enable_vertical_final` setting by default. This feature should not be used because it has a bug: [#64543](https://github.com/ClickHouse/ClickHouse/issues/64543). [#64544](https://github.com/ClickHouse/ClickHouse/pull/64544) ([Alexander Tokmakov](https://github.com/tavplubix)). +* This PR fixes an error when a user in a specific situation can escalate their privileges on the default database without necessary grants. [#64769](https://github.com/ClickHouse/ClickHouse/pull/64769) ([pufit](https://github.com/pufit)). +* Fix crash with UniqInjectiveFunctionsEliminationPass and uniqCombined. [#65188](https://github.com/ClickHouse/ClickHouse/pull/65188) ([Raúl Marín](https://github.com/Algunenano)). +* Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Forbid `QUALIFY` clause in the old analyzer. The old analyzer ignored `QUALIFY`, so it could lead to unexpected data removal in mutations. Closes [#64596](https://github.com/ClickHouse/ClickHouse/issues/64596). [#65356](https://github.com/ClickHouse/ClickHouse/pull/65356) ([Dmitry Novik](https://github.com/novikd)). +* Use correct memory alignment for Distinct combinator. Previously, crash could happen because of invalid memory allocation when the combinator was used. [#65379](https://github.com/ClickHouse/ClickHouse/pull/65379) ([Antonio Andelic](https://github.com/antonio2368)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Refactoring of Server.h: Isolate server management from other logic"'. [#64425](https://github.com/ClickHouse/ClickHouse/pull/64425) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Remove some unnecessary `UNREACHABLE`s"'. [#64430](https://github.com/ClickHouse/ClickHouse/pull/64430) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "CI: fix build_report selection in case of job reuse"'. [#64516](https://github.com/ClickHouse/ClickHouse/pull/64516) ([Max K.](https://github.com/maxknv)). +* NO CL ENTRY: 'Revert "Revert "CI: fix build_report selection in case of job reuse""'. [#64531](https://github.com/ClickHouse/ClickHouse/pull/64531) ([Max K.](https://github.com/maxknv)). +* NO CL ENTRY: 'Revert "Add `fromReadableSize` function"'. [#64616](https://github.com/ClickHouse/ClickHouse/pull/64616) ([Robert Schulze](https://github.com/rschu1ze)). +* NO CL ENTRY: 'Update CHANGELOG.md'. [#64816](https://github.com/ClickHouse/ClickHouse/pull/64816) ([Paweł Kudzia](https://github.com/pakud)). +* NO CL ENTRY: 'Revert "Reduce lock contention for MergeTree tables (by renaming parts without holding lock)"'. [#64899](https://github.com/ClickHouse/ClickHouse/pull/64899) ([alesapin](https://github.com/alesapin)). +* NO CL ENTRY: 'Revert "Add dynamic untracked memory limits for more precise memory tracking"'. [#64969](https://github.com/ClickHouse/ClickHouse/pull/64969) ([Sergei Trifonov](https://github.com/serxa)). +* NO CL ENTRY: 'Revert "Fix duplicating Delete events in blob_storage_log"'. [#65049](https://github.com/ClickHouse/ClickHouse/pull/65049) ([Alexander Tokmakov](https://github.com/tavplubix)). +* NO CL ENTRY: 'Revert "Revert "Fix duplicating Delete events in blob_storage_log""'. [#65053](https://github.com/ClickHouse/ClickHouse/pull/65053) ([vdimir](https://github.com/vdimir)). +* NO CL ENTRY: 'Revert "S3: reduce retires time for queries, increase retries count for backups"'. [#65148](https://github.com/ClickHouse/ClickHouse/pull/65148) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Small fix for 02340_parts_refcnt_mergetree"'. [#65149](https://github.com/ClickHouse/ClickHouse/pull/65149) ([Raúl Marín](https://github.com/Algunenano)). +* NO CL ENTRY: 'Revert "Change default s3_throw_on_zero_files_match to true, document that presigned S3 URLs are not supported"'. [#65250](https://github.com/ClickHouse/ClickHouse/pull/65250) ([Max K.](https://github.com/maxknv)). +* NO CL ENTRY: 'Revert "Fix AWS ECS"'. [#65361](https://github.com/ClickHouse/ClickHouse/pull/65361) ([Alexander Tokmakov](https://github.com/tavplubix)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Try abort on current thread join. [#42544](https://github.com/ClickHouse/ClickHouse/pull/42544) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Change default s3_throw_on_zero_files_match to true, document that presigned S3 URLs are not supported. [#51008](https://github.com/ClickHouse/ClickHouse/pull/51008) ([Michael Kolupaev](https://github.com/al13n321)). +* Analyzer fuzzer 2. [#57098](https://github.com/ClickHouse/ClickHouse/pull/57098) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Analyzer fuzzer 4. [#57101](https://github.com/ClickHouse/ClickHouse/pull/57101) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Check python code with flake8. [#58349](https://github.com/ClickHouse/ClickHouse/pull/58349) ([Azat Khuzhin](https://github.com/azat)). +* Unite s3/hdfs/azure storage implementations into a single class working with IObjectStorage. Same for *Cluster, data lakes and Queue storages. [#59767](https://github.com/ClickHouse/ClickHouse/pull/59767) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Refactor KeyCondition and key analysis. [#61459](https://github.com/ClickHouse/ClickHouse/pull/61459) ([Amos Bird](https://github.com/amosbird)). +* Replay ZK logs using keeper-bench. [#62481](https://github.com/ClickHouse/ClickHouse/pull/62481) ([Antonio Andelic](https://github.com/antonio2368)). +* Improve builds' profiling upload. [#63325](https://github.com/ClickHouse/ClickHouse/pull/63325) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Refactor data part writer to remove dependencies on MergeTreeData and DataPart. [#63620](https://github.com/ClickHouse/ClickHouse/pull/63620) ([Alexander Gololobov](https://github.com/davenger)). +* Try to fix flaky s3 tests test_seekable_formats and test_seekable_formats_url. [#63720](https://github.com/ClickHouse/ClickHouse/pull/63720) ([Kruglov Pavel](https://github.com/Avogar)). +* Rewrite plan for parallel replicas in Planner. [#63796](https://github.com/ClickHouse/ClickHouse/pull/63796) ([Igor Nikonov](https://github.com/devcrafter)). +* S3: reduce retires time for queries, increase retries count for backups. [#63857](https://github.com/ClickHouse/ClickHouse/pull/63857) ([Sema Checherinda](https://github.com/CheSema)). +* Implement a single point for "Check timeout expired" test result. [#63982](https://github.com/ClickHouse/ClickHouse/pull/63982) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Remove some unnecessary `UNREACHABLE`s. [#64035](https://github.com/ClickHouse/ClickHouse/pull/64035) ([Robert Schulze](https://github.com/rschu1ze)). +* Throw out some `inline`s. [#64110](https://github.com/ClickHouse/ClickHouse/pull/64110) ([Robert Schulze](https://github.com/rschu1ze)). +* Refactoring of Server.h: Isolate server management from other logic. [#64132](https://github.com/ClickHouse/ClickHouse/pull/64132) ([TTPO100AJIEX](https://github.com/TTPO100AJIEX)). +* Fix: 02124_insert_deduplication_token_multiple_blocks_replica. [#64181](https://github.com/ClickHouse/ClickHouse/pull/64181) ([Igor Nikonov](https://github.com/devcrafter)). +* Add profile events for number of rows read during/after prewhere. [#64198](https://github.com/ClickHouse/ClickHouse/pull/64198) ([Nikita Taranov](https://github.com/nickitat)). +* Update InterpreterCreateQuery.cpp. [#64207](https://github.com/ClickHouse/ClickHouse/pull/64207) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Remove unused storage_snapshot field from MergeTreeSelectProcessor. [#64217](https://github.com/ClickHouse/ClickHouse/pull/64217) ([Alexander Gololobov](https://github.com/davenger)). +* Add test for [#37090](https://github.com/ClickHouse/ClickHouse/issues/37090). [#64220](https://github.com/ClickHouse/ClickHouse/pull/64220) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Make `settings_changes_history` const. [#64230](https://github.com/ClickHouse/ClickHouse/pull/64230) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* test for [#45804](https://github.com/ClickHouse/ClickHouse/issues/45804). [#64245](https://github.com/ClickHouse/ClickHouse/pull/64245) ([Denny Crane](https://github.com/den-crane)). +* Update version after release. [#64283](https://github.com/ClickHouse/ClickHouse/pull/64283) ([Raúl Marín](https://github.com/Algunenano)). +* Followup for [#63691](https://github.com/ClickHouse/ClickHouse/issues/63691). [#64285](https://github.com/ClickHouse/ClickHouse/pull/64285) ([vdimir](https://github.com/vdimir)). +* CI: dependency fix for changelog.py. [#64293](https://github.com/ClickHouse/ClickHouse/pull/64293) ([Max K.](https://github.com/maxknv)). +* Print query in explain plan with parallel replicas. [#64298](https://github.com/ClickHouse/ClickHouse/pull/64298) ([vdimir](https://github.com/vdimir)). +* CI: Cancel sync wf on new push. [#64299](https://github.com/ClickHouse/ClickHouse/pull/64299) ([Max K.](https://github.com/maxknv)). +* CI: master workflow with folded jobs. [#64340](https://github.com/ClickHouse/ClickHouse/pull/64340) ([Max K.](https://github.com/maxknv)). +* CI: Sync, Merge check, CI gh's statuses fixes. [#64348](https://github.com/ClickHouse/ClickHouse/pull/64348) ([Max K.](https://github.com/maxknv)). +* Enable 02494_query_cache_nested_query_bug for Analyzer. [#64357](https://github.com/ClickHouse/ClickHouse/pull/64357) ([Robert Schulze](https://github.com/rschu1ze)). +* Rename allow_deprecated_functions to allow_deprecated_error_prone_win…. [#64358](https://github.com/ClickHouse/ClickHouse/pull/64358) ([Raúl Marín](https://github.com/Algunenano)). +* Change input_format_parquet_use_native_reader to 24.6. [#64359](https://github.com/ClickHouse/ClickHouse/pull/64359) ([Raúl Marín](https://github.com/Algunenano)). +* Update description for settings `cross_join_min_rows_to_compress` and `cross_join_min_bytes_to_compress`. [#64360](https://github.com/ClickHouse/ClickHouse/pull/64360) ([Nikita Fomichev](https://github.com/fm4v)). +* Rename aggregate_function_group_array_has_limit_size. [#64362](https://github.com/ClickHouse/ClickHouse/pull/64362) ([Raúl Marín](https://github.com/Algunenano)). +* Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts. [#64363](https://github.com/ClickHouse/ClickHouse/pull/64363) ([Kruglov Pavel](https://github.com/Avogar)). +* Try to fix GWPAsan. [#64365](https://github.com/ClickHouse/ClickHouse/pull/64365) ([Antonio Andelic](https://github.com/antonio2368)). +* CI: add secrets to reusable stage wf yml. [#64366](https://github.com/ClickHouse/ClickHouse/pull/64366) ([Max K.](https://github.com/maxknv)). +* Do not run tests tagged 'no-s3-storage-with-slow-build' with ASan. [#64367](https://github.com/ClickHouse/ClickHouse/pull/64367) ([vdimir](https://github.com/vdimir)). +* Add `fromReadableSize` function. [#64386](https://github.com/ClickHouse/ClickHouse/pull/64386) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). +* Update s3queue.md. [#64389](https://github.com/ClickHouse/ClickHouse/pull/64389) ([Kseniia Sumarokova](https://github.com/kssenii)). +* test for [#64211](https://github.com/ClickHouse/ClickHouse/issues/64211). [#64390](https://github.com/ClickHouse/ClickHouse/pull/64390) ([Denny Crane](https://github.com/den-crane)). +* Follow-up to [#59767](https://github.com/ClickHouse/ClickHouse/issues/59767). [#64398](https://github.com/ClickHouse/ClickHouse/pull/64398) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Remove wrong comment. [#64403](https://github.com/ClickHouse/ClickHouse/pull/64403) ([Sergei Trifonov](https://github.com/serxa)). +* Follow up to [#59767](https://github.com/ClickHouse/ClickHouse/issues/59767). [#64404](https://github.com/ClickHouse/ClickHouse/pull/64404) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Refactor s3 settings (move settings parsing into single place). [#64412](https://github.com/ClickHouse/ClickHouse/pull/64412) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add dynamic untracked memory limits for more precise memory tracking. [#64423](https://github.com/ClickHouse/ClickHouse/pull/64423) ([Sergei Trifonov](https://github.com/serxa)). +* Fix test after [#64404](https://github.com/ClickHouse/ClickHouse/issues/64404). [#64432](https://github.com/ClickHouse/ClickHouse/pull/64432) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Faster TestKeeper shutdown. [#64433](https://github.com/ClickHouse/ClickHouse/pull/64433) ([Alexander Gololobov](https://github.com/davenger)). +* Remove some logging. [#64434](https://github.com/ClickHouse/ClickHouse/pull/64434) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Revert "Revert "Remove some unnecessary UNREACHABLEs"". [#64435](https://github.com/ClickHouse/ClickHouse/pull/64435) ([Robert Schulze](https://github.com/rschu1ze)). +* Clean settings in 02943_variant_read_subcolumns test. [#64437](https://github.com/ClickHouse/ClickHouse/pull/64437) ([Kruglov Pavel](https://github.com/Avogar)). +* Add a comment after [#64226](https://github.com/ClickHouse/ClickHouse/issues/64226). [#64449](https://github.com/ClickHouse/ClickHouse/pull/64449) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* CI: fix build_report selection in case of job reuse. [#64459](https://github.com/ClickHouse/ClickHouse/pull/64459) ([Max K.](https://github.com/maxknv)). +* CI: Critical bugfix category in PR template. [#64480](https://github.com/ClickHouse/ClickHouse/pull/64480) ([Max K.](https://github.com/maxknv)). +* Remove `generateSnowflakeIDThreadMonotonic`. [#64499](https://github.com/ClickHouse/ClickHouse/pull/64499) ([Robert Schulze](https://github.com/rschu1ze)). +* Move analyzer attempt 2. [#64500](https://github.com/ClickHouse/ClickHouse/pull/64500) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Sync some code back from internal to public repository. [#64502](https://github.com/ClickHouse/ClickHouse/pull/64502) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove `generateUUIDv7(NonMonotonic|ThreadMonotonic)` functions. [#64506](https://github.com/ClickHouse/ClickHouse/pull/64506) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix bash completion for settings. [#64521](https://github.com/ClickHouse/ClickHouse/pull/64521) ([Azat Khuzhin](https://github.com/azat)). +* Use max_read_buffer_size for file descriptors as well in file(). [#64532](https://github.com/ClickHouse/ClickHouse/pull/64532) ([Azat Khuzhin](https://github.com/azat)). +* Bring hash table sizes tests back. [#64550](https://github.com/ClickHouse/ClickHouse/pull/64550) ([Nikita Taranov](https://github.com/nickitat)). +* Sync code moved in private repo back back to public repo. [#64551](https://github.com/ClickHouse/ClickHouse/pull/64551) ([Robert Schulze](https://github.com/rschu1ze)). +* ASTLiteral support custom type. [#64562](https://github.com/ClickHouse/ClickHouse/pull/64562) ([shuai.xu](https://github.com/shuai-xu)). +* Add a temporary known host for git over ssh. [#64569](https://github.com/ClickHouse/ClickHouse/pull/64569) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Cache first analysis result in ReadFromMergeTree. [#64579](https://github.com/ClickHouse/ClickHouse/pull/64579) ([Igor Nikonov](https://github.com/devcrafter)). +* CI: Backports updates. [#64603](https://github.com/ClickHouse/ClickHouse/pull/64603) ([Max K.](https://github.com/maxknv)). +* CI: Change Required Check list, few fixes. [#64605](https://github.com/ClickHouse/ClickHouse/pull/64605) ([Max K.](https://github.com/maxknv)). +* Double-checking [#59318](https://github.com/ClickHouse/ClickHouse/issues/59318) and docs for `Map`. [#64606](https://github.com/ClickHouse/ClickHouse/pull/64606) ([Robert Schulze](https://github.com/rschu1ze)). +* Update CHANGELOG.md. [#64609](https://github.com/ClickHouse/ClickHouse/pull/64609) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Tests: Convert numeric to symbolic error codes. [#64635](https://github.com/ClickHouse/ClickHouse/pull/64635) ([Robert Schulze](https://github.com/rschu1ze)). +* Move NamedCollectionsFactory into a separate file. [#64642](https://github.com/ClickHouse/ClickHouse/pull/64642) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Shuffle tests for parallel execution. [#64646](https://github.com/ClickHouse/ClickHouse/pull/64646) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* CI: Do not upload binaries for special builds in PRs. [#64653](https://github.com/ClickHouse/ClickHouse/pull/64653) ([Max K.](https://github.com/maxknv)). +* Update changelog. [#64654](https://github.com/ClickHouse/ClickHouse/pull/64654) ([Robert Schulze](https://github.com/rschu1ze)). +* Parallel replicas: simple cleanup. [#64655](https://github.com/ClickHouse/ClickHouse/pull/64655) ([Igor Nikonov](https://github.com/devcrafter)). +* Be more graceful with existing tables with `inverted` indexes. [#64656](https://github.com/ClickHouse/ClickHouse/pull/64656) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: Build Report Check to verify only enabled builds. [#64669](https://github.com/ClickHouse/ClickHouse/pull/64669) ([Max K.](https://github.com/maxknv)). +* Tests: Convert error numbers to symbolic error codes, pt. II. [#64670](https://github.com/ClickHouse/ClickHouse/pull/64670) ([Robert Schulze](https://github.com/rschu1ze)). +* Split query analyzer. [#64672](https://github.com/ClickHouse/ClickHouse/pull/64672) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* CI: CI running status fix. [#64693](https://github.com/ClickHouse/ClickHouse/pull/64693) ([Max K.](https://github.com/maxknv)). +* Cleanup the changelog for 24.5. [#64704](https://github.com/ClickHouse/ClickHouse/pull/64704) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* CI: MergeQueue: add binary_release and unit tests. [#64705](https://github.com/ClickHouse/ClickHouse/pull/64705) ([Max K.](https://github.com/maxknv)). +* CI: get_best_robot_token fix to get good enough token. [#64709](https://github.com/ClickHouse/ClickHouse/pull/64709) ([Max K.](https://github.com/maxknv)). +* Check for missing Upload ID in CreateMultipartUpload reply. [#64714](https://github.com/ClickHouse/ClickHouse/pull/64714) ([Michael Kolupaev](https://github.com/al13n321)). +* Update version_date.tsv and changelogs after v24.5.1.1763-stable. [#64715](https://github.com/ClickHouse/ClickHouse/pull/64715) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update CHANGELOG.md. [#64730](https://github.com/ClickHouse/ClickHouse/pull/64730) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CI: ci.py refactoring. [#64734](https://github.com/ClickHouse/ClickHouse/pull/64734) ([Max K.](https://github.com/maxknv)). +* Return the explanation for session moved error. [#64747](https://github.com/ClickHouse/ClickHouse/pull/64747) ([Antonio Andelic](https://github.com/antonio2368)). +* Adjust the `version_helper` and script to a new release scheme. [#64759](https://github.com/ClickHouse/ClickHouse/pull/64759) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Do not try to write columns.txt if it does not exists for write-once storages. [#64762](https://github.com/ClickHouse/ClickHouse/pull/64762) ([Azat Khuzhin](https://github.com/azat)). +* Update 02482_load_parts_refcounts.sh. [#64765](https://github.com/ClickHouse/ClickHouse/pull/64765) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix crash with DISTINCT and window functions. [#64767](https://github.com/ClickHouse/ClickHouse/pull/64767) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix assert in IObjectStorageIteratorAsync. [#64770](https://github.com/ClickHouse/ClickHouse/pull/64770) ([Michael Kolupaev](https://github.com/al13n321)). +* Make table functions always report engine 'StorageProxy' in system.tables. [#64771](https://github.com/ClickHouse/ClickHouse/pull/64771) ([Michael Kolupaev](https://github.com/al13n321)). +* Ask about company name on GitHub. [#64774](https://github.com/ClickHouse/ClickHouse/pull/64774) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix flaky tests about SQLite. [#64776](https://github.com/ClickHouse/ClickHouse/pull/64776) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove iostream debug helpers. [#64777](https://github.com/ClickHouse/ClickHouse/pull/64777) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Remove unnecessary comment. [#64785](https://github.com/ClickHouse/ClickHouse/pull/64785) ([Raúl Marín](https://github.com/Algunenano)). +* Strict `StatusType`, increased `SAFE_REQUESTS_LIMIT` for get_best_robot_token. [#64787](https://github.com/ClickHouse/ClickHouse/pull/64787) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Attempt to fix 02228_merge_tree_insert_memory_usage.sql flakiness for s3. [#64800](https://github.com/ClickHouse/ClickHouse/pull/64800) ([Raúl Marín](https://github.com/Algunenano)). +* Add regression test for filter propagation through `Merge` engine. [#64806](https://github.com/ClickHouse/ClickHouse/pull/64806) ([Nikita Taranov](https://github.com/nickitat)). +* Migrate changelog.py to a descendant of fuzzywuzzy. [#64807](https://github.com/ClickHouse/ClickHouse/pull/64807) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Changelog in ci. [#64813](https://github.com/ClickHouse/ClickHouse/pull/64813) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Make row order optimization non-experimental. [#64814](https://github.com/ClickHouse/ClickHouse/pull/64814) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix wrong supported year recognition. [#64817](https://github.com/ClickHouse/ClickHouse/pull/64817) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix clang-tidy build. [#64823](https://github.com/ClickHouse/ClickHouse/pull/64823) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: Move coverage build to non-special build list. [#64824](https://github.com/ClickHouse/ClickHouse/pull/64824) ([Max K.](https://github.com/maxknv)). +* CI: fix CI await feature. [#64825](https://github.com/ClickHouse/ClickHouse/pull/64825) ([Max K.](https://github.com/maxknv)). +* Fix clang-tidy. [#64827](https://github.com/ClickHouse/ClickHouse/pull/64827) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Upload blob_storage_log from stateless tests. [#64843](https://github.com/ClickHouse/ClickHouse/pull/64843) ([alesapin](https://github.com/alesapin)). +* Follow-up to [#64349](https://github.com/ClickHouse/ClickHouse/issues/64349). [#64845](https://github.com/ClickHouse/ClickHouse/pull/64845) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Simplify handling of old 'inverted' indexes. [#64846](https://github.com/ClickHouse/ClickHouse/pull/64846) ([Robert Schulze](https://github.com/rschu1ze)). +* Replace Markdown with YAML for ISSUE_TEMPLATE. [#64850](https://github.com/ClickHouse/ClickHouse/pull/64850) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Handle logs from rocksdb by ClickHouse internal logging (and enable debug rocksdb logging on CI). [#64856](https://github.com/ClickHouse/ClickHouse/pull/64856) ([Azat Khuzhin](https://github.com/azat)). +* Add new settings to changes history. [#64860](https://github.com/ClickHouse/ClickHouse/pull/64860) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* added mlock and mlockall to aspell-dict to be ignored. [#64863](https://github.com/ClickHouse/ClickHouse/pull/64863) ([Ali](https://github.com/xogoodnow)). +* A tiny fix for fancy quotes. [#64883](https://github.com/ClickHouse/ClickHouse/pull/64883) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix possible loss of "Query was cancelled" message in client (fixes 03023_zeros_generate_random_with_limit_progress_bar flakiness). [#64888](https://github.com/ClickHouse/ClickHouse/pull/64888) ([Azat Khuzhin](https://github.com/azat)). +* Fix documentation enforcement. [#64890](https://github.com/ClickHouse/ClickHouse/pull/64890) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix global trace collector. [#64896](https://github.com/ClickHouse/ClickHouse/pull/64896) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix test_mask_sensitive_info/test.py::test_create_table. [#64901](https://github.com/ClickHouse/ClickHouse/pull/64901) ([Azat Khuzhin](https://github.com/azat)). +* Update 03165_string_functions_with_token_text_indexes.sql. [#64903](https://github.com/ClickHouse/ClickHouse/pull/64903) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Persistent PRInfo diff url. [#64904](https://github.com/ClickHouse/ClickHouse/pull/64904) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Disable transactions for unsupported storages even for materialized v…. [#64918](https://github.com/ClickHouse/ClickHouse/pull/64918) ([alesapin](https://github.com/alesapin)). +* additional log for cleanupDetachedTables. [#64919](https://github.com/ClickHouse/ClickHouse/pull/64919) ([Konstantin Morozov](https://github.com/k-morozov)). +* Fix tupleConcat of two empty tuples. [#64923](https://github.com/ClickHouse/ClickHouse/pull/64923) ([Amos Bird](https://github.com/amosbird)). +* CI: Minor fixes in ci scripts. [#64950](https://github.com/ClickHouse/ClickHouse/pull/64950) ([Max K.](https://github.com/maxknv)). +* Fix error message (it was strange). [#64952](https://github.com/ClickHouse/ClickHouse/pull/64952) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Update fmtlib version to 9.1.0. [#64959](https://github.com/ClickHouse/ClickHouse/pull/64959) ([Duc Canh Le](https://github.com/canhld94)). +* Disable 02908_many_requests_to_system_replicas with ASAN. [#64966](https://github.com/ClickHouse/ClickHouse/pull/64966) ([Alexander Gololobov](https://github.com/davenger)). +* Fix bug in short circuit evaluation. [#64967](https://github.com/ClickHouse/ClickHouse/pull/64967) ([Raúl Marín](https://github.com/Algunenano)). +* Update version_date.tsv and changelogs after v24.4.2.141-stable. [#64968](https://github.com/ClickHouse/ClickHouse/pull/64968) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix `test_attach_partition_using_copy`. [#64977](https://github.com/ClickHouse/ClickHouse/pull/64977) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Faster processing of scheduler queue activations. [#64985](https://github.com/ClickHouse/ClickHouse/pull/64985) ([Sergei Trifonov](https://github.com/serxa)). +* CI: Fix nightly workflow. [#64987](https://github.com/ClickHouse/ClickHouse/pull/64987) ([Max K.](https://github.com/maxknv)). +* Fix innocuous data race in detectLanguage. [#64988](https://github.com/ClickHouse/ClickHouse/pull/64988) ([Raúl Marín](https://github.com/Algunenano)). +* CI: Builds in CI settings. [#64994](https://github.com/ClickHouse/ClickHouse/pull/64994) ([Max K.](https://github.com/maxknv)). +* Aggregate function groupConcat ([#63671](https://github.com/ClickHouse/ClickHouse/issues/63671)). [#65009](https://github.com/ClickHouse/ClickHouse/pull/65009) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* CI: Fix backports. [#65010](https://github.com/ClickHouse/ClickHouse/pull/65010) ([Max K.](https://github.com/maxknv)). +* Try fix 03143_prewhere_profile_events. [#65014](https://github.com/ClickHouse/ClickHouse/pull/65014) ([Nikita Taranov](https://github.com/nickitat)). +* Fix 03165_string_functions_with_token_text_indexes. [#65018](https://github.com/ClickHouse/ClickHouse/pull/65018) ([Julia Kartseva](https://github.com/jkartseva)). +* Bump googletest to latest HEAD. [#65038](https://github.com/ClickHouse/ClickHouse/pull/65038) ([Robert Schulze](https://github.com/rschu1ze)). +* Improve comment about AsynchronousMetrics. [#65040](https://github.com/ClickHouse/ClickHouse/pull/65040) ([Antonio Andelic](https://github.com/antonio2368)). +* CI: Remove fuzzer build from normal CI run (bugfix). [#65041](https://github.com/ClickHouse/ClickHouse/pull/65041) ([Max K.](https://github.com/maxknv)). +* CI: ci_config refactoring. [#65045](https://github.com/ClickHouse/ClickHouse/pull/65045) ([Max K.](https://github.com/maxknv)). +* Bump abseil to latest HEAD. [#65048](https://github.com/ClickHouse/ClickHouse/pull/65048) ([Robert Schulze](https://github.com/rschu1ze)). +* Capture weak_ptr of ContextAccess for safety. [#65051](https://github.com/ClickHouse/ClickHouse/pull/65051) ([Alexander Gololobov](https://github.com/davenger)). +* Stateless tests: add test for SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT. [#65056](https://github.com/ClickHouse/ClickHouse/pull/65056) ([Nikita Fomichev](https://github.com/fm4v)). +* Increase timeout in wait_for_all_mutations. [#65058](https://github.com/ClickHouse/ClickHouse/pull/65058) ([Alexander Gololobov](https://github.com/davenger)). +* Tests for _time virtual column in file alike storages. [#65064](https://github.com/ClickHouse/ClickHouse/pull/65064) ([Ilya Golshtein](https://github.com/ilejn)). +* Update odbc-bridge.md. [#65099](https://github.com/ClickHouse/ClickHouse/pull/65099) ([Alexander Gololobov](https://github.com/davenger)). +* Small fix for 02340_parts_refcnt_mergetree. [#65105](https://github.com/ClickHouse/ClickHouse/pull/65105) ([Nikita Taranov](https://github.com/nickitat)). +* Re-enable OpenSSL session caching. [#65111](https://github.com/ClickHouse/ClickHouse/pull/65111) ([Robert Schulze](https://github.com/rschu1ze)). +* Update test_replicated_database/test.py. [#65112](https://github.com/ClickHouse/ClickHouse/pull/65112) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix false positives leaky memory warnings in OpenSSL. [#65125](https://github.com/ClickHouse/ClickHouse/pull/65125) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix `Initiator received more initial requests than there are replicas` with `loop` engine. [#65133](https://github.com/ClickHouse/ClickHouse/pull/65133) ([Nikita Taranov](https://github.com/nickitat)). +* Fix 'Tasks in BackgroundSchedulePool cannot throw' caused by MergeTreeData::loadUnexpectedDataParts(). [#65135](https://github.com/ClickHouse/ClickHouse/pull/65135) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix bad error message. [#65137](https://github.com/ClickHouse/ClickHouse/pull/65137) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* rework tests with sleep: use sleep_until instead sleep_for. [#65152](https://github.com/ClickHouse/ClickHouse/pull/65152) ([Sema Checherinda](https://github.com/CheSema)). +* Fix AWS ECS. [#65164](https://github.com/ClickHouse/ClickHouse/pull/65164) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Ensure submodules are named consistently. [#65167](https://github.com/ClickHouse/ClickHouse/pull/65167) ([Robert Schulze](https://github.com/rschu1ze)). +* Remove obsolete fix from aws submodule. [#65168](https://github.com/ClickHouse/ClickHouse/pull/65168) ([Robert Schulze](https://github.com/rschu1ze)). +* CI: Fix not-merged cherry-picks for backports. [#65181](https://github.com/ClickHouse/ClickHouse/pull/65181) ([Max K.](https://github.com/maxknv)). +* Add an assertion in ReplicatedMergeTreeQueue. [#65184](https://github.com/ClickHouse/ClickHouse/pull/65184) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix bug in short circuit optimization with cache dictionaries. [#65185](https://github.com/ClickHouse/ClickHouse/pull/65185) ([Raúl Marín](https://github.com/Algunenano)). +* Fix docs for skipping-indexes.md. [#65194](https://github.com/ClickHouse/ClickHouse/pull/65194) ([morning-color](https://github.com/morning-color)). +* Fix the descriptions of some server settings. [#65200](https://github.com/ClickHouse/ClickHouse/pull/65200) ([Raúl Marín](https://github.com/Algunenano)). +* Fix search issues for progress_func, add zstd for GH cache. [#65202](https://github.com/ClickHouse/ClickHouse/pull/65202) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix bug in short circuit optimization with direct dictionaries. [#65203](https://github.com/ClickHouse/ClickHouse/pull/65203) ([Raúl Marín](https://github.com/Algunenano)). +* Add test prewhere merge. [#65207](https://github.com/ClickHouse/ClickHouse/pull/65207) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Sync ProfileEvents.h. [#65208](https://github.com/ClickHouse/ClickHouse/pull/65208) ([Kseniia Sumarokova](https://github.com/kssenii)). +* CI: FinishCheck to set failure if workflow failed. [#65228](https://github.com/ClickHouse/ClickHouse/pull/65228) ([Max K.](https://github.com/maxknv)). +* Update version_date.tsv and changelogs after v24.3.4.147-lts. [#65235](https://github.com/ClickHouse/ClickHouse/pull/65235) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Update version_date.tsv and changelogs after v24.5.3.5-stable. [#65240](https://github.com/ClickHouse/ClickHouse/pull/65240) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Less flaky `01019_alter_materialized_view_consistent`. [#65245](https://github.com/ClickHouse/ClickHouse/pull/65245) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix libunwind in CI. [#65247](https://github.com/ClickHouse/ClickHouse/pull/65247) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* CI: Do not skip FinishCheck in Merge Queue. [#65249](https://github.com/ClickHouse/ClickHouse/pull/65249) ([Max K.](https://github.com/maxknv)). +* Add a test just in case. [#65271](https://github.com/ClickHouse/ClickHouse/pull/65271) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Disable 02581_share_big_sets_between_multiple_mutations_tasks_long in coverage run. [#65295](https://github.com/ClickHouse/ClickHouse/pull/65295) ([Alexander Gololobov](https://github.com/davenger)). +* Update version_date.tsv and changelogs after v23.8.15.35-lts. [#65300](https://github.com/ClickHouse/ClickHouse/pull/65300) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* mute test test_query_is_canceled_with_inf_retries. [#65301](https://github.com/ClickHouse/ClickHouse/pull/65301) ([Sema Checherinda](https://github.com/CheSema)). +* Fix silly typo that caused wrong tags messages. [#65307](https://github.com/ClickHouse/ClickHouse/pull/65307) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Save server data for failed stateless tests. [#65309](https://github.com/ClickHouse/ClickHouse/pull/65309) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix 01246_buffer_flush flakiness (by tuning timeouts). [#65310](https://github.com/ClickHouse/ClickHouse/pull/65310) ([Azat Khuzhin](https://github.com/azat)). +* Remove outdated override in stress tests. [#65323](https://github.com/ClickHouse/ClickHouse/pull/65323) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix bad code in `system.session_log`. [#65332](https://github.com/ClickHouse/ClickHouse/pull/65332) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* add tests for 'boom filter index with map'. [#65333](https://github.com/ClickHouse/ClickHouse/pull/65333) ([iceFireser](https://github.com/iceFireser)). +* Fix crash in 03036_dynamic_read_subcolumns. [#65341](https://github.com/ClickHouse/ClickHouse/pull/65341) ([Kruglov Pavel](https://github.com/Avogar)). +* Move tests 02942_variant_cast and 02944_variant_as_common_type to analyzer_tech_debt.txt. [#65342](https://github.com/ClickHouse/ClickHouse/pull/65342) ([Kruglov Pavel](https://github.com/Avogar)). +* Add docs for groupConcat. [#65384](https://github.com/ClickHouse/ClickHouse/pull/65384) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* CI: Add Non-blocking (Woolen wolfdog) CI mode. [#65385](https://github.com/ClickHouse/ClickHouse/pull/65385) ([Max K.](https://github.com/maxknv)). +* Fix compatibility release check. [#65394](https://github.com/ClickHouse/ClickHouse/pull/65394) ([Alexey Katsman](https://github.com/alexkats)). +* Move a leaksan suppression from Poco into OpenSSL. [#65396](https://github.com/ClickHouse/ClickHouse/pull/65396) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix tidy build. [#65415](https://github.com/ClickHouse/ClickHouse/pull/65415) ([Sergei Trifonov](https://github.com/serxa)). +* CI: Master workflow dependencies fix. [#65416](https://github.com/ClickHouse/ClickHouse/pull/65416) ([Max K.](https://github.com/maxknv)). +* CI: PR workflow dependencies fix. [#65442](https://github.com/ClickHouse/ClickHouse/pull/65442) ([Max K.](https://github.com/maxknv)). +* Fix test_storage_s3_queue/test.py::test_max_set_age. [#65452](https://github.com/ClickHouse/ClickHouse/pull/65452) ([Kseniia Sumarokova](https://github.com/kssenii)). +* CI: Rename A Sync status. [#65456](https://github.com/ClickHouse/ClickHouse/pull/65456) ([Max K.](https://github.com/maxknv)). +* CI: Rename sync status. [#65464](https://github.com/ClickHouse/ClickHouse/pull/65464) ([Max K.](https://github.com/maxknv)). +* Fix `01926_order_by_desc_limit` test for 1MB HTTP buffers. [#65466](https://github.com/ClickHouse/ClickHouse/pull/65466) ([Sergei Trifonov](https://github.com/serxa)). +* Remove groupConcat. [#65480](https://github.com/ClickHouse/ClickHouse/pull/65480) ([Raúl Marín](https://github.com/Algunenano)). + +#### Packaging Improvement + +* Let `network` service be required when using the init script to start the ClickHouse server daemon. [#60650](https://github.com/ClickHouse/ClickHouse/pull/60650) ([Chun-Sheng, Li](https://github.com/peter279k)). + ### ClickHouse release 24.5, 2024-05-30 #### Backward Incompatible Change From 9c66375e6849bf2b427900018bb5ae553f4f5ae9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 19 Jun 2024 17:54:40 +0300 Subject: [PATCH 269/318] 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 b33a373a970..b1ab5561958 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 dbcaf5f00a7..51f6bc83dac 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 136d474751a..25895788e2e 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 0dacdc0b958..671b3b2e05c 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 11371578c79..5dbf6fa3318 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 bc84e31be62..e33d67d7150 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h @@ -41,7 +41,7 @@ private: 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 126d4824fff..ca4ea4f3704 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 @@ public: 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 @@ protected: 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 9379cb5a1c6..a8713c61e4d 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 1ed4f7a7611..a8fa22f71b2 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -37,11 +37,12 @@ public: 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 afc458ea612..fc85bde11d9 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 fb6af21907e..2cd7e3676b5 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 270/318] 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 51f6bc83dac..da4e104d18b 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 271/318] 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 dad4e6747e0..d1cbc54d294 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}" From 5edf7dd9df7786556a65b7b4f516bef82a708147 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Jun 2024 15:13:20 +0200 Subject: [PATCH 272/318] Strict parsing in Keeper client --- programs/keeper-client/KeeperClient.cpp | 2 +- programs/keeper-client/Parser.cpp | 10 ++-------- programs/keeper-client/Parser.h | 1 - 3 files changed, 3 insertions(+), 10 deletions(-) diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index ebec337060c..68adc2c2aac 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -368,7 +368,7 @@ int KeeperClient::main(const std::vector & /* args */) return 0; } - DB::ConfigProcessor config_processor(config().getString("config-file", "config.xml")); + ConfigProcessor config_processor(config().getString("config-file", "config.xml")); /// This will handle a situation when clickhouse is running on the embedded config, but config.d folder is also present. ConfigProcessor::registerEmbeddedConfig("config.xml", ""); diff --git a/programs/keeper-client/Parser.cpp b/programs/keeper-client/Parser.cpp index 5b16e6d2c23..2c8adee53e4 100644 --- a/programs/keeper-client/Parser.cpp +++ b/programs/keeper-client/Parser.cpp @@ -13,12 +13,6 @@ bool parseKeeperArg(IParser::Pos & pos, Expected & expected, String & result) return false; } - while (pos->type != TokenType::Whitespace && pos->type != TokenType::EndOfStream && pos->type != TokenType::Semicolon) - { - result.append(pos->begin, pos->end); - ++pos; - } - ParserToken{TokenType::Whitespace}.ignore(pos); if (result.empty()) @@ -40,8 +34,8 @@ bool KeeperParser::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) for (const auto & pair : KeeperClient::commands) expected.add(pos, pair.first.data()); - for (const auto & flwc : four_letter_word_commands) - expected.add(pos, flwc.data()); + for (const auto & four_letter_word_command : four_letter_word_commands) + expected.add(pos, four_letter_word_command.data()); if (pos->type != TokenType::BareWord) return false; diff --git a/programs/keeper-client/Parser.h b/programs/keeper-client/Parser.h index 57ee6ce4a18..503edfa4f73 100644 --- a/programs/keeper-client/Parser.h +++ b/programs/keeper-client/Parser.h @@ -11,7 +11,6 @@ namespace DB { bool parseKeeperArg(IParser::Pos & pos, Expected & expected, String & result); - bool parseKeeperPath(IParser::Pos & pos, Expected & expected, String & path); From a447a25a9d779693e329cd743309b067a5ab1c00 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Jun 2024 15:17:05 +0200 Subject: [PATCH 273/318] Strict parsing in Keeper client --- programs/keeper-client/Parser.cpp | 5 ++++ ...lickhouse_keeper_client_no_confirmation.sh | 8 +++--- .../02883_zookeeper_finalize_stress.sh | 2 +- .../03135_keeper_client_find_commands.sh | 28 +++++++++---------- 4 files changed, 24 insertions(+), 19 deletions(-) diff --git a/programs/keeper-client/Parser.cpp b/programs/keeper-client/Parser.cpp index 2c8adee53e4..51f85cf4a69 100644 --- a/programs/keeper-client/Parser.cpp +++ b/programs/keeper-client/Parser.cpp @@ -12,6 +12,11 @@ bool parseKeeperArg(IParser::Pos & pos, Expected & expected, String & result) if (!parseIdentifierOrStringLiteral(pos, expected, result)) return false; } + else if (pos->type == TokenType::Number) + { + result.append(pos->begin, pos->end); + ++pos; + } ParserToken{TokenType::Whitespace}.ignore(pos); diff --git a/tests/queries/0_stateless/02882_clickhouse_keeper_client_no_confirmation.sh b/tests/queries/0_stateless/02882_clickhouse_keeper_client_no_confirmation.sh index 4bda0cfa5b0..43f86b8a58a 100755 --- a/tests/queries/0_stateless/02882_clickhouse_keeper_client_no_confirmation.sh +++ b/tests/queries/0_stateless/02882_clickhouse_keeper_client_no_confirmation.sh @@ -6,8 +6,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) path="/test-keeper-client-$CLICKHOUSE_DATABASE" -$CLICKHOUSE_KEEPER_CLIENT -q "rm $path" >& /dev/null +$CLICKHOUSE_KEEPER_CLIENT -q "rm '$path'" >& /dev/null -$CLICKHOUSE_KEEPER_CLIENT -q "create $path 'foobar'" -$CLICKHOUSE_KEEPER_CLIENT -q "rmr $path" -$CLICKHOUSE_KEEPER_CLIENT -q "get $path" 2>&1 +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path' 'foobar'" +$CLICKHOUSE_KEEPER_CLIENT -q "rmr '$path'" +$CLICKHOUSE_KEEPER_CLIENT -q "get '$path'" 2>&1 diff --git a/tests/queries/0_stateless/02883_zookeeper_finalize_stress.sh b/tests/queries/0_stateless/02883_zookeeper_finalize_stress.sh index dc7d67fbdd4..c883cd8f58a 100755 --- a/tests/queries/0_stateless/02883_zookeeper_finalize_stress.sh +++ b/tests/queries/0_stateless/02883_zookeeper_finalize_stress.sh @@ -7,4 +7,4 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -yes /keeper/api_version | head -n1000 | xargs -P30 -i $CLICKHOUSE_KEEPER_CLIENT -q 'get {}' > /dev/null +yes /keeper/api_version | head -n1000 | xargs -P30 -i $CLICKHOUSE_KEEPER_CLIENT -q "get '{}'" > /dev/null diff --git a/tests/queries/0_stateless/03135_keeper_client_find_commands.sh b/tests/queries/0_stateless/03135_keeper_client_find_commands.sh index 0f57694028d..43ffdec7346 100755 --- a/tests/queries/0_stateless/03135_keeper_client_find_commands.sh +++ b/tests/queries/0_stateless/03135_keeper_client_find_commands.sh @@ -6,24 +6,24 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) path="/test-keeper-client-$CLICKHOUSE_DATABASE" -$CLICKHOUSE_KEEPER_CLIENT -q "rm $path" >& /dev/null +$CLICKHOUSE_KEEPER_CLIENT -q "rm '$path'" >& /dev/null -$CLICKHOUSE_KEEPER_CLIENT -q "create $path 'foobar'" -$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1 'foobar'" -$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/a 'foobar'" -$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/a/a 'foobar'" -$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/b 'foobar'" -$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/c 'foobar'" -$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/d 'foobar'" -$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/d/a 'foobar'" -$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/d/b 'foobar'" -$CLICKHOUSE_KEEPER_CLIENT -q "create $path/1/d/c 'foobar'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path' 'foobar'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1' 'foobar'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/a' 'foobar'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/a/a' 'foobar'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/b' 'foobar'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/c' 'foobar'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/d' 'foobar'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/d/a' 'foobar'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/d/b' 'foobar'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/1/d/c' 'foobar'" echo 'find_super_nodes' $CLICKHOUSE_KEEPER_CLIENT -q "find_super_nodes 1000000000" -$CLICKHOUSE_KEEPER_CLIENT -q "find_super_nodes 3 $path" | sort +$CLICKHOUSE_KEEPER_CLIENT -q "find_super_nodes 3 '$path'" | sort echo 'find_big_family' -$CLICKHOUSE_KEEPER_CLIENT -q "find_big_family $path 3" +$CLICKHOUSE_KEEPER_CLIENT -q "find_big_family '$path' 3" -$CLICKHOUSE_KEEPER_CLIENT -q "rmr $path" +$CLICKHOUSE_KEEPER_CLIENT -q "rmr '$path'" From fbd9fc9680a2b7e413e2879100f345ff8b458789 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 20 Jun 2024 15:18:22 +0100 Subject: [PATCH 274/318] impl --- src/Common/CurrentMetrics.cpp | 1 + src/Common/ProfileEvents.cpp | 1 + src/Core/ServerSettings.h | 1 + src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.h | 1 + 5 files changed, 5 insertions(+) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 731c72d65f2..8516a88c7af 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -1,6 +1,7 @@ #include +// clang-format off /// Available metrics. Add something here as you wish. /// If the metric is generic (i.e. not server specific) /// it should be also added to src/Coordination/KeeperConstant.cpp diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index fef1c4a2b75..3c1e1724359 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -3,6 +3,7 @@ #include +// clang-format off /// Available events. Add something here as you wish. /// If the event is generic (i.e. not server specific) /// it should be also added to src/Coordination/KeeperConstant.cpp diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 1fde8d58c7b..2dec122425e 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -14,6 +14,7 @@ class AbstractConfiguration; namespace DB { +// clang-format off #define SERVER_SETTINGS(M, ALIAS) \ M(Bool, show_addresses_in_stack_traces, true, "If it is set true will show addresses in stack traces", 0) \ M(Bool, shutdown_wait_unfinished_queries, false, "If set true ClickHouse will wait for running queries finish before shutdown.", 0) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e3c122467bd..838cb1e0b1c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -31,6 +31,7 @@ class IColumn; * for tracking settings changes in different versions and for special `compatibility` setting to work correctly. */ +// clang-format off #define COMMON_SETTINGS(M, ALIAS) \ M(Dialect, dialect, Dialect::clickhouse, "Which dialect will be used to parse query", 0)\ M(UInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index cdc955b38bc..895db9c7ca0 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -75,6 +75,7 @@ namespace SettingsChangesHistory using SettingsChanges = std::vector; } +// clang-format off /// History of settings changes that controls some backward incompatible changes /// across all ClickHouse versions. It maps ClickHouse version to settings changes that were done /// in this version. This history contains both changes to existing settings and newly added settings. From fb603c65ac1328242e6dbfa9c37094a9dd97dccd Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 20 Jun 2024 14:28:33 +0000 Subject: [PATCH 275/318] Add ZSTD codec to system.error_log columns --- src/Interpreters/ErrorLog.cpp | 52 ++++++++++++++++++++++++++++++----- 1 file changed, 45 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ErrorLog.cpp b/src/Interpreters/ErrorLog.cpp index 051db08e790..3cf8bf131c4 100644 --- a/src/Interpreters/ErrorLog.cpp +++ b/src/Interpreters/ErrorLog.cpp @@ -9,6 +9,8 @@ #include #include #include +#include +#include #include @@ -17,14 +19,50 @@ namespace DB ColumnsDescription ErrorLogElement::getColumnsDescription() { + ParserCodec codec_parser; return ColumnsDescription { - {"hostname", std::make_shared(std::make_shared()), "Hostname of the server executing the query."}, - {"event_date", std::make_shared(), "Event date."}, - {"event_time", std::make_shared(), "Event time."}, - {"code", std::make_shared(), "Error code}"}, - {"error", std::make_shared(std::make_shared()), "Error name."}, - {"value", std::make_shared(), "Number of errors happened in time interval."}, - {"remote", std::make_shared(), "Remote exception (i.e. received during one of the distributed queries)."} + { + "hostname", + std::make_shared(std::make_shared()), + parseQuery(codec_parser, "(ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS), + "Hostname of the server executing the query." + }, + { + "event_date", + std::make_shared(), + parseQuery(codec_parser, "(Delta(2), ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS), + "Event date." + }, + { + "event_time", + std::make_shared(), + parseQuery(codec_parser, "(Delta(4), ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS), + "Event time." + }, + { + "code", + std::make_shared(), + parseQuery(codec_parser, "(ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS), + "Error code." + }, + { + "error", + std::make_shared(std::make_shared()), + parseQuery(codec_parser, "(ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS), + "Error name." + }, + { + "value", + std::make_shared(), + parseQuery(codec_parser, "(ZSTD(3))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS), + "Number of errors happened in time interval." + }, + { + "remote", + std::make_shared(), + parseQuery(codec_parser, "(ZSTD(1))", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS), + "Remote exception (i.e. received during one of the distributed queries)." + } }; } From d82759760b904e6fb478b1185ee209cae7a89aa7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 20 Jun 2024 16:38:07 +0200 Subject: [PATCH 276/318] Fix test_keeper_snapshots --- tests/integration/test_keeper_snapshots/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_snapshots/test.py b/tests/integration/test_keeper_snapshots/test.py index 951970dba23..fa472367bd7 100644 --- a/tests/integration/test_keeper_snapshots/test.py +++ b/tests/integration/test_keeper_snapshots/test.py @@ -189,7 +189,7 @@ def test_invalid_snapshot(started_cluster): f"/var/lib/clickhouse/coordination/snapshots/{last_snapshot}", ] ) - node.start_clickhouse(expected_to_fail=True) + node.start_clickhouse(start_wait_sec=120, expected_to_fail=True) assert node.contains_in_log( "Aborting because of failure to load from latest snapshot with index" ) From 216b9a7a003c4e27f44726152962599ccfbfde53 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Thu, 20 Jun 2024 07:55:53 -0700 Subject: [PATCH 277/318] Fix out-of-range exception in parsing Dwarf5 on s390x --- src/Common/Dwarf.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Common/Dwarf.cpp b/src/Common/Dwarf.cpp index 99da3b75429..8439c01b22c 100644 --- a/src/Common/Dwarf.cpp +++ b/src/Common/Dwarf.cpp @@ -202,7 +202,10 @@ uint64_t readU64(std::string_view & sp) { SAFE_CHECK(sp.size() >= N, "underflow"); uint64_t x = 0; - memcpy(&x, sp.data(), N); + if constexpr (std::endian::native == std::endian::little) + memcpy(&x, sp.data(), N); + else + memcpy(reinterpret_cast(&x) + sizeof(uint64_t) - N, sp.data(), N); sp.remove_prefix(N); return x; } From e97c730e852e40152cca66cdab516af894cb7273 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 20 Jun 2024 17:06:18 +0200 Subject: [PATCH 278/318] Revert "Use 1MB HTTP buffers to avoid frequent send syscalls" --- base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h | 2 +- tests/integration/test_checking_s3_blobs_paranoid/test.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h b/base/poco/Net/include/Poco/Net/HTTPBasicStreamBuf.h index 3178306363c..c87719b63a4 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 = 1024 * 1024; + constexpr size_t HTTP_DEFAULT_BUFFER_SIZE = 8 * 1024; typedef Poco::BasicBufferedStreamBuf> HTTPBasicStreamBuf; diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 912fd3cc163..b3b8bf86800 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=100000, + s3_min_upload_part_size=1000000, 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 == 69 + assert upload_parts == 7 assert s3_errors == 3 broken_s3.setup_at_part_upload( From 400596756effd74ab50a1237f083c8c45c5dae4c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 20 Jun 2024 17:09:08 +0200 Subject: [PATCH 279/318] Revert "Resubmit http_external_tables_memory_tracking test" --- ..._external_tables_memory_tracking.reference | 16 ------ ...52_http_external_tables_memory_tracking.sh | 57 ------------------- 2 files changed, 73 deletions(-) delete mode 100644 tests/queries/0_stateless/02152_http_external_tables_memory_tracking.reference delete mode 100755 tests/queries/0_stateless/02152_http_external_tables_memory_tracking.sh diff --git a/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.reference b/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.reference deleted file mode 100644 index 1fc09c8d154..00000000000 --- a/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.reference +++ /dev/null @@ -1,16 +0,0 @@ -Checking input_format_parallel_parsing=false& -1 -Checking input_format_parallel_parsing=false&cancel_http_readonly_queries_on_client_close=1&readonly=1 -1 -Checking input_format_parallel_parsing=false&send_progress_in_http_headers=true -1 -Checking input_format_parallel_parsing=false&cancel_http_readonly_queries_on_client_close=1&readonly=1&send_progress_in_http_headers=true -1 -Checking input_format_parallel_parsing=true& -1 -Checking input_format_parallel_parsing=true&cancel_http_readonly_queries_on_client_close=1&readonly=1 -1 -Checking input_format_parallel_parsing=true&send_progress_in_http_headers=true -1 -Checking input_format_parallel_parsing=true&cancel_http_readonly_queries_on_client_close=1&readonly=1&send_progress_in_http_headers=true -1 diff --git a/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.sh b/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.sh deleted file mode 100755 index 5494f7d59cb..00000000000 --- a/tests/queries/0_stateless/02152_http_external_tables_memory_tracking.sh +++ /dev/null @@ -1,57 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-tsan, no-cpu-aarch64, no-parallel, no-debug -# TSan does not supports tracing. -# trace_log doesn't work on aarch64 - -# Regression for proper release of Context, -# via tracking memory of external tables. - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -tmp_file=$(mktemp "$CURDIR/clickhouse.XXXXXX.csv") -trap 'rm $tmp_file' EXIT - -$CLICKHOUSE_CLIENT -q "SELECT toString(number) FROM numbers(1e6) FORMAT TSV" > "$tmp_file" - -function run_and_check() -{ - local query_id - query_id="$(${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @- <<<'SELECT generateUUIDv4()')" - - echo "Checking $*" - - # Run query with external table (implicit StorageMemory user) - $CLICKHOUSE_CURL -sS -F "s=@$tmp_file;" "$CLICKHOUSE_URL&s_structure=key+Int&query=SELECT+count()+FROM+s&memory_profiler_sample_probability=1&max_untracked_memory=0&query_id=$query_id&$*" -o /dev/null - - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" --data-binary @- <<<'SYSTEM FLUSH LOGS' - - # Check that temporary table had been destroyed. - ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&allow_introspection_functions=1" --data-binary @- <<<" - WITH arrayStringConcat(arrayMap(x -> demangle(addressToSymbol(x)), trace), '\n') AS sym - SELECT 1 FROM system.trace_log - PREWHERE - query_id = '$query_id' AND - trace_type = 'MemorySample' AND - /* only deallocations */ - size < 0 AND - event_date >= yesterday() - WHERE - sym LIKE '%DB::StorageMemory::drop%\n%TemporaryTableHolder::~TemporaryTableHolder%' - LIMIT 1 - " -} - -for input_format_parallel_parsing in false true; do - query_args_variants=( - "" - "cancel_http_readonly_queries_on_client_close=1&readonly=1" - "send_progress_in_http_headers=true" - # nested progress callback - "cancel_http_readonly_queries_on_client_close=1&readonly=1&send_progress_in_http_headers=true" - ) - for query_args in "${query_args_variants[@]}"; do - run_and_check "input_format_parallel_parsing=$input_format_parallel_parsing&$query_args" - done -done From a878c941c19703bd91c16f5aeda6a41b76942724 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 20 Jun 2024 15:13:14 +0000 Subject: [PATCH 280/318] Revert #65466 --- 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 89aeb5b22e5..a0047a2925a 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 = '600M'; +SETTINGS max_memory_usage = '400M'; SELECT s FROM order_by_desc ORDER BY u LIMIT 10 FORMAT Null -SETTINGS max_memory_usage = '600M'; +SETTINGS max_memory_usage = '400M'; SYSTEM FLUSH LOGS; From 6277d97105db7b3c529ba28685c916d1c9e28655 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 20 Jun 2024 15:35:09 +0000 Subject: [PATCH 281/318] Remove another unused header --- src/Interpreters/StorageID.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index 69dac8ea32d..f9afbc7b98d 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -1,7 +1,6 @@ #pragma once #include #include -#include #include #include #include From 079b7af58a6ffe07a997d69516a7150eeba0511a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Jun 2024 17:40:01 +0200 Subject: [PATCH 282/318] Fix integration tests --- tests/integration/helpers/keeper_utils.py | 24 +++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 39fa0d0f074..ab66d1a8866 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -124,27 +124,27 @@ class KeeperClient(object): return data def cd(self, path: str, timeout: float = 60.0): - self.execute_query(f"cd {path}", timeout) + self.execute_query(f"cd '{path}'", timeout) def ls(self, path: str, timeout: float = 60.0) -> list[str]: - return self.execute_query(f"ls {path}", timeout).split(" ") + return self.execute_query(f"ls '{path}'", timeout).split(" ") def create(self, path: str, value: str, timeout: float = 60.0): - self.execute_query(f"create {path} {value}", timeout) + self.execute_query(f"create '{path}' '{value}'", timeout) def get(self, path: str, timeout: float = 60.0) -> str: - return self.execute_query(f"get {path}", timeout) + return self.execute_query(f"get '{path}'", timeout) def set(self, path: str, value: str, version: tp.Optional[int] = None) -> None: self.execute_query( - f"set {path} {value} {version if version is not None else ''}" + f"set '{path}' '{value}' '{version if version is not None else ''}'" ) def rm(self, path: str, version: tp.Optional[int] = None) -> None: - self.execute_query(f"rm {path} {version if version is not None else ''}") + self.execute_query(f"rm '{path}' {version if version is not None else ''}") def exists(self, path: str, timeout: float = 60.0) -> bool: - return bool(int(self.execute_query(f"exists {path}", timeout))) + return bool(int(self.execute_query(f"exists '{path}'", timeout))) def stop(self): if not self.stopped: @@ -152,22 +152,22 @@ class KeeperClient(object): self.proc.communicate(b"exit\n", timeout=10.0) def sync(self, path: str, timeout: float = 60.0): - self.execute_query(f"sync {path}", timeout) + self.execute_query(f"sync '{path}'", timeout) def touch(self, path: str, timeout: float = 60.0): - self.execute_query(f"touch {path}", timeout) + self.execute_query(f"touch '{path}'", timeout) def find_big_family(self, path: str, n: int = 10, timeout: float = 60.0) -> str: - return self.execute_query(f"find_big_family {path} {n}", timeout) + return self.execute_query(f"find_big_family '{path}' {n}", timeout) def find_super_nodes(self, threshold: int, timeout: float = 60.0) -> str: return self.execute_query(f"find_super_nodes {threshold}", timeout) def get_direct_children_number(self, path: str, timeout: float = 60.0) -> str: - return self.execute_query(f"get_direct_children_number {path}", timeout) + return self.execute_query(f"get_direct_children_number '{path}'", timeout) def get_all_children_number(self, path: str, timeout: float = 60.0) -> str: - return self.execute_query(f"get_all_children_number {path}", timeout) + return self.execute_query(f"get_all_children_number '{path}'", timeout) def delete_stale_backups(self, timeout: float = 60.0) -> str: return self.execute_query("delete_stale_backups", timeout) From 5982aa3d19f74ec16fd7156495baf64716045bcb Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 20 Jun 2024 15:40:50 +0000 Subject: [PATCH 283/318] Consolidate code into PeriodicLog --- src/Interpreters/ErrorLog.cpp | 92 +++++++++----------------------- src/Interpreters/ErrorLog.h | 26 ++------- src/Interpreters/MetricLog.cpp | 88 ++++++++---------------------- src/Interpreters/MetricLog.h | 24 ++------- src/Interpreters/PeriodicLog.cpp | 64 ++++++++++++++++++++++ src/Interpreters/PeriodicLog.h | 44 +++++++++++++++ src/Interpreters/SystemLog.cpp | 4 +- 7 files changed, 166 insertions(+), 176 deletions(-) create mode 100644 src/Interpreters/PeriodicLog.cpp create mode 100644 src/Interpreters/PeriodicLog.h diff --git a/src/Interpreters/ErrorLog.cpp b/src/Interpreters/ErrorLog.cpp index 3cf8bf131c4..42616f13e24 100644 --- a/src/Interpreters/ErrorLog.cpp +++ b/src/Interpreters/ErrorLog.cpp @@ -79,85 +79,43 @@ void ErrorLogElement::appendToBlock(MutableColumns & columns) const columns[column_idx++]->insert(remote); } -void ErrorLog::startCollectError(size_t collect_interval_milliseconds_) -{ - collect_interval_milliseconds = collect_interval_milliseconds_; - is_shutdown_error_thread = false; - flush_thread = std::make_unique([this] { threadFunction(); }); -} - - -void ErrorLog::stopCollectError() -{ - bool old_val = false; - if (!is_shutdown_error_thread.compare_exchange_strong(old_val, true)) - return; - if (flush_thread) - flush_thread->join(); -} - - -void ErrorLog::shutdown() -{ - stopCollectError(); - stopFlushThread(); -} - struct ValuePair { UInt64 local = 0; UInt64 remote = 0; }; -void ErrorLog::threadFunction() +void ErrorLog::stepFunction(TimePoint current_time) { - auto desired_timepoint = std::chrono::system_clock::now(); - std::vector previous_values(ErrorCodes::end()); + /// Static lazy initialization to avoid polluting the header with implementation details + static std::vector previous_values(ErrorCodes::end()); - while (!is_shutdown_error_thread) + auto event_time = std::chrono::system_clock::to_time_t(current_time); + + for (ErrorCodes::ErrorCode code = 0, end = ErrorCodes::end(); code < end; ++code) { - try + const auto & error = ErrorCodes::values[code].get(); + if (error.local.count != previous_values.at(code).local) { - const auto current_time = std::chrono::system_clock::now(); - auto event_time = std::chrono::system_clock::to_time_t(current_time); - - for (ErrorCodes::ErrorCode code = 0, end = ErrorCodes::end(); code < end; ++code) - { - const auto & error = ErrorCodes::values[code].get(); - if (error.local.count != previous_values.at(code).local) - { - ErrorLogElement local_elem { - .event_time=event_time, - .code=code, - .value=error.local.count - previous_values.at(code).local, - .remote=false - }; - this->add(std::move(local_elem)); - previous_values[code].local = error.local.count; - } - if (error.remote.count != previous_values.at(code).remote) - { - ErrorLogElement remote_elem { - .event_time=event_time, - .code=code, - .value=error.remote.count - previous_values.at(code).remote, - .remote=true - }; - this->add(std::move(remote_elem)); - previous_values[code].remote = error.remote.count; - } - } - - /// We will record current time into table but align it to regular time intervals to avoid time drift. - /// We may drop some time points if the server is overloaded and recording took too much time. - while (desired_timepoint <= current_time) - desired_timepoint += std::chrono::milliseconds(collect_interval_milliseconds); - - std::this_thread::sleep_until(desired_timepoint); + ErrorLogElement local_elem { + .event_time=event_time, + .code=code, + .value=error.local.count - previous_values.at(code).local, + .remote=false + }; + this->add(std::move(local_elem)); + previous_values[code].local = error.local.count; } - catch (...) + if (error.remote.count != previous_values.at(code).remote) { - tryLogCurrentException(__PRETTY_FUNCTION__); + ErrorLogElement remote_elem { + .event_time=event_time, + .code=code, + .value=error.remote.count - previous_values.at(code).remote, + .remote=true + }; + this->add(std::move(remote_elem)); + previous_values[code].remote = error.remote.count; } } } diff --git a/src/Interpreters/ErrorLog.h b/src/Interpreters/ErrorLog.h index 6fb5ead559b..4afe334d4de 100644 --- a/src/Interpreters/ErrorLog.h +++ b/src/Interpreters/ErrorLog.h @@ -1,15 +1,12 @@ #pragma once #include +#include #include -#include #include #include #include -#include -#include - namespace DB { @@ -31,25 +28,12 @@ struct ErrorLogElement }; -class ErrorLog : public SystemLog +class ErrorLog : public PeriodicLog { - using SystemLog::SystemLog; + using PeriodicLog::PeriodicLog; -public: - void shutdown() override; - - /// Launches a background thread to collect errors with interval - void startCollectError(size_t collect_interval_milliseconds_); - - /// Stop background thread - void stopCollectError(); - -private: - void threadFunction(); - - std::unique_ptr flush_thread; - size_t collect_interval_milliseconds; - std::atomic is_shutdown_error_thread{false}; +protected: + void stepFunction(TimePoint current_time) override; }; } diff --git a/src/Interpreters/MetricLog.cpp b/src/Interpreters/MetricLog.cpp index 6ed29cfadcb..cc4cb4957ac 100644 --- a/src/Interpreters/MetricLog.cpp +++ b/src/Interpreters/MetricLog.cpp @@ -56,78 +56,32 @@ void MetricLogElement::appendToBlock(MutableColumns & columns) const columns[column_idx++]->insert(current_metrics[i].toUnderType()); } - -void MetricLog::startCollectMetric(size_t collect_interval_milliseconds_) +void MetricLog::stepFunction(const std::chrono::system_clock::time_point current_time) { - collect_interval_milliseconds = collect_interval_milliseconds_; - is_shutdown_metric_thread = false; - metric_flush_thread = std::make_unique([this] { metricThreadFunction(); }); -} - - -void MetricLog::stopCollectMetric() -{ - bool old_val = false; - if (!is_shutdown_metric_thread.compare_exchange_strong(old_val, true)) - return; - if (metric_flush_thread) - metric_flush_thread->join(); -} - - -void MetricLog::shutdown() -{ - stopCollectMetric(); - stopFlushThread(); -} - - -void MetricLog::metricThreadFunction() -{ - auto desired_timepoint = std::chrono::system_clock::now(); - + /// Static lazy initialization to avoid polluting the header with implementation details /// For differentiation of ProfileEvents counters. - std::vector prev_profile_events(ProfileEvents::end()); + static std::vector prev_profile_events(ProfileEvents::end()); - while (!is_shutdown_metric_thread) + MetricLogElement elem; + elem.event_time = std::chrono::system_clock::to_time_t(current_time); + elem.event_time_microseconds = timeInMicroseconds(current_time); + + elem.profile_events.resize(ProfileEvents::end()); + for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) { - try - { - const auto current_time = std::chrono::system_clock::now(); - - MetricLogElement elem; - elem.event_time = std::chrono::system_clock::to_time_t(current_time); - elem.event_time_microseconds = timeInMicroseconds(current_time); - - elem.profile_events.resize(ProfileEvents::end()); - for (ProfileEvents::Event i = ProfileEvents::Event(0), end = ProfileEvents::end(); i < end; ++i) - { - const ProfileEvents::Count new_value = ProfileEvents::global_counters[i].load(std::memory_order_relaxed); - auto & old_value = prev_profile_events[i]; - elem.profile_events[i] = new_value - old_value; - old_value = new_value; - } - - elem.current_metrics.resize(CurrentMetrics::end()); - for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) - { - elem.current_metrics[i] = CurrentMetrics::values[i]; - } - - this->add(std::move(elem)); - - /// We will record current time into table but align it to regular time intervals to avoid time drift. - /// We may drop some time points if the server is overloaded and recording took too much time. - while (desired_timepoint <= current_time) - desired_timepoint += std::chrono::milliseconds(collect_interval_milliseconds); - - std::this_thread::sleep_until(desired_timepoint); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + const ProfileEvents::Count new_value = ProfileEvents::global_counters[i].load(std::memory_order_relaxed); + auto & old_value = prev_profile_events[i]; + elem.profile_events[i] = new_value - old_value; + old_value = new_value; } + + elem.current_metrics.resize(CurrentMetrics::end()); + for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) + { + elem.current_metrics[i] = CurrentMetrics::values[i]; + } + + this->add(std::move(elem)); } } diff --git a/src/Interpreters/MetricLog.h b/src/Interpreters/MetricLog.h index 3a079792bc8..a6fd3ecfcd3 100644 --- a/src/Interpreters/MetricLog.h +++ b/src/Interpreters/MetricLog.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -9,7 +10,6 @@ #include #include -#include #include @@ -33,26 +33,12 @@ struct MetricLogElement void appendToBlock(MutableColumns & columns) const; }; - -class MetricLog : public SystemLog +class MetricLog : public PeriodicLog { - using SystemLog::SystemLog; + using PeriodicLog::PeriodicLog; -public: - void shutdown() override; - - /// Launches a background thread to collect metrics with interval - void startCollectMetric(size_t collect_interval_milliseconds_); - - /// Stop background thread - void stopCollectMetric(); - -private: - void metricThreadFunction(); - - std::unique_ptr metric_flush_thread; - size_t collect_interval_milliseconds; - std::atomic is_shutdown_metric_thread{false}; +protected: + void stepFunction(TimePoint current_time) override; }; } diff --git a/src/Interpreters/PeriodicLog.cpp b/src/Interpreters/PeriodicLog.cpp new file mode 100644 index 00000000000..877dabb06f0 --- /dev/null +++ b/src/Interpreters/PeriodicLog.cpp @@ -0,0 +1,64 @@ +#include +#include +#include + +namespace DB +{ + + + +template +void PeriodicLog::startCollect(size_t collect_interval_milliseconds_) +{ + collect_interval_milliseconds = collect_interval_milliseconds_; + is_shutdown_metric_thread = false; + flush_thread = std::make_unique([this] { threadFunction(); }); +} + +template +void PeriodicLog::stopCollect() +{ + bool old_val = false; + if (!is_shutdown_metric_thread.compare_exchange_strong(old_val, true)) + return; + if (flush_thread) + flush_thread->join(); +} + +template +void PeriodicLog::shutdown() +{ + stopCollect(); + this->stopFlushThread(); +} + +template +void PeriodicLog::threadFunction() +{ + auto desired_timepoint = std::chrono::system_clock::now(); + while (!is_shutdown_metric_thread) + { + try + { + const auto current_time = std::chrono::system_clock::now(); + + stepFunction(current_time); + + /// We will record current time into table but align it to regular time intervals to avoid time drift. + /// We may drop some time points if the server is overloaded and recording took too much time. + while (desired_timepoint <= current_time) + desired_timepoint += std::chrono::milliseconds(collect_interval_milliseconds); + + std::this_thread::sleep_until(desired_timepoint); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } +} + +#define INSTANTIATE_SYSTEM_LOG(ELEMENT) template class PeriodicLog; +SYSTEM_PERIODIC_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG) + +} diff --git a/src/Interpreters/PeriodicLog.h b/src/Interpreters/PeriodicLog.h new file mode 100644 index 00000000000..12277a32226 --- /dev/null +++ b/src/Interpreters/PeriodicLog.h @@ -0,0 +1,44 @@ + +#pragma once + +#include +#include + +#include +#include + +#define SYSTEM_PERIODIC_LOG_ELEMENTS(M) \ + M(ErrorLogElement) \ + M(MetricLogElement) + +namespace DB +{ + +template +class PeriodicLog : public SystemLog +{ + using SystemLog::SystemLog; + +public: + using TimePoint = std::chrono::system_clock::time_point; + + /// Launches a background thread to collect metrics with interval + void startCollect(size_t collect_interval_milliseconds_); + + /// Stop background thread + void stopCollect(); + + void shutdown() final; + +protected: + virtual void stepFunction(TimePoint current_time) = 0; + +private: + void threadFunction(); + + std::unique_ptr flush_thread; + size_t collect_interval_milliseconds; + std::atomic is_shutdown_metric_thread{false}; +}; + +} diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index 8b0f1239da5..7508d2a7e34 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -371,14 +371,14 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf { size_t collect_interval_milliseconds = config.getUInt64("metric_log.collect_interval_milliseconds", DEFAULT_METRIC_LOG_COLLECT_INTERVAL_MILLISECONDS); - metric_log->startCollectMetric(collect_interval_milliseconds); + metric_log->startCollect(collect_interval_milliseconds); } if (error_log) { size_t collect_interval_milliseconds = config.getUInt64("error_log.collect_interval_milliseconds", DEFAULT_ERROR_LOG_COLLECT_INTERVAL_MILLISECONDS); - error_log->startCollectError(collect_interval_milliseconds); + error_log->startCollect(collect_interval_milliseconds); } if (crash_log) From 978458015e54eb1f5a63f2d1ac87ed6e270caace Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 20 Jun 2024 16:29:37 +0000 Subject: [PATCH 284/318] Fix coding style --- src/Interpreters/MetricLog.cpp | 2 +- src/Interpreters/PeriodicLog.cpp | 2 -- src/Interpreters/PeriodicLog.h | 1 - 3 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Interpreters/MetricLog.cpp b/src/Interpreters/MetricLog.cpp index cc4cb4957ac..596b0e4f96c 100644 --- a/src/Interpreters/MetricLog.cpp +++ b/src/Interpreters/MetricLog.cpp @@ -56,7 +56,7 @@ void MetricLogElement::appendToBlock(MutableColumns & columns) const columns[column_idx++]->insert(current_metrics[i].toUnderType()); } -void MetricLog::stepFunction(const std::chrono::system_clock::time_point current_time) +void MetricLog::stepFunction(const std::chrono::system_clock::time_point current_time) { /// Static lazy initialization to avoid polluting the header with implementation details /// For differentiation of ProfileEvents counters. diff --git a/src/Interpreters/PeriodicLog.cpp b/src/Interpreters/PeriodicLog.cpp index 877dabb06f0..9d2891e11eb 100644 --- a/src/Interpreters/PeriodicLog.cpp +++ b/src/Interpreters/PeriodicLog.cpp @@ -5,8 +5,6 @@ namespace DB { - - template void PeriodicLog::startCollect(size_t collect_interval_milliseconds_) { diff --git a/src/Interpreters/PeriodicLog.h b/src/Interpreters/PeriodicLog.h index 12277a32226..08c3f7eb23f 100644 --- a/src/Interpreters/PeriodicLog.h +++ b/src/Interpreters/PeriodicLog.h @@ -1,4 +1,3 @@ - #pragma once #include From 8b54e4b59386823ebccf5bc736c9f69fedcbad73 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 20 Jun 2024 18:01:07 +0000 Subject: [PATCH 285/318] Changelog improvements --- tests/ci/changelog.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index fcb61d3f605..d7a46d0a556 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -35,8 +35,8 @@ categories_preferred_order = ( "New Feature", "Performance Improvement", "Improvement", - "Critical Bug Fix", - "Bug Fix", + "Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)", + "Bug Fix (user-visible misbehavior in an official stable release)", "Build/Testing/Packaging Improvement", "Other", ) @@ -205,7 +205,7 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri try: item = gh.get_pull_cached(repo, int(branch_parts[-1])) except Exception as e: - logging.warning("unable to get backpoted PR, exception: %s", e) + logging.warning("unable to get backported PR, exception: %s", e) else: logging.warning( "The branch %s doesn't match backport template, using PR %s as is", @@ -280,13 +280,15 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri category, ): category = "NOT FOR CHANGELOG / INSIGNIFICANT" - entry = item.title + # Sometimes we declare not for changelog but still write a description. Keep it + if len(entry) <= 4 or "Documentation entry" in entry: + entry = item.title # Normalize bug fixes if re.match( r"(?i)bug\Wfix", category, - ): + ) and "Critical Bug Fix" not in category: category = "Bug Fix (user-visible misbehavior in an official stable release)" if backport_number != item.number: From 991aa0517b685eac74e5dc5055a706bf4a86502e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 20 Jun 2024 18:01:40 +0000 Subject: [PATCH 286/318] Update 24.6 changelog --- CHANGELOG.md | 295 +++++++++++++++++++++++++-------------------------- 1 file changed, 145 insertions(+), 150 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3965014707f..ce6b46f70d6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -13,66 +13,67 @@ #### Backward Incompatible Change * Some invalid queries will fail earlier during parsing. Note: disabled the support for inline KQL expressions (the experimental Kusto language) when they are put into a `kql` table function without a string literal, e.g. `kql(garbage | trash)` instead of `kql('garbage | trash')` or `kql($$garbage | trash$$)`. This feature was introduced unintentionally and should not exist. [#61500](https://github.com/ClickHouse/ClickHouse/pull/61500) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Rework parallel processing in `Ordered` mode of storage `S3Queue`. This PR is backward incompatible for Ordered mode if you used settings `s3queue_processing_threads_num` or `s3queue_total_shards_num`. Setting `s3queue_total_shards_num` is deleted, previously it was allowed to use only under `s3queue_allow_experimental_sharded_mode`, which is not deprecated. A new setting is added - `s3queue_buckets`. [#64349](https://github.com/ClickHouse/ClickHouse/pull/64349) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Rework parallel processing in `Ordered` mode of storage `S3Queue`. This PR is backward incompatible for Ordered mode if you used settings `s3queue_processing_threads_num` or `s3queue_total_shards_num`. Setting `s3queue_total_shards_num` is deleted, previously it was allowed to use only under `s3queue_allow_experimental_sharded_mode`, which is now deprecated. A new setting is added - `s3queue_buckets`. [#64349](https://github.com/ClickHouse/ClickHouse/pull/64349) ([Kseniia Sumarokova](https://github.com/kssenii)). * New functions `snowflakeIDToDateTime`, `snowflakeIDToDateTime64`, `dateTimeToSnowflakeID`, and `dateTime64ToSnowflakeID` were added. Unlike the existing functions `snowflakeToDateTime`, `snowflakeToDateTime64`, `dateTimeToSnowflake`, and `dateTime64ToSnowflake`, the new functions are compatible with function `generateSnowflakeID`, i.e. they accept the snowflake IDs generated by `generateSnowflakeID` and produce snowflake IDs of the same type as `generateSnowflakeID` (i.e. `UInt64`). Furthermore, the new functions default to the UNIX epoch (aka. 1970-01-01), just like `generateSnowflakeID`. If necessary, a different epoch, e.g. Twitter's/X's epoch 2010-11-04 aka. 1288834974657 msec since UNIX epoch, can be passed. The old conversion functions are deprecated and will be removed after a transition period: to use them regardless, disable setting `uniform_snowflake_conversion_functions` (default: `true`). [#64948](https://github.com/ClickHouse/ClickHouse/pull/64948) ([Robert Schulze](https://github.com/rschu1ze)). #### New Feature * Introduce statistics of type "number of distinct values". [#59357](https://github.com/ClickHouse/ClickHouse/pull/59357) ([Han Fei](https://github.com/hanfei1991)). -* Adds Hilbert Curve encode and decode functions - https://en.wikipedia.org/wiki/Hilbert_curve. Closes [#55520](https://github.com/ClickHouse/ClickHouse/issues/55520). [#60156](https://github.com/ClickHouse/ClickHouse/pull/60156) ([Artem Mustafin](https://github.com/Artemmm91)). +* Add Hilbert Curve encode and decode functions. [#60156](https://github.com/ClickHouse/ClickHouse/pull/60156) ([Artem Mustafin](https://github.com/Artemmm91)). * Added support for reading LINESTRING geometry in WKT format using function `readWKTLineString`. [#62519](https://github.com/ClickHouse/ClickHouse/pull/62519) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Implement Dynamic data type that allows to store values of any type inside it without knowing all of them in advance. Dynamic type is available under a setting `allow_experimental_dynamic_type`. Reference: [#54864](https://github.com/ClickHouse/ClickHouse/issues/54864). [#63058](https://github.com/ClickHouse/ClickHouse/pull/63058) ([Kruglov Pavel](https://github.com/Avogar)). -* Allow to attach parts from a different disk * attach partition from the table on other disks using copy instead of hard link (such as instant table) * attach partition using copy when the hard link fails even on the same disk. [#63087](https://github.com/ClickHouse/ClickHouse/pull/63087) ([Unalian](https://github.com/Unalian)). +* Implement Dynamic data type that allows to store values of any type inside it without knowing all of them in advance. Dynamic type is available under a setting `allow_experimental_dynamic_type`. [#63058](https://github.com/ClickHouse/ClickHouse/pull/63058) ([Kruglov Pavel](https://github.com/Avogar)). +* Allow to attach parts from a different disk. [#63087](https://github.com/ClickHouse/ClickHouse/pull/63087) ([Unalian](https://github.com/Unalian)). * Allow proxy to be bypassed for hosts specified in `no_proxy` env variable and ClickHouse proxy configuration. [#63314](https://github.com/ClickHouse/ClickHouse/pull/63314) ([Arthur Passos](https://github.com/arthurpassos)). * Added a new table function `loop` to support returning query results in an infinite loop. [#63452](https://github.com/ClickHouse/ClickHouse/pull/63452) ([Sariel](https://github.com/sarielwxm)). * Added new SQL functions `generateSnowflakeID` for generating Twitter-style Snowflake IDs. [#63577](https://github.com/ClickHouse/ClickHouse/pull/63577) ([Danila Puzov](https://github.com/kazalika)). +* Add the ability to reshuffle rows during insert to optimize for size without violating the order set by `PRIMARY KEY`. It's controlled by the setting `optimize_row_order` (off by default). [#63578](https://github.com/ClickHouse/ClickHouse/pull/63578) ([Igor Markelov](https://github.com/ElderlyPassionFruit)). * Added `merge_workload` and `mutation_workload` settings to regulate how resources are utilized and shared between merges, mutations and other workloads. [#64061](https://github.com/ClickHouse/ClickHouse/pull/64061) ([Sergei Trifonov](https://github.com/serxa)). -* Equality between IPv4 and IPv6 values can now be evaluated using the `=` operator. [#64292](https://github.com/ClickHouse/ClickHouse/pull/64292) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). -* Allow to store named collections in zookeeper. To enable - add `named_collections_storage` section to server configuration file, where add `type` equal to `zookeeper` and `path` equal to whatever zookeeper path you want to use to store named collections there. [#64574](https://github.com/ClickHouse/ClickHouse/pull/64574) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Decimals are supported as arguments for binary math functions (pow(), atan2(), max2, min2(), hypot(). [#64582](https://github.com/ClickHouse/ClickHouse/pull/64582) ([Mikhail Gorshkov](https://github.com/mgorshkov)). -* Adds index analysis for Hilbert curve. If a table has hilbertEncode (aka Hilbert curve) in its index, e.g. `ORDER BY hilbertEncode(x, y)` then queries that use both `x` and `y` can be processed efficient, for example `WHERE 1 <= x AND x <= 2 AND 3 <= y AND y <= 4`. Works with the setting `analyze_index_with_space_filling_curves`. [#64662](https://github.com/ClickHouse/ClickHouse/pull/64662) ([Artem Mustafin](https://github.com/Artemmm91)). -* Added SQL functions `parseReadableSize` (along with `OrNull` and `OrZero` variants). These functions perform the opposite operation of function `formatReadableSize` and `formatReadableDecimalSize`, i.e. the given human-readable byte size, they return the number of bytes. Example: `SELECT formatReadableSize('3.0 MiB')` returns `3145728`. [#64742](https://github.com/ClickHouse/ClickHouse/pull/64742) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). -* Add 2 server_setting to limit numebr of table/db max_table_num_to_throw max_database_num_to_throw. [#64781](https://github.com/ClickHouse/ClickHouse/pull/64781) ([Xu Jia](https://github.com/XuJia0210)). +* Add support for comparing IPv4 and IPv6 types using the `=` operator. [#64292](https://github.com/ClickHouse/ClickHouse/pull/64292) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). +* Allow to store named collections in zookeeper. [#64574](https://github.com/ClickHouse/ClickHouse/pull/64574) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support decimal arguments in binary math functions (pow(), atan2(), max2, min2(), hypot(). [#64582](https://github.com/ClickHouse/ClickHouse/pull/64582) ([Mikhail Gorshkov](https://github.com/mgorshkov)). +* Add support for index analysis over `hilbertEncode`. [#64662](https://github.com/ClickHouse/ClickHouse/pull/64662) ([Artem Mustafin](https://github.com/Artemmm91)). +* Added SQL functions `parseReadableSize` (along with `OrNull` and `OrZero` variants). [#64742](https://github.com/ClickHouse/ClickHouse/pull/64742) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). +* Add server settings `max_table_num_to_throw` and `max_database_num_to_throw` to limit the number of databases or tables on `CREATE` queries. [#64781](https://github.com/ClickHouse/ClickHouse/pull/64781) ([Xu Jia](https://github.com/XuJia0210)). * Add _time virtual column to file alike storages (s3/file/hdfs/url/azureBlobStorage). [#64947](https://github.com/ClickHouse/ClickHouse/pull/64947) ([Ilya Golshtein](https://github.com/ilejn)). * Introduced new functions `base64UrlEncode`, `base64UrlDecode` and `tryBase64UrlDecode`. [#64991](https://github.com/ClickHouse/ClickHouse/pull/64991) ([Mikhail Gorshkov](https://github.com/mgorshkov)). * Add new function `editDistanceUTF8`, which calculates the [edit distance](https://en.wikipedia.org/wiki/Edit_distance) between two UTF8 strings. [#65269](https://github.com/ClickHouse/ClickHouse/pull/65269) ([LiuNeng](https://github.com/liuneng1994)). #### Performance Improvement -* A native parquet reader, which can read parquet binary to ClickHouse Columns directly. Now this feature can be activated by setting `input_format_parquet_use_native_reader` to true. [#60361](https://github.com/ClickHouse/ClickHouse/pull/60361) ([ZhiHong Zhang](https://github.com/copperybean)). -* Reduce useless virtual func call in ColumnNullable of size() func. [#60556](https://github.com/ClickHouse/ClickHouse/pull/60556) ([HappenLee](https://github.com/HappenLee)). -* Reduce lock contention for MergeTree tables (by renaming parts without holding lock). [#61973](https://github.com/ClickHouse/ClickHouse/pull/61973) ([Azat Khuzhin](https://github.com/azat)). -* Function `splitByRegexp` is now faster when the regular expression argument is a single-character, trivial regular expression (in this case, it now falls back internally to `splitByChar`). [#62696](https://github.com/ClickHouse/ClickHouse/pull/62696) ([Robert Schulze](https://github.com/rschu1ze)). -* Add min/max in fixedHashTable to limit the array index and reduce the `isZero()` loop in iterator++. [#62746](https://github.com/ClickHouse/ClickHouse/pull/62746) ([Jiebin Sun](https://github.com/jiebinn)). -* This pull request is the basic solution for [Feature: best effort sorting to optimize compression [#4413](https://github.com/ClickHouse/ClickHouse/issues/4413)](https://github.com/ClickHouse/ClickHouse/issues/4413). [#63578](https://github.com/ClickHouse/ClickHouse/pull/63578) ([Igor Markelov](https://github.com/ElderlyPassionFruit)). -* The queries of 3.3, 3.4, 4.1 and 4.2 from **Star Schema Benchmark** share a common pattern of predicate: `col = a or col = b` where `col` is a `LowCardinality(String)`. And during syntax optimization, this predicate is then transformed equivalently to `col in (a, b)`, which is executed by `FunctionIn` with `col` and a constant tuple `(a, b)` as its arguments. [#64060](https://github.com/ClickHouse/ClickHouse/pull/64060) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). -* Initialisation and destruction of hash tables inside `ConcurrentHashJoin` is executed on a thread pool now. [#64241](https://github.com/ClickHouse/ClickHouse/pull/64241) ([Nikita Taranov](https://github.com/nickitat)). +* Add a native parquet reader, which can read parquet binary to ClickHouse Columns directly. It's controlled by the setting `input_format_parquet_use_native_reader` (disabled by default). [#60361](https://github.com/ClickHouse/ClickHouse/pull/60361) ([ZhiHong Zhang](https://github.com/copperybean)). +* Reduce the number of virtual function calls in ColumnNullable::size(). [#60556](https://github.com/ClickHouse/ClickHouse/pull/60556) ([HappenLee](https://github.com/HappenLee)). +* Speedup `splitByRegexp` when the regular expression argument is a single-character. [#62696](https://github.com/ClickHouse/ClickHouse/pull/62696) ([Robert Schulze](https://github.com/rschu1ze)). +* Speed up FixedHashTable by keeping track of the min and max keys used. This allows to reduce the number of cells that need to be verified. [#62746](https://github.com/ClickHouse/ClickHouse/pull/62746) ([Jiebin Sun](https://github.com/jiebinn)). +* Optimize the resolution of in(LowCardinality, ConstantSet). [#64060](https://github.com/ClickHouse/ClickHouse/pull/64060) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Use a thread pool to initialize and destroy hash tables inside `ConcurrentHashJoin`. [#64241](https://github.com/ClickHouse/ClickHouse/pull/64241) ([Nikita Taranov](https://github.com/nickitat)). * Optimized vertical merges in tables with sparse columns. [#64311](https://github.com/ClickHouse/ClickHouse/pull/64311) ([Anton Popov](https://github.com/CurtizJ)). * Enabled prefetches of data from remote filesystem during vertical merges. It improves latency of vertical merges in tables with data stored on remote filesystem. [#64314](https://github.com/ClickHouse/ClickHouse/pull/64314) ([Anton Popov](https://github.com/CurtizJ)). -* ... Reduce the redundant `isDefault()` of `ColumnSparse::filter` to improve performance. [#64426](https://github.com/ClickHouse/ClickHouse/pull/64426) ([Jiebin Sun](https://github.com/jiebinn)). +* Reduce redundant calls to `isDefault()` of `ColumnSparse::filter` to improve performance. [#64426](https://github.com/ClickHouse/ClickHouse/pull/64426) ([Jiebin Sun](https://github.com/jiebinn)). * Speedup `find_super_nodes` and `find_big_family` keeper-client commands by making multiple asynchronous getChildren requests. [#64628](https://github.com/ClickHouse/ClickHouse/pull/64628) ([Alexander Gololobov](https://github.com/davenger)). +* Improve function least/greatest for nullable numberic type arguments. [#64668](https://github.com/ClickHouse/ClickHouse/pull/64668) ([KevinyhZou](https://github.com/KevinyhZou)). * Allow merging two consequent `FilterSteps` of a query plan. This improves filter-push-down optimization if the filter condition can be pushed down from the parent step. [#64760](https://github.com/ClickHouse/ClickHouse/pull/64760) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Remove bad optimization in vertical final implementation and re-enable vertical final algorithm by default. [#64783](https://github.com/ClickHouse/ClickHouse/pull/64783) ([Duc Canh Le](https://github.com/canhld94)). * Remove ALIAS nodes from the filter expression. This slightly improves performance for queries with `PREWHERE` (with new analyzer). [#64793](https://github.com/ClickHouse/ClickHouse/pull/64793) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix performance regression in cross join introduced in [#60459](https://github.com/ClickHouse/ClickHouse/issues/60459) (24.5). [#65243](https://github.com/ClickHouse/ClickHouse/pull/65243) ([Nikita Taranov](https://github.com/nickitat)). #### Improvement -* Now, empty tuples can be stored, queried, and inserted just like other columns. The implementation is different from [#55021](https://github.com/ClickHouse/ClickHouse/issues/55021). It specializes all related methods of `ColumnTuple`, `DataTypeTuple`, and `SerializationTuple` for empty tuples, which is more complicated but fundamentally correct. [#55061](https://github.com/ClickHouse/ClickHouse/pull/55061) ([Amos Bird](https://github.com/amosbird)). +* Support empty tuples. [#55061](https://github.com/ClickHouse/ClickHouse/pull/55061) ([Amos Bird](https://github.com/amosbird)). * Enable asynchronous load of databases and tables by default. See the `async_load_databases` in config.xml. [#57695](https://github.com/ClickHouse/ClickHouse/pull/57695) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Hot reload storage policy for distributed tables when adding a new disk. [#58285](https://github.com/ClickHouse/ClickHouse/pull/58285) ([Duc Canh Le](https://github.com/canhld94)). -* ... Use scheduleOrThrow in MergeTree data selection and initialization to avoid deadlocks. [#59427](https://github.com/ClickHouse/ClickHouse/pull/59427) ([Sean Haynes](https://github.com/seandhaynes)). -* Support partial trivial count optimization when query filter is able to select exact ranges from merge tree tables. This implements a more general version of [#36732](https://github.com/ClickHouse/ClickHouse/issues/36732). [#60463](https://github.com/ClickHouse/ClickHouse/pull/60463) ([Amos Bird](https://github.com/amosbird)). -* This implements new transform `PlanSquashingTransform` which provides improvements for insert process and availability to check memory during inserts. Ref [#59112](https://github.com/ClickHouse/ClickHouse/issues/59112). [#61047](https://github.com/ClickHouse/ClickHouse/pull/61047) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). -* Fixed memory allocation by avoiding last extra buffer allocation. [#63160](https://github.com/ClickHouse/ClickHouse/pull/63160) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). -* Few corner case fixes to proxy support & tunneling. Adjust tests accordingly.. [#63427](https://github.com/ClickHouse/ClickHouse/pull/63427) ([Arthur Passos](https://github.com/arthurpassos)). -* For HTTP Handlers rules add `http_response_headers` setting to support custom response headers. [#63562](https://github.com/ClickHouse/ClickHouse/pull/63562) ([Grigorii](https://github.com/GSokol)). +* Avoid possible deadlock during MergeTree index analysis when scheduling threads in a saturated service. [#59427](https://github.com/ClickHouse/ClickHouse/pull/59427) ([Sean Haynes](https://github.com/seandhaynes)). +* Support partial trivial count optimization when the query filter is able to select exact ranges from merge tree tables. [#60463](https://github.com/ClickHouse/ClickHouse/pull/60463) ([Amos Bird](https://github.com/amosbird)). +* Reduce max memory usage of multithreaded `INSERT`s by collecting chunks of multiple threads in a single transform. [#61047](https://github.com/ClickHouse/ClickHouse/pull/61047) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* Reduce the memory usage when using Azure object storage by using fixed memory allocation, avoiding the allocation of an extra buffer. [#63160](https://github.com/ClickHouse/ClickHouse/pull/63160) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Several minor corner case fixes to proxy support & tunneling. [#63427](https://github.com/ClickHouse/ClickHouse/pull/63427) ([Arthur Passos](https://github.com/arthurpassos)). +* Add `http_response_headers` setting to support custom response headers in custom HTTP handlers. [#63562](https://github.com/ClickHouse/ClickHouse/pull/63562) ([Grigorii](https://github.com/GSokol)). * Improve io_uring resubmits visibility. Rename profile event `IOUringSQEsResubmits` -> `IOUringSQEsResubmitsAsync` and add a new one `IOUringSQEsResubmitsSync`. [#63699](https://github.com/ClickHouse/ClickHouse/pull/63699) ([Tomer Shafir](https://github.com/tomershafir)). * Introduce assertions to verify all functions are called with columns of the right size. [#63723](https://github.com/ClickHouse/ClickHouse/pull/63723) ([Raúl Marín](https://github.com/Algunenano)). * `SHOW CREATE TABLE` executed on top of system tables will now show the super handy comment unique for each table which will explain why this table is needed. [#63788](https://github.com/ClickHouse/ClickHouse/pull/63788) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Added knob `metadata_storage_type` to keep free space on metadata storage disk. [#64128](https://github.com/ClickHouse/ClickHouse/pull/64128) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). -* Metrics to track the number of directories created and removed by the plain_rewritable metadata storage, and the number of entries in the local-to-remote in-memory map. [#64175](https://github.com/ClickHouse/ClickHouse/pull/64175) ([Julia Kartseva](https://github.com/jkartseva)). +* Added setting `metadata_storage_type` to keep free space on metadata storage disk. [#64128](https://github.com/ClickHouse/ClickHouse/pull/64128) ([MikhailBurdukov](https://github.com/MikhailBurdukov)). +* Add metrics to track the number of directories created and removed by the plain_rewritable metadata storage, and the number of entries in the local-to-remote in-memory map. [#64175](https://github.com/ClickHouse/ClickHouse/pull/64175) ([Julia Kartseva](https://github.com/jkartseva)). * The query cache now considers identical queries with different settings as different. This increases robustness in cases where different settings (e.g. `limit` or `additional_table_filters`) would affect the query result. [#64205](https://github.com/ClickHouse/ClickHouse/pull/64205) ([Robert Schulze](https://github.com/rschu1ze)). * Better Exception Message in Delete Table with Projection, users can understand the error and the steps should be taken. [#64212](https://github.com/ClickHouse/ClickHouse/pull/64212) ([jsc0218](https://github.com/jsc0218)). -* Test that non standard error code `QpsLimitExceeded` is supported and it is retryable error. [#64225](https://github.com/ClickHouse/ClickHouse/pull/64225) ([Sema Checherinda](https://github.com/CheSema)). -* It is forbidden to convert MergeTree to replicated if the zookeeper path for this table already exists. [#64244](https://github.com/ClickHouse/ClickHouse/pull/64244) ([Kirill](https://github.com/kirillgarbar)). -* If "replica group" is configured for a `Replicated` database, automatically create a cluster that includes replicas from all groups. Closes [#62725](https://github.com/ClickHouse/ClickHouse/issues/62725). [#64312](https://github.com/ClickHouse/ClickHouse/pull/64312) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Support the non standard error code `QpsLimitExceeded` in object storage as a retryable error. [#64225](https://github.com/ClickHouse/ClickHouse/pull/64225) ([Sema Checherinda](https://github.com/CheSema)). +* Forbid converting a MergeTree table to replicated if the zookeeper path for this table already exists. [#64244](https://github.com/ClickHouse/ClickHouse/pull/64244) ([Kirill](https://github.com/kirillgarbar)). +* If "replica group" is configured for a `Replicated` database, automatically create a cluster that includes replicas from all groups. [#64312](https://github.com/ClickHouse/ClickHouse/pull/64312) ([Alexander Tokmakov](https://github.com/tavplubix)). * Added settings to disable materialization of skip indexes and statistics on inserts (`materialize_skip_indexes_on_insert` and `materialize_statistics_on_insert`). [#64391](https://github.com/ClickHouse/ClickHouse/pull/64391) ([Anton Popov](https://github.com/CurtizJ)). * Use the allocated memory size to calculate the row group size and reduce the peak memory of the parquet writer in single-threaded mode. [#64424](https://github.com/ClickHouse/ClickHouse/pull/64424) ([LiuNeng](https://github.com/liuneng1994)). * Added new configuration input_format_parquet_prefer_block_bytes to control the average output block bytes, and modified the default value of input_format_parquet_max_block_size to 65409. [#64427](https://github.com/ClickHouse/ClickHouse/pull/64427) ([LiuNeng](https://github.com/liuneng1994)). @@ -81,26 +82,85 @@ * Setting `replace_long_file_name_to_hash` is enabled by default for `MergeTree` tables. [#64457](https://github.com/ClickHouse/ClickHouse/pull/64457) ([Anton Popov](https://github.com/CurtizJ)). * Improve the iterator of sparse column to reduce call of size(). [#64497](https://github.com/ClickHouse/ClickHouse/pull/64497) ([Jiebin Sun](https://github.com/jiebinn)). * Update condition to use copy for azure blob storage. [#64518](https://github.com/ClickHouse/ClickHouse/pull/64518) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). -* Test that totalqpslimitexceeded is retriable s3 error. [#64520](https://github.com/ClickHouse/ClickHouse/pull/64520) ([Sema Checherinda](https://github.com/CheSema)). +* Support the non standard error code `TotalQpsLimitExceeded` in object storage as a retryable error. [#64520](https://github.com/ClickHouse/ClickHouse/pull/64520) ([Sema Checherinda](https://github.com/CheSema)). * Optimized memory usage of vertical merges for tables with high number of skip indexes. [#64580](https://github.com/ClickHouse/ClickHouse/pull/64580) ([Anton Popov](https://github.com/CurtizJ)). * 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. [#64597](https://github.com/ClickHouse/ClickHouse/pull/64597) ([Alexey Katsman](https://github.com/alexkats)). * Add settings `parallel_replicas_custom_key_range_lower` and `parallel_replicas_custom_key_range_upper` to control how parallel replicas with dynamic shards parallelizes queries when using a range filter. [#64604](https://github.com/ClickHouse/ClickHouse/pull/64604) ([josh-hildred](https://github.com/josh-hildred)). * Updated Advanced Dashboard for both open-source and ClickHouse Cloud versions to include a chart for 'Maximum concurrent network connections'. [#64610](https://github.com/ClickHouse/ClickHouse/pull/64610) ([Thom O'Connor](https://github.com/thomoco)). -* If the arguments is numberic but nullable, and has 2 arguments, it would use `FunctionLeastGreatestGeneric` to do the comparsion, which is not efficient. and use `GreatestBaseImpl/LeastBaseImpl` for `FunctionBinaryArithmetic` would have better performance. So here we improve these two function by allow this. [#64668](https://github.com/ClickHouse/ClickHouse/pull/64668) ([KevinyhZou](https://github.com/KevinyhZou)). * The second argument (scale) of functions `round()`, `roundBankers()`, `floor()`, `ceil()` and `trunc()` can now be non-const. [#64798](https://github.com/ClickHouse/ClickHouse/pull/64798) ([Mikhail Gorshkov](https://github.com/mgorshkov)). * Improve progress report on zeros_mt and generateRandom. [#64804](https://github.com/ClickHouse/ClickHouse/pull/64804) ([Raúl Marín](https://github.com/Algunenano)). * Add an asynchronous metric jemalloc.profile.active to show whether sampling is currently active. This is an activation mechanism in addition to prof.active; both must be active for the calling thread to sample. [#64842](https://github.com/ClickHouse/ClickHouse/pull/64842) ([Unalian](https://github.com/Unalian)). -* Make statistics work on replicated merge tree. [#64934](https://github.com/ClickHouse/ClickHouse/pull/64934) ([Han Fei](https://github.com/hanfei1991)). -* The setting `allow_experimental_join_condition` was accidentally marked as important which may prevent distributed queries in a mixed versions cluster from being executed successfully. [#65008](https://github.com/ClickHouse/ClickHouse/pull/65008) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* All HTTP buffers are now 1MB instead of 8KB to avoid too frequent send syscalls. [#65028](https://github.com/ClickHouse/ClickHouse/pull/65028) ([Sergei Trifonov](https://github.com/serxa)). +* Support statistics with ReplicatedMergeTree. [#64934](https://github.com/ClickHouse/ClickHouse/pull/64934) ([Han Fei](https://github.com/hanfei1991)). +* Remove mark of `allow_experimental_join_condition` as important. This mark may have prevented distributed queries in a mixed versions cluster from being executed successfully. [#65008](https://github.com/ClickHouse/ClickHouse/pull/65008) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Added server Asynchronous metrics `DiskGetObjectThrottler*` and `DiskGetObjectThrottler*` reflecting request per second rate limit defined with `s3_max_get_rps` and `s3_max_put_rps` disk settings and currently available number of requests that could be sent without hitting throttling limit on the disk. Metrics are defined for every disk that has a configured limit. [#65050](https://github.com/ClickHouse/ClickHouse/pull/65050) ([Sergei Trifonov](https://github.com/serxa)). * Added a setting `output_format_pretty_display_footer_column_names` which when enabled displays column names at the end of the table for long tables (50 rows by default), with the threshold value for minimum number of rows controlled by `output_format_pretty_display_footer_column_names_min_rows`. [#65144](https://github.com/ClickHouse/ClickHouse/pull/65144) ([Shaun Struwig](https://github.com/Blargian)). * Returned back the behaviour of how ClickHouse works and interprets Tuples in CSV format. This change effectively reverts https://github.com/ClickHouse/ClickHouse/pull/60994 and makes it available only under a few settings: `output_format_csv_serialize_tuple_into_separate_columns`, `input_format_csv_deserialize_separate_columns_into_tuple` and `input_format_csv_try_infer_strings_from_quoted_tuples`. [#65170](https://github.com/ClickHouse/ClickHouse/pull/65170) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Initialize global trace collector for Poco::ThreadPool (needed for keeper, etc). [#65239](https://github.com/ClickHouse/ClickHouse/pull/65239) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add validation when creating a user with bcrypt_hash. [#65242](https://github.com/ClickHouse/ClickHouse/pull/65242) ([Raúl Marín](https://github.com/Algunenano)). -* Fix performance regression introduced in [#60459](https://github.com/ClickHouse/ClickHouse/issues/60459). [#65243](https://github.com/ClickHouse/ClickHouse/pull/65243) ([Nikita Taranov](https://github.com/nickitat)). + +#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +* Fix a permission error where a user in a specific situation can escalate their privileges on the default database without necessary grants. [#64769](https://github.com/ClickHouse/ClickHouse/pull/64769) ([pufit](https://github.com/pufit)). +* Fix crash with UniqInjectiveFunctionsEliminationPass and uniqCombined. [#65188](https://github.com/ClickHouse/ClickHouse/pull/65188) ([Raúl Marín](https://github.com/Algunenano)). +* Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). +* Forbid `QUALIFY` clause in the old analyzer. The old analyzer ignored `QUALIFY`, so it could lead to unexpected data removal in mutations. [#65356](https://github.com/ClickHouse/ClickHouse/pull/65356) ([Dmitry Novik](https://github.com/novikd)). +* Use correct memory alignment for Distinct combinator. Previously, crash could happen because of invalid memory allocation when the combinator was used. [#65379](https://github.com/ClickHouse/ClickHouse/pull/65379) ([Antonio Andelic](https://github.com/antonio2368)). + +#### Bug Fix (user-visible misbehavior in an official stable release) +* Fixed 'set' skip index not working with IN and indexHint(). [#62083](https://github.com/ClickHouse/ClickHouse/pull/62083) ([Michael Kolupaev](https://github.com/al13n321)). +* Support executing function during assignment of parameterized view value. [#63502](https://github.com/ClickHouse/ClickHouse/pull/63502) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fixed parquet memory tracking. [#63584](https://github.com/ClickHouse/ClickHouse/pull/63584) ([Michael Kolupaev](https://github.com/al13n321)). +* Fixed reading of columns of type `Tuple(Map(LowCardinality(String), String), ...)`. [#63956](https://github.com/ClickHouse/ClickHouse/pull/63956) ([Anton Popov](https://github.com/CurtizJ)). +* Fix an `Cyclic aliases` error for cyclic aliases of different type (expression and function). [#63993](https://github.com/ClickHouse/ClickHouse/pull/63993) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* This fix will use a proper redefined context with the correct definer for each individual view in the query pipeline. [#64079](https://github.com/ClickHouse/ClickHouse/pull/64079) ([pufit](https://github.com/pufit)). +* Fix analyzer: "Not found column" error is fixed when using INTERPOLATE. [#64096](https://github.com/ClickHouse/ClickHouse/pull/64096) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). +* Fix creating backups to S3 buckets with different credentials from the disk containing the file. [#64153](https://github.com/ClickHouse/ClickHouse/pull/64153) ([Antonio Andelic](https://github.com/antonio2368)). +* The query cache now considers two identical queries against different databases as different. The previous behavior could be used to bypass missing privileges to read from a table. [#64199](https://github.com/ClickHouse/ClickHouse/pull/64199) ([Robert Schulze](https://github.com/rschu1ze)). +* Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix `duplicate alias` error for distributed queries with `ARRAY JOIN`. [#64226](https://github.com/ClickHouse/ClickHouse/pull/64226) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix unexpected accurateCast from string to integer. [#64255](https://github.com/ClickHouse/ClickHouse/pull/64255) ([wudidapaopao](https://github.com/wudidapaopao)). +* Fixed CNF simplification, in case any OR group contains mutually exclusive atoms. [#64256](https://github.com/ClickHouse/ClickHouse/pull/64256) ([Eduard Karacharov](https://github.com/korowa)). +* Fix Query Tree size validation. [#64377](https://github.com/ClickHouse/ClickHouse/pull/64377) ([Dmitry Novik](https://github.com/novikd)). +* Fix `Logical error: Bad cast` for `Buffer` table with `PREWHERE`. [#64388](https://github.com/ClickHouse/ClickHouse/pull/64388) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Prevent recursive logging in `blob_storage_log` when it's stored on object storage. [#64393](https://github.com/ClickHouse/ClickHouse/pull/64393) ([vdimir](https://github.com/vdimir)). +* Fixed `CREATE TABLE AS` queries for tables with default expressions. [#64455](https://github.com/ClickHouse/ClickHouse/pull/64455) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed `optimize_read_in_order` behaviour for ORDER BY ... NULLS FIRST / LAST on tables with nullable keys. [#64483](https://github.com/ClickHouse/ClickHouse/pull/64483) ([Eduard Karacharov](https://github.com/korowa)). +* Fix the `Expression nodes list expected 1 projection names` and `Unknown expression or identifier` errors for queries with aliases to `GLOBAL IN.`. [#64517](https://github.com/ClickHouse/ClickHouse/pull/64517) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix an error `Cannot find column` in distributed queries with constant CTE in the `GROUP BY` key. [#64519](https://github.com/ClickHouse/ClickHouse/pull/64519) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed ORC statistics calculation, when writing, for unsigned types on all platforms and Int8 on ARM. [#64563](https://github.com/ClickHouse/ClickHouse/pull/64563) ([Michael Kolupaev](https://github.com/al13n321)). +* Fix the crash loop when restoring from backup is blocked by creating an MV with a definer that hasn't been restored yet. [#64595](https://github.com/ClickHouse/ClickHouse/pull/64595) ([pufit](https://github.com/pufit)). +* Fix the output of function `formatDateTimeInJodaSyntax` when a formatter generates an uneven number of characters and the last character is `0`. For example, `SELECT formatDateTimeInJodaSyntax(toDate('2012-05-29'), 'D')` now correctly returns `150` instead of previously `15`. [#64614](https://github.com/ClickHouse/ClickHouse/pull/64614) ([LiuNeng](https://github.com/liuneng1994)). +* Do not rewrite aggregation if `-If` combinator is already used. [#64638](https://github.com/ClickHouse/ClickHouse/pull/64638) ([Dmitry Novik](https://github.com/novikd)). +* Fix type inference for float (in case of small buffer, i.e. `--max_read_buffer_size 1`). [#64641](https://github.com/ClickHouse/ClickHouse/pull/64641) ([Azat Khuzhin](https://github.com/azat)). +* Fix bug which could lead to non-working TTLs with expressions. [#64694](https://github.com/ClickHouse/ClickHouse/pull/64694) ([alesapin](https://github.com/alesapin)). +* Fix removing the `WHERE` and `PREWHERE` expressions, which are always true (for the new analyzer). [#64695](https://github.com/ClickHouse/ClickHouse/pull/64695) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed excessive part elimination by token-based text indexes (`ngrambf` , `full_text`) when filtering by result of `startsWith`, `endsWith`, `match`, `multiSearchAny`. [#64720](https://github.com/ClickHouse/ClickHouse/pull/64720) ([Eduard Karacharov](https://github.com/korowa)). +* Fixes incorrect behaviour of ANSI CSI escaping in the `UTF8::computeWidth` function. [#64756](https://github.com/ClickHouse/ClickHouse/pull/64756) ([Shaun Struwig](https://github.com/Blargian)). +* Fix a case of incorrect removal of `ORDER BY` / `LIMIT BY` across subqueries. [#64766](https://github.com/ClickHouse/ClickHouse/pull/64766) ([Raúl Marín](https://github.com/Algunenano)). +* Fix (experimental) unequal join with subqueries for sets which are in the mixed join conditions. [#64775](https://github.com/ClickHouse/ClickHouse/pull/64775) ([lgbo](https://github.com/lgbo-ustc)). +* Fix crash in a local cache over `plain_rewritable` disk. [#64778](https://github.com/ClickHouse/ClickHouse/pull/64778) ([Julia Kartseva](https://github.com/jkartseva)). +* Keeper fix: return correct value for `zk_latest_snapshot_size` in `mntr` command. [#64784](https://github.com/ClickHouse/ClickHouse/pull/64784) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix `Cannot find column` in distributed query with `ARRAY JOIN` by `Nested` column. Fixes [#64755](https://github.com/ClickHouse/ClickHouse/issues/64755). [#64801](https://github.com/ClickHouse/ClickHouse/pull/64801) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix memory leak in slru cache policy. [#64803](https://github.com/ClickHouse/ClickHouse/pull/64803) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed possible incorrect memory tracking in several kinds of queries: queries that read any data from S3, queries via http protocol, asynchronous inserts. [#64844](https://github.com/ClickHouse/ClickHouse/pull/64844) ([Anton Popov](https://github.com/CurtizJ)). +* Fix the `Block structure mismatch` error for queries reading with `PREWHERE` from the materialized view when the materialized view has columns of different types than the source table. Fixes [#64611](https://github.com/ClickHouse/ClickHouse/issues/64611). [#64855](https://github.com/ClickHouse/ClickHouse/pull/64855) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix rare crash when table has TTL with subquery + database replicated + parallel replicas + analyzer. It's really rare, but please don't use TTLs with subqueries. [#64858](https://github.com/ClickHouse/ClickHouse/pull/64858) ([alesapin](https://github.com/alesapin)). +* Fix duplicating `Delete` events in `blob_storage_log` in case of large batch to delete. [#64924](https://github.com/ClickHouse/ClickHouse/pull/64924) ([vdimir](https://github.com/vdimir)). +* Fixed `Session moved to another server` error from [Zoo]Keeper that might happen after server startup when the config has includes from [Zoo]Keeper. [#64986](https://github.com/ClickHouse/ClickHouse/pull/64986) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Fix `ALTER MODIFY COMMENT` query that was broken for parameterized VIEWs in https://github.com/ClickHouse/ClickHouse/pull/54211. [#65031](https://github.com/ClickHouse/ClickHouse/pull/65031) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix `host_id` in DatabaseReplicated when `cluster_secure_connection` parameter is enabled. Previously all the connections within the cluster created by DatabaseReplicated were not secure, even if the parameter was enabled. [#65054](https://github.com/ClickHouse/ClickHouse/pull/65054) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fixing the `Not-ready Set` error after the `PREWHERE` optimization for StorageMerge. [#65057](https://github.com/ClickHouse/ClickHouse/pull/65057) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Avoid writing to finalized buffer in File-like storages. [#65063](https://github.com/ClickHouse/ClickHouse/pull/65063) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix possible infinite query duration in case of cyclic aliases. Fixes [#64849](https://github.com/ClickHouse/ClickHouse/issues/64849). [#65081](https://github.com/ClickHouse/ClickHouse/pull/65081) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix the `Unknown expression identifier` error for remote queries with `INTERPOLATE (alias)` (new analyzer). Fixes [#64636](https://github.com/ClickHouse/ClickHouse/issues/64636). [#65090](https://github.com/ClickHouse/ClickHouse/pull/65090) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix pushing arithmetic operations out of aggregation. In the new analyzer, optimization was applied only once. [#65104](https://github.com/ClickHouse/ClickHouse/pull/65104) ([Dmitry Novik](https://github.com/novikd)). +* Fix aggregate function name rewriting in the new analyzer. [#65110](https://github.com/ClickHouse/ClickHouse/pull/65110) ([Dmitry Novik](https://github.com/novikd)). +* Respond with 5xx instead of 200 OK in case of receive timeout while reading (parts of) the request body from the client socket. [#65118](https://github.com/ClickHouse/ClickHouse/pull/65118) ([Julian Maicher](https://github.com/jmaicher)). +* Fix possible crash for hedged requests. [#65206](https://github.com/ClickHouse/ClickHouse/pull/65206) ([Azat Khuzhin](https://github.com/azat)). +* Fix the bug in Hashed and Hashed_Array dictionary short circuit evaluation, which may read uninitialized number, leading to various errors. [#65256](https://github.com/ClickHouse/ClickHouse/pull/65256) ([jsc0218](https://github.com/jsc0218)). +* This PR ensures that the type of the constant(IN operator's second parameter) is always visible during the IN operator's type conversion process. Otherwise, losing type information may cause some conversions to fail, such as the conversion from DateTime to Date. fix ([#64487](https://github.com/ClickHouse/ClickHouse/issues/64487)). [#65315](https://github.com/ClickHouse/ClickHouse/pull/65315) ([pn](https://github.com/chloro-pn)). #### Build/Testing/Packaging Improvement +* Make `network` service be required when using the rc init script to start the ClickHouse server daemon. [#60650](https://github.com/ClickHouse/ClickHouse/pull/60650) ([Chun-Sheng, Li](https://github.com/peter279k)). * Fix typo in test_hdfsCluster_unset_skip_unavailable_shards. The test writes data to unskip_unavailable_shards, but uses skip_unavailable_shards from the previous test. [#64243](https://github.com/ClickHouse/ClickHouse/pull/64243) ([Mikhail Artemenko](https://github.com/Michicosun)). * Reduce the size of some slow tests. [#64387](https://github.com/ClickHouse/ClickHouse/pull/64387) ([Raúl Marín](https://github.com/Algunenano)). * Reduce the size of some slow tests. [#64452](https://github.com/ClickHouse/ClickHouse/pull/64452) ([Raúl Marín](https://github.com/Algunenano)). @@ -114,71 +174,6 @@ * Get rid of custom code in `tests/ci/download_release_packages.py` and `tests/ci/get_previous_release_tag.py` to avoid issues after the https://github.com/ClickHouse/ClickHouse/pull/64759 is merged. [#64848](https://github.com/ClickHouse/ClickHouse/pull/64848) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Decrease the `unit-test` image a few times. [#65102](https://github.com/ClickHouse/ClickHouse/pull/65102) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -#### Bug Fix (user-visible misbehavior in an official stable release) - -* Fixed 'set' skip index not working with IN and indexHint(). [#62083](https://github.com/ClickHouse/ClickHouse/pull/62083) ([Michael Kolupaev](https://github.com/al13n321)). -* Support executing function during assignment of parameterized view value. [#63502](https://github.com/ClickHouse/ClickHouse/pull/63502) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). -* Fixed parquet memory tracking. [#63584](https://github.com/ClickHouse/ClickHouse/pull/63584) ([Michael Kolupaev](https://github.com/al13n321)). -* Fixed reading of columns of type `Tuple(Map(LowCardinality(String), String), ...)`. [#63956](https://github.com/ClickHouse/ClickHouse/pull/63956) ([Anton Popov](https://github.com/CurtizJ)). -* Fix an `Cyclic aliases` error for cyclic aliases of different type (expression and function). Fixes [#63205](https://github.com/ClickHouse/ClickHouse/issues/63205). [#63993](https://github.com/ClickHouse/ClickHouse/pull/63993) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* This fix will use a proper redefined context with the correct definer for each individual view in the query pipeline Closes [#63777](https://github.com/ClickHouse/ClickHouse/issues/63777). [#64079](https://github.com/ClickHouse/ClickHouse/pull/64079) ([pufit](https://github.com/pufit)). -* Fix analyzer: "Not found column" error is fixed when using INTERPOLATE. [#64096](https://github.com/ClickHouse/ClickHouse/pull/64096) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). -* Fix creating backups to S3 buckets with different credentials from the disk containing the file. [#64153](https://github.com/ClickHouse/ClickHouse/pull/64153) ([Antonio Andelic](https://github.com/antonio2368)). -* The query cache now considers two identical queries against different databases as different. The previous behavior could be used to bypass missing privileges to read from a table. [#64199](https://github.com/ClickHouse/ClickHouse/pull/64199) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix possible abort on uncaught exception in ~WriteBufferFromFileDescriptor in StatusFile. [#64206](https://github.com/ClickHouse/ClickHouse/pull/64206) ([Kruglov Pavel](https://github.com/Avogar)). -* Fix `duplicate alias` error for distributed queries with `ARRAY JOIN`. Fixes [#62043](https://github.com/ClickHouse/ClickHouse/issues/62043). [#64226](https://github.com/ClickHouse/ClickHouse/pull/64226) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix unexpected accurateCast from string to integer ``` SELECT accurateCast('1234', 'UInt8') Before: ┌─accurateCast('1234', 'UInt8')─┐ │ 210 │ └───────────────────────────────┘. [#64255](https://github.com/ClickHouse/ClickHouse/pull/64255) ([wudidapaopao](https://github.com/wudidapaopao)). -* Fixed CNF simplification, in case any OR group contains mutually exclusive atoms. [#64256](https://github.com/ClickHouse/ClickHouse/pull/64256) ([Eduard Karacharov](https://github.com/korowa)). -* Fix Query Tree size validation. Closes [#63701](https://github.com/ClickHouse/ClickHouse/issues/63701). [#64377](https://github.com/ClickHouse/ClickHouse/pull/64377) ([Dmitry Novik](https://github.com/novikd)). -* Fix `Logical error: Bad cast` for `Buffer` table with `PREWHERE`. Fixes [#64172](https://github.com/ClickHouse/ClickHouse/issues/64172). [#64388](https://github.com/ClickHouse/ClickHouse/pull/64388) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Prevent recursive logging in `blob_storage_log` when it's stored on object storage. [#64393](https://github.com/ClickHouse/ClickHouse/pull/64393) ([vdimir](https://github.com/vdimir)). -* Fixed `CREATE TABLE AS` queries for tables with default expressions. [#64455](https://github.com/ClickHouse/ClickHouse/pull/64455) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed `optimize_read_in_order` behaviour for ORDER BY ... NULLS FIRST / LAST on tables with nullable keys. [#64483](https://github.com/ClickHouse/ClickHouse/pull/64483) ([Eduard Karacharov](https://github.com/korowa)). -* Fix the `Expression nodes list expected 1 projection names` and `Unknown expression or identifier` errors for queries with aliases to `GLOBAL IN.` Fixes [#64445](https://github.com/ClickHouse/ClickHouse/issues/64445). [#64517](https://github.com/ClickHouse/ClickHouse/pull/64517) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix an error `Cannot find column` in distributed queries with constant CTE in the `GROUP BY` key. [#64519](https://github.com/ClickHouse/ClickHouse/pull/64519) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed ORC statistics calculation, when writing, for unsigned types on all platforms and Int8 on ARM. [#64563](https://github.com/ClickHouse/ClickHouse/pull/64563) ([Michael Kolupaev](https://github.com/al13n321)). -* Fix the crash loop when restoring from backup is blocked by creating an MV with a definer that hasn't been restored yet. [#64595](https://github.com/ClickHouse/ClickHouse/pull/64595) ([pufit](https://github.com/pufit)). -* Fix the output of function `formatDateTimeInJodaSyntax` when a formatter generates an uneven number of characters and the last character is `0`. For example, `SELECT formatDateTimeInJodaSyntax(toDate('2012-05-29'), 'D')` now correctly returns `150` instead of previously `15`. [#64614](https://github.com/ClickHouse/ClickHouse/pull/64614) ([LiuNeng](https://github.com/liuneng1994)). -* Fix [#64612](https://github.com/ClickHouse/ClickHouse/issues/64612). Do not rewrite aggregation if `-If` combinator is already used. [#64638](https://github.com/ClickHouse/ClickHouse/pull/64638) ([Dmitry Novik](https://github.com/novikd)). -* Fix type inference for float (in case of small buffer, i.e. `--max_read_buffer_size 1`). [#64641](https://github.com/ClickHouse/ClickHouse/pull/64641) ([Azat Khuzhin](https://github.com/azat)). -* Fix bug which could lead to non-working TTLs with expressions. Fixes [#63700](https://github.com/ClickHouse/ClickHouse/issues/63700). [#64694](https://github.com/ClickHouse/ClickHouse/pull/64694) ([alesapin](https://github.com/alesapin)). -* Fix removing the `WHERE` and `PREWHERE` expressions, which are always true (for the new analyzer). Fixes [#64575](https://github.com/ClickHouse/ClickHouse/issues/64575). [#64695](https://github.com/ClickHouse/ClickHouse/pull/64695) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix `loop()` table function crashing on empty table name. [#64716](https://github.com/ClickHouse/ClickHouse/pull/64716) ([Michael Kolupaev](https://github.com/al13n321)). -* Fixed excessive part elimination by token-based text indexes (`ngrambf` , `full_text`) when filtering by result of `startsWith`, `endsWith`, `match`, `multiSearchAny`. [#64720](https://github.com/ClickHouse/ClickHouse/pull/64720) ([Eduard Karacharov](https://github.com/korowa)). -* Fixes incorrect behaviour of ANSI CSI escaping in the `UTF8::computeWidth` function. [#64756](https://github.com/ClickHouse/ClickHouse/pull/64756) ([Shaun Struwig](https://github.com/Blargian)). -* Fix OrderByLimitByDuplicateEliminationVisitor across subqueries. [#64766](https://github.com/ClickHouse/ClickHouse/pull/64766) ([Raúl Marín](https://github.com/Algunenano)). -* Fixed issue [#64513](https://github.com/ClickHouse/ClickHouse/issues/64513). Build subqueries for sets which are in the mixed join conditions. [#64775](https://github.com/ClickHouse/ClickHouse/pull/64775) ([lgbo](https://github.com/lgbo-ustc)). -* Fix crash in a local cache over `plain_rewritable` disk. [#64778](https://github.com/ClickHouse/ClickHouse/pull/64778) ([Julia Kartseva](https://github.com/jkartseva)). -* Keeper fix: return correct value for `zk_latest_snapshot_size` in `mntr` command. [#64784](https://github.com/ClickHouse/ClickHouse/pull/64784) ([Antonio Andelic](https://github.com/antonio2368)). -* Fix `Cannot find column` in distributed query with `ARRAY JOIN` by `Nested` column. Fixes [#64755](https://github.com/ClickHouse/ClickHouse/issues/64755). [#64801](https://github.com/ClickHouse/ClickHouse/pull/64801) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix memory leak in slru cache policy. Closes [#64791](https://github.com/ClickHouse/ClickHouse/issues/64791). [#64803](https://github.com/ClickHouse/ClickHouse/pull/64803) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fixed memory possible incorrect memory tracking in several kinds of queries: queries that read any data from S3, queries via http protocol, asynchronous inserts. [#64844](https://github.com/ClickHouse/ClickHouse/pull/64844) ([Anton Popov](https://github.com/CurtizJ)). -* Fix the `Block structure mismatch` error for queries reading with `PREWHERE` from the materialized view when the materialized view has columns of different types than the source table. Fixes [#64611](https://github.com/ClickHouse/ClickHouse/issues/64611). [#64855](https://github.com/ClickHouse/ClickHouse/pull/64855) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix rare crash when table has TTL with subquery + database replicated + parallel replicas + analyzer. It's really rare, but please don't use TTLs with subqueries. [#64858](https://github.com/ClickHouse/ClickHouse/pull/64858) ([alesapin](https://github.com/alesapin)). -* Fix duplicating `Delete` events in `blob_storage_log` in case of large batch to delete. [#64924](https://github.com/ClickHouse/ClickHouse/pull/64924) ([vdimir](https://github.com/vdimir)). -* Fixed `Session moved to another server` error from [Zoo]Keeper that might happen after server startup when the config has includes from [Zoo]Keeper. [#64986](https://github.com/ClickHouse/ClickHouse/pull/64986) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix `ALTER MODIFY COMMENT` query that was broken for parameterized VIEWs in https://github.com/ClickHouse/ClickHouse/pull/54211. [#65031](https://github.com/ClickHouse/ClickHouse/pull/65031) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix `host_id` in DatabaseReplicated when `cluster_secure_connection` parameter is enabled. Previously all the connections within the cluster created by DatabaseReplicated were not secure, even if the parameter was enabled. Closes [#64216](https://github.com/ClickHouse/ClickHouse/issues/64216). [#65054](https://github.com/ClickHouse/ClickHouse/pull/65054) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fixing the `Not-ready Set` error after the `PREWHERE` optimization for StorageMerge. [#65057](https://github.com/ClickHouse/ClickHouse/pull/65057) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Avoid writing to finalized buffer in File-like storages. Closes [#62962](https://github.com/ClickHouse/ClickHouse/issues/62962). [#65063](https://github.com/ClickHouse/ClickHouse/pull/65063) ([Kruglov Pavel](https://github.com/Avogar)). -* Fix possible infinite query duration in case of cyclic aliases. Fixes [#64849](https://github.com/ClickHouse/ClickHouse/issues/64849). [#65081](https://github.com/ClickHouse/ClickHouse/pull/65081) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix the `Unknown expression identifier` error for remote queries with `INTERPOLATE (alias)` (new analyzer). Fixes [#64636](https://github.com/ClickHouse/ClickHouse/issues/64636). [#65090](https://github.com/ClickHouse/ClickHouse/pull/65090) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix pushing arithmetic operations out of aggregation. In the new analyzer, optimization was applied only once. Part of [#62245](https://github.com/ClickHouse/ClickHouse/issues/62245). [#65104](https://github.com/ClickHouse/ClickHouse/pull/65104) ([Dmitry Novik](https://github.com/novikd)). -* Fix aggregate function name rewriting in the new analyzer. Part of [#62245](https://github.com/ClickHouse/ClickHouse/issues/62245). [#65110](https://github.com/ClickHouse/ClickHouse/pull/65110) ([Dmitry Novik](https://github.com/novikd)). -* Respond with 5xx instead of 200 OK in case of receive timeout while reading (parts of) the request body from the client socket. [#65118](https://github.com/ClickHouse/ClickHouse/pull/65118) ([Julian Maicher](https://github.com/jmaicher)). -* Fix possible crash for hedged requests. [#65206](https://github.com/ClickHouse/ClickHouse/pull/65206) ([Azat Khuzhin](https://github.com/azat)). -* Fix the bug in Hashed and Hashed_Array dictionary short circuit evaluation, which may read uninitialized number, leading to various errors. [#65256](https://github.com/ClickHouse/ClickHouse/pull/65256) ([jsc0218](https://github.com/jsc0218)). -* This PR ensures that the type of the constant(IN operator's second parameter) is always visible during the IN operator's type conversion process. Otherwise, losing type information may cause some conversions to fail, such as the conversion from DateTime to Date. fix ([#64487](https://github.com/ClickHouse/ClickHouse/issues/64487)). [#65315](https://github.com/ClickHouse/ClickHouse/pull/65315) ([pn](https://github.com/chloro-pn)). - -#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) - -* Disabled `enable_vertical_final` setting by default. This feature should not be used because it has a bug: [#64543](https://github.com/ClickHouse/ClickHouse/issues/64543). [#64544](https://github.com/ClickHouse/ClickHouse/pull/64544) ([Alexander Tokmakov](https://github.com/tavplubix)). -* This PR fixes an error when a user in a specific situation can escalate their privileges on the default database without necessary grants. [#64769](https://github.com/ClickHouse/ClickHouse/pull/64769) ([pufit](https://github.com/pufit)). -* Fix crash with UniqInjectiveFunctionsEliminationPass and uniqCombined. [#65188](https://github.com/ClickHouse/ClickHouse/pull/65188) ([Raúl Marín](https://github.com/Algunenano)). -* Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). -* Forbid `QUALIFY` clause in the old analyzer. The old analyzer ignored `QUALIFY`, so it could lead to unexpected data removal in mutations. Closes [#64596](https://github.com/ClickHouse/ClickHouse/issues/64596). [#65356](https://github.com/ClickHouse/ClickHouse/pull/65356) ([Dmitry Novik](https://github.com/novikd)). -* Use correct memory alignment for Distinct combinator. Previously, crash could happen because of invalid memory allocation when the combinator was used. [#65379](https://github.com/ClickHouse/ClickHouse/pull/65379) ([Antonio Andelic](https://github.com/antonio2368)). - #### NO CL ENTRY * NO CL ENTRY: 'Revert "Refactoring of Server.h: Isolate server management from other logic"'. [#64425](https://github.com/ClickHouse/ClickHouse/pull/64425) ([Alexander Tokmakov](https://github.com/tavplubix)). @@ -199,19 +194,20 @@ #### NOT FOR CHANGELOG / INSIGNIFICANT * Try abort on current thread join. [#42544](https://github.com/ClickHouse/ClickHouse/pull/42544) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Change default s3_throw_on_zero_files_match to true, document that presigned S3 URLs are not supported. [#51008](https://github.com/ClickHouse/ClickHouse/pull/51008) ([Michael Kolupaev](https://github.com/al13n321)). +* This change was reverted. [#51008](https://github.com/ClickHouse/ClickHouse/pull/51008) ([Michael Kolupaev](https://github.com/al13n321)). * Analyzer fuzzer 2. [#57098](https://github.com/ClickHouse/ClickHouse/pull/57098) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Analyzer fuzzer 4. [#57101](https://github.com/ClickHouse/ClickHouse/pull/57101) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Check python code with flake8. [#58349](https://github.com/ClickHouse/ClickHouse/pull/58349) ([Azat Khuzhin](https://github.com/azat)). * Unite s3/hdfs/azure storage implementations into a single class working with IObjectStorage. Same for *Cluster, data lakes and Queue storages. [#59767](https://github.com/ClickHouse/ClickHouse/pull/59767) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Refactor KeyCondition and key analysis. [#61459](https://github.com/ClickHouse/ClickHouse/pull/61459) ([Amos Bird](https://github.com/amosbird)). +* Refactor KeyCondition and key analysis to improve PartitionPruner and trivial count optimization. This is separated from [#60463](https://github.com/ClickHouse/ClickHouse/issues/60463) . [#61459](https://github.com/ClickHouse/ClickHouse/pull/61459) ([Amos Bird](https://github.com/amosbird)). +* This change was reverted. [#61973](https://github.com/ClickHouse/ClickHouse/pull/61973) ([Azat Khuzhin](https://github.com/azat)). * Replay ZK logs using keeper-bench. [#62481](https://github.com/ClickHouse/ClickHouse/pull/62481) ([Antonio Andelic](https://github.com/antonio2368)). -* Improve builds' profiling upload. [#63325](https://github.com/ClickHouse/ClickHouse/pull/63325) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Reduce time-to-insert profiling data in case of logs cluster issues. [#63325](https://github.com/ClickHouse/ClickHouse/pull/63325) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Refactor data part writer to remove dependencies on MergeTreeData and DataPart. [#63620](https://github.com/ClickHouse/ClickHouse/pull/63620) ([Alexander Gololobov](https://github.com/davenger)). * Try to fix flaky s3 tests test_seekable_formats and test_seekable_formats_url. [#63720](https://github.com/ClickHouse/ClickHouse/pull/63720) ([Kruglov Pavel](https://github.com/Avogar)). * Rewrite plan for parallel replicas in Planner. [#63796](https://github.com/ClickHouse/ClickHouse/pull/63796) ([Igor Nikonov](https://github.com/devcrafter)). -* S3: reduce retires time for queries, increase retries count for backups. [#63857](https://github.com/ClickHouse/ClickHouse/pull/63857) ([Sema Checherinda](https://github.com/CheSema)). -* Implement a single point for "Check timeout expired" test result. [#63982](https://github.com/ClickHouse/ClickHouse/pull/63982) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* This PR was reverted. [#63857](https://github.com/ClickHouse/ClickHouse/pull/63857) ([Sema Checherinda](https://github.com/CheSema)). +* Make events like [timeouts](https://play.clickhouse.com/play?user=play#U0VMRUNUICogRlJPTSBjaGVja3MgV0hFUkUgdGVzdF9uYW1lID09ICdDaGVjayB0aW1lb3V0IGV4cGlyZWQnIEFORCBjaGVja19zdGFydF90aW1lIEJFVFdFRU4gdG9EYXRlKCcyMDI0LTA1LTEwJykgQU5EIHRvRGF0ZSgnMjAyNC0wNS0xNScp) visible in CI DB. [#63982](https://github.com/ClickHouse/ClickHouse/pull/63982) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Remove some unnecessary `UNREACHABLE`s. [#64035](https://github.com/ClickHouse/ClickHouse/pull/64035) ([Robert Schulze](https://github.com/rschu1ze)). * Throw out some `inline`s. [#64110](https://github.com/ClickHouse/ClickHouse/pull/64110) ([Robert Schulze](https://github.com/rschu1ze)). * Refactoring of Server.h: Isolate server management from other logic. [#64132](https://github.com/ClickHouse/ClickHouse/pull/64132) ([TTPO100AJIEX](https://github.com/TTPO100AJIEX)). @@ -230,22 +226,22 @@ * CI: master workflow with folded jobs. [#64340](https://github.com/ClickHouse/ClickHouse/pull/64340) ([Max K.](https://github.com/maxknv)). * CI: Sync, Merge check, CI gh's statuses fixes. [#64348](https://github.com/ClickHouse/ClickHouse/pull/64348) ([Max K.](https://github.com/maxknv)). * Enable 02494_query_cache_nested_query_bug for Analyzer. [#64357](https://github.com/ClickHouse/ClickHouse/pull/64357) ([Robert Schulze](https://github.com/rschu1ze)). -* Rename allow_deprecated_functions to allow_deprecated_error_prone_win…. [#64358](https://github.com/ClickHouse/ClickHouse/pull/64358) ([Raúl Marín](https://github.com/Algunenano)). +* Rename allow_deprecated_functions to allow_deprecated_error_prone_window_functions. [#64358](https://github.com/ClickHouse/ClickHouse/pull/64358) ([Raúl Marín](https://github.com/Algunenano)). * Change input_format_parquet_use_native_reader to 24.6. [#64359](https://github.com/ClickHouse/ClickHouse/pull/64359) ([Raúl Marín](https://github.com/Algunenano)). * Update description for settings `cross_join_min_rows_to_compress` and `cross_join_min_bytes_to_compress`. [#64360](https://github.com/ClickHouse/ClickHouse/pull/64360) ([Nikita Fomichev](https://github.com/fm4v)). -* Rename aggregate_function_group_array_has_limit_size. [#64362](https://github.com/ClickHouse/ClickHouse/pull/64362) ([Raúl Marín](https://github.com/Algunenano)). +* Changed the unreleased setting `aggregate_function_group_array_has_limit_size` to `aggregate_function_group_array_action_when_limit_is_reached`. [#64362](https://github.com/ClickHouse/ClickHouse/pull/64362) ([Raúl Marín](https://github.com/Algunenano)). * Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts. [#64363](https://github.com/ClickHouse/ClickHouse/pull/64363) ([Kruglov Pavel](https://github.com/Avogar)). * Try to fix GWPAsan. [#64365](https://github.com/ClickHouse/ClickHouse/pull/64365) ([Antonio Andelic](https://github.com/antonio2368)). * CI: add secrets to reusable stage wf yml. [#64366](https://github.com/ClickHouse/ClickHouse/pull/64366) ([Max K.](https://github.com/maxknv)). * Do not run tests tagged 'no-s3-storage-with-slow-build' with ASan. [#64367](https://github.com/ClickHouse/ClickHouse/pull/64367) ([vdimir](https://github.com/vdimir)). -* Add `fromReadableSize` function. [#64386](https://github.com/ClickHouse/ClickHouse/pull/64386) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). +* This change was reverted. [#64386](https://github.com/ClickHouse/ClickHouse/pull/64386) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). * Update s3queue.md. [#64389](https://github.com/ClickHouse/ClickHouse/pull/64389) ([Kseniia Sumarokova](https://github.com/kssenii)). * test for [#64211](https://github.com/ClickHouse/ClickHouse/issues/64211). [#64390](https://github.com/ClickHouse/ClickHouse/pull/64390) ([Denny Crane](https://github.com/den-crane)). * Follow-up to [#59767](https://github.com/ClickHouse/ClickHouse/issues/59767). [#64398](https://github.com/ClickHouse/ClickHouse/pull/64398) ([Kseniia Sumarokova](https://github.com/kssenii)). * Remove wrong comment. [#64403](https://github.com/ClickHouse/ClickHouse/pull/64403) ([Sergei Trifonov](https://github.com/serxa)). * Follow up to [#59767](https://github.com/ClickHouse/ClickHouse/issues/59767). [#64404](https://github.com/ClickHouse/ClickHouse/pull/64404) ([Kseniia Sumarokova](https://github.com/kssenii)). * Refactor s3 settings (move settings parsing into single place). [#64412](https://github.com/ClickHouse/ClickHouse/pull/64412) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Add dynamic untracked memory limits for more precise memory tracking. [#64423](https://github.com/ClickHouse/ClickHouse/pull/64423) ([Sergei Trifonov](https://github.com/serxa)). +* This PR was reverted. [#64423](https://github.com/ClickHouse/ClickHouse/pull/64423) ([Sergei Trifonov](https://github.com/serxa)). * Fix test after [#64404](https://github.com/ClickHouse/ClickHouse/issues/64404). [#64432](https://github.com/ClickHouse/ClickHouse/pull/64432) ([Kseniia Sumarokova](https://github.com/kssenii)). * Faster TestKeeper shutdown. [#64433](https://github.com/ClickHouse/ClickHouse/pull/64433) ([Alexander Gololobov](https://github.com/davenger)). * Remove some logging. [#64434](https://github.com/ClickHouse/ClickHouse/pull/64434) ([Kseniia Sumarokova](https://github.com/kssenii)). @@ -253,20 +249,21 @@ * Clean settings in 02943_variant_read_subcolumns test. [#64437](https://github.com/ClickHouse/ClickHouse/pull/64437) ([Kruglov Pavel](https://github.com/Avogar)). * Add a comment after [#64226](https://github.com/ClickHouse/ClickHouse/issues/64226). [#64449](https://github.com/ClickHouse/ClickHouse/pull/64449) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * CI: fix build_report selection in case of job reuse. [#64459](https://github.com/ClickHouse/ClickHouse/pull/64459) ([Max K.](https://github.com/maxknv)). -* CI: Critical bugfix category in PR template. [#64480](https://github.com/ClickHouse/ClickHouse/pull/64480) ([Max K.](https://github.com/maxknv)). +* Add Critical bugfix category in PR template. [#64480](https://github.com/ClickHouse/ClickHouse/pull/64480) ([Max K.](https://github.com/maxknv)). * Remove `generateSnowflakeIDThreadMonotonic`. [#64499](https://github.com/ClickHouse/ClickHouse/pull/64499) ([Robert Schulze](https://github.com/rschu1ze)). * Move analyzer attempt 2. [#64500](https://github.com/ClickHouse/ClickHouse/pull/64500) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Sync some code back from internal to public repository. [#64502](https://github.com/ClickHouse/ClickHouse/pull/64502) ([Robert Schulze](https://github.com/rschu1ze)). * Remove `generateUUIDv7(NonMonotonic|ThreadMonotonic)` functions. [#64506](https://github.com/ClickHouse/ClickHouse/pull/64506) ([Robert Schulze](https://github.com/rschu1ze)). * Fix bash completion for settings. [#64521](https://github.com/ClickHouse/ClickHouse/pull/64521) ([Azat Khuzhin](https://github.com/azat)). * Use max_read_buffer_size for file descriptors as well in file(). [#64532](https://github.com/ClickHouse/ClickHouse/pull/64532) ([Azat Khuzhin](https://github.com/azat)). -* Bring hash table sizes tests back. [#64550](https://github.com/ClickHouse/ClickHouse/pull/64550) ([Nikita Taranov](https://github.com/nickitat)). +* Temporarily disable `enable_vertical_final` setting by default. This feature should not be used in older releases because it [might crash](https://github.com/ClickHouse/ClickHouse/issues/64543), but it's already fixed in 24.6 where this setting change has been reverted and `enable_vertical_final` is again enabled by default. [#64544](https://github.com/ClickHouse/ClickHouse/pull/64544) ([Alexander Tokmakov](https://github.com/tavplubix)). +* Removed excessive calls to `flush logs` and disabled under sanitizers. [#64550](https://github.com/ClickHouse/ClickHouse/pull/64550) ([Nikita Taranov](https://github.com/nickitat)). * Sync code moved in private repo back back to public repo. [#64551](https://github.com/ClickHouse/ClickHouse/pull/64551) ([Robert Schulze](https://github.com/rschu1ze)). -* ASTLiteral support custom type. [#64562](https://github.com/ClickHouse/ClickHouse/pull/64562) ([shuai.xu](https://github.com/shuai-xu)). +* Add support for custom type to ASTLiteral, or else the type may be lost when parse the ast. E.g. set a ASTLiteral to DataTime32 with value 19870, then it will be parsed to Int16. [#64562](https://github.com/ClickHouse/ClickHouse/pull/64562) ([shuai.xu](https://github.com/shuai-xu)). * Add a temporary known host for git over ssh. [#64569](https://github.com/ClickHouse/ClickHouse/pull/64569) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Cache first analysis result in ReadFromMergeTree. [#64579](https://github.com/ClickHouse/ClickHouse/pull/64579) ([Igor Nikonov](https://github.com/devcrafter)). -* CI: Backports updates. [#64603](https://github.com/ClickHouse/ClickHouse/pull/64603) ([Max K.](https://github.com/maxknv)). -* CI: Change Required Check list, few fixes. [#64605](https://github.com/ClickHouse/ClickHouse/pull/64605) ([Max K.](https://github.com/maxknv)). +* Derive script parameters (labels) from the --repo/--from-repo - fix to not create backports for all release branches if backport for specific branch only. [#64603](https://github.com/ClickHouse/ClickHouse/pull/64603) ([Max K.](https://github.com/maxknv)). +* Add Stateful asan test, Stateless asan, Stateless flaky chek into Required - Move binary_release to normal builds (not special). It builds fast and there are test jobs depending on it. - Add job description for A Sync. [#64605](https://github.com/ClickHouse/ClickHouse/pull/64605) ([Max K.](https://github.com/maxknv)). * Double-checking [#59318](https://github.com/ClickHouse/ClickHouse/issues/59318) and docs for `Map`. [#64606](https://github.com/ClickHouse/ClickHouse/pull/64606) ([Robert Schulze](https://github.com/rschu1ze)). * Update CHANGELOG.md. [#64609](https://github.com/ClickHouse/ClickHouse/pull/64609) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Tests: Convert numeric to symbolic error codes. [#64635](https://github.com/ClickHouse/ClickHouse/pull/64635) ([Robert Schulze](https://github.com/rschu1ze)). @@ -279,17 +276,18 @@ * CI: Build Report Check to verify only enabled builds. [#64669](https://github.com/ClickHouse/ClickHouse/pull/64669) ([Max K.](https://github.com/maxknv)). * Tests: Convert error numbers to symbolic error codes, pt. II. [#64670](https://github.com/ClickHouse/ClickHouse/pull/64670) ([Robert Schulze](https://github.com/rschu1ze)). * Split query analyzer. [#64672](https://github.com/ClickHouse/ClickHouse/pull/64672) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* CI: CI running status fix. [#64693](https://github.com/ClickHouse/ClickHouse/pull/64693) ([Max K.](https://github.com/maxknv)). -* Cleanup the changelog for 24.5. [#64704](https://github.com/ClickHouse/ClickHouse/pull/64704) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* By the end of CI, CI_Running status must be SUCCESS or FAILURE never PENDING. [#64693](https://github.com/ClickHouse/ClickHouse/pull/64693) ([Max K.](https://github.com/maxknv)). +* The following list of merged PRs is not present in the release branch and was added to the changelog by mistake:. [#64704](https://github.com/ClickHouse/ClickHouse/pull/64704) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * CI: MergeQueue: add binary_release and unit tests. [#64705](https://github.com/ClickHouse/ClickHouse/pull/64705) ([Max K.](https://github.com/maxknv)). -* CI: get_best_robot_token fix to get good enough token. [#64709](https://github.com/ClickHouse/ClickHouse/pull/64709) ([Max K.](https://github.com/maxknv)). +* Fix to get first good enough GH token instead of getting and comparing all of them. [#64709](https://github.com/ClickHouse/ClickHouse/pull/64709) ([Max K.](https://github.com/maxknv)). * Check for missing Upload ID in CreateMultipartUpload reply. [#64714](https://github.com/ClickHouse/ClickHouse/pull/64714) ([Michael Kolupaev](https://github.com/al13n321)). * Update version_date.tsv and changelogs after v24.5.1.1763-stable. [#64715](https://github.com/ClickHouse/ClickHouse/pull/64715) ([robot-clickhouse](https://github.com/robot-clickhouse)). +* Fix (unreleased) `loop()` table function crashing on empty table name. [#64716](https://github.com/ClickHouse/ClickHouse/pull/64716) ([Michael Kolupaev](https://github.com/al13n321)). * Update CHANGELOG.md. [#64730](https://github.com/ClickHouse/ClickHouse/pull/64730) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * CI: ci.py refactoring. [#64734](https://github.com/ClickHouse/ClickHouse/pull/64734) ([Max K.](https://github.com/maxknv)). * Return the explanation for session moved error. [#64747](https://github.com/ClickHouse/ClickHouse/pull/64747) ([Antonio Andelic](https://github.com/antonio2368)). -* Adjust the `version_helper` and script to a new release scheme. [#64759](https://github.com/ClickHouse/ClickHouse/pull/64759) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Do not try to write columns.txt if it does not exists for write-once storages. [#64762](https://github.com/ClickHouse/ClickHouse/pull/64762) ([Azat Khuzhin](https://github.com/azat)). +* It's another follow-up for https://github.com/ClickHouse/ClickHouse/pull/64039. We need to backport it to avoid issues with the new tag name. [#64759](https://github.com/ClickHouse/ClickHouse/pull/64759) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Do not try to write columns.txt if it does not exist for write-once storages. [#64762](https://github.com/ClickHouse/ClickHouse/pull/64762) ([Azat Khuzhin](https://github.com/azat)). * Update 02482_load_parts_refcounts.sh. [#64765](https://github.com/ClickHouse/ClickHouse/pull/64765) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix crash with DISTINCT and window functions. [#64767](https://github.com/ClickHouse/ClickHouse/pull/64767) ([Igor Nikonov](https://github.com/devcrafter)). * Fix assert in IObjectStorageIteratorAsync. [#64770](https://github.com/ClickHouse/ClickHouse/pull/64770) ([Michael Kolupaev](https://github.com/al13n321)). @@ -298,53 +296,54 @@ * Fix flaky tests about SQLite. [#64776](https://github.com/ClickHouse/ClickHouse/pull/64776) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Remove iostream debug helpers. [#64777](https://github.com/ClickHouse/ClickHouse/pull/64777) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Remove unnecessary comment. [#64785](https://github.com/ClickHouse/ClickHouse/pull/64785) ([Raúl Marín](https://github.com/Algunenano)). -* Strict `StatusType`, increased `SAFE_REQUESTS_LIMIT` for get_best_robot_token. [#64787](https://github.com/ClickHouse/ClickHouse/pull/64787) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Follow-ups to some PRs. [#64787](https://github.com/ClickHouse/ClickHouse/pull/64787) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Attempt to fix 02228_merge_tree_insert_memory_usage.sql flakiness for s3. [#64800](https://github.com/ClickHouse/ClickHouse/pull/64800) ([Raúl Marín](https://github.com/Algunenano)). * Add regression test for filter propagation through `Merge` engine. [#64806](https://github.com/ClickHouse/ClickHouse/pull/64806) ([Nikita Taranov](https://github.com/nickitat)). * Migrate changelog.py to a descendant of fuzzywuzzy. [#64807](https://github.com/ClickHouse/ClickHouse/pull/64807) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Changelog in ci. [#64813](https://github.com/ClickHouse/ClickHouse/pull/64813) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* A follow-up for https://github.com/ClickHouse/ClickHouse/pull/64039 and [#64759](https://github.com/ClickHouse/ClickHouse/issues/64759). [#64813](https://github.com/ClickHouse/ClickHouse/pull/64813) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Make row order optimization non-experimental. [#64814](https://github.com/ClickHouse/ClickHouse/pull/64814) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix wrong supported year recognition. [#64817](https://github.com/ClickHouse/ClickHouse/pull/64817) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Didn't catch it at the time when all versions belonged to the current year. [#64817](https://github.com/ClickHouse/ClickHouse/pull/64817) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix clang-tidy build. [#64823](https://github.com/ClickHouse/ClickHouse/pull/64823) ([Robert Schulze](https://github.com/rschu1ze)). -* CI: Move coverage build to non-special build list. [#64824](https://github.com/ClickHouse/ClickHouse/pull/64824) ([Max K.](https://github.com/maxknv)). +* Sets all builds that we run tests on to normal build list. [#64824](https://github.com/ClickHouse/ClickHouse/pull/64824) ([Max K.](https://github.com/maxknv)). * CI: fix CI await feature. [#64825](https://github.com/ClickHouse/ClickHouse/pull/64825) ([Max K.](https://github.com/maxknv)). * Fix clang-tidy. [#64827](https://github.com/ClickHouse/ClickHouse/pull/64827) ([Kseniia Sumarokova](https://github.com/kssenii)). * Upload blob_storage_log from stateless tests. [#64843](https://github.com/ClickHouse/ClickHouse/pull/64843) ([alesapin](https://github.com/alesapin)). * Follow-up to [#64349](https://github.com/ClickHouse/ClickHouse/issues/64349). [#64845](https://github.com/ClickHouse/ClickHouse/pull/64845) ([Kseniia Sumarokova](https://github.com/kssenii)). * Simplify handling of old 'inverted' indexes. [#64846](https://github.com/ClickHouse/ClickHouse/pull/64846) ([Robert Schulze](https://github.com/rschu1ze)). -* Replace Markdown with YAML for ISSUE_TEMPLATE. [#64850](https://github.com/ClickHouse/ClickHouse/pull/64850) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Handle logs from rocksdb by ClickHouse internal logging (and enable debug rocksdb logging on CI). [#64856](https://github.com/ClickHouse/ClickHouse/pull/64856) ([Azat Khuzhin](https://github.com/azat)). -* Add new settings to changes history. [#64860](https://github.com/ClickHouse/ClickHouse/pull/64860) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Templates defined in YAML provide more user-friendly experience. References: - [Documentation](https://docs.github.com/en/communities/using-templates-to-encourage-useful-issues-and-pull-requests/syntax-for-issue-forms) - [How it looks like in other projects. ](https://github.com/angular/angular/issues/new?assignees=&labels=&projects=&template=1-bug-report.yaml). [#64850](https://github.com/ClickHouse/ClickHouse/pull/64850) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Handle logs from rocksdb by ClickHouse internal logging. [#64856](https://github.com/ClickHouse/ClickHouse/pull/64856) ([Azat Khuzhin](https://github.com/azat)). +* Follow-up for https://github.com/ClickHouse/ClickHouse/pull/59357. [#64860](https://github.com/ClickHouse/ClickHouse/pull/64860) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * added mlock and mlockall to aspell-dict to be ignored. [#64863](https://github.com/ClickHouse/ClickHouse/pull/64863) ([Ali](https://github.com/xogoodnow)). * A tiny fix for fancy quotes. [#64883](https://github.com/ClickHouse/ClickHouse/pull/64883) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix possible loss of "Query was cancelled" message in client (fixes 03023_zeros_generate_random_with_limit_progress_bar flakiness). [#64888](https://github.com/ClickHouse/ClickHouse/pull/64888) ([Azat Khuzhin](https://github.com/azat)). -* Fix documentation enforcement. [#64890](https://github.com/ClickHouse/ClickHouse/pull/64890) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix possible loss of "Query was cancelled" message in client. [#64888](https://github.com/ClickHouse/ClickHouse/pull/64888) ([Azat Khuzhin](https://github.com/azat)). +* We accidentally lost the way to set `PR Check` failure at some point. [#64890](https://github.com/ClickHouse/ClickHouse/pull/64890) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix global trace collector. [#64896](https://github.com/ClickHouse/ClickHouse/pull/64896) ([Antonio Andelic](https://github.com/antonio2368)). * Fix test_mask_sensitive_info/test.py::test_create_table. [#64901](https://github.com/ClickHouse/ClickHouse/pull/64901) ([Azat Khuzhin](https://github.com/azat)). * Update 03165_string_functions_with_token_text_indexes.sql. [#64903](https://github.com/ClickHouse/ClickHouse/pull/64903) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Persistent PRInfo diff url. [#64904](https://github.com/ClickHouse/ClickHouse/pull/64904) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* When the branch is removed, it's impossible to get the diff by the labels. `print` in imported files spoils the `ipython` output. [#64904](https://github.com/ClickHouse/ClickHouse/pull/64904) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Disable transactions for unsupported storages even for materialized v…. [#64918](https://github.com/ClickHouse/ClickHouse/pull/64918) ([alesapin](https://github.com/alesapin)). * additional log for cleanupDetachedTables. [#64919](https://github.com/ClickHouse/ClickHouse/pull/64919) ([Konstantin Morozov](https://github.com/k-morozov)). -* Fix tupleConcat of two empty tuples. [#64923](https://github.com/ClickHouse/ClickHouse/pull/64923) ([Amos Bird](https://github.com/amosbird)). +* Fix tupleConcat of two empty tuples. This fixes [#64885](https://github.com/ClickHouse/ClickHouse/issues/64885). [#64923](https://github.com/ClickHouse/ClickHouse/pull/64923) ([Amos Bird](https://github.com/amosbird)). * CI: Minor fixes in ci scripts. [#64950](https://github.com/ClickHouse/ClickHouse/pull/64950) ([Max K.](https://github.com/maxknv)). * Fix error message (it was strange). [#64952](https://github.com/ClickHouse/ClickHouse/pull/64952) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Update fmtlib version to 9.1.0. [#64959](https://github.com/ClickHouse/ClickHouse/pull/64959) ([Duc Canh Le](https://github.com/canhld94)). -* Disable 02908_many_requests_to_system_replicas with ASAN. [#64966](https://github.com/ClickHouse/ClickHouse/pull/64966) ([Alexander Gololobov](https://github.com/davenger)). -* Fix bug in short circuit evaluation. [#64967](https://github.com/ClickHouse/ClickHouse/pull/64967) ([Raúl Marín](https://github.com/Algunenano)). +* Test 02908_many_requests_to_system_replicas makes a lot of heavy requests and it overloads server if it's an ASAN build. [#64966](https://github.com/ClickHouse/ClickHouse/pull/64966) ([Alexander Gololobov](https://github.com/davenger)). +* Fix (unreleased) bug in short circuit evaluation. [#64967](https://github.com/ClickHouse/ClickHouse/pull/64967) ([Raúl Marín](https://github.com/Algunenano)). * Update version_date.tsv and changelogs after v24.4.2.141-stable. [#64968](https://github.com/ClickHouse/ClickHouse/pull/64968) ([robot-clickhouse](https://github.com/robot-clickhouse)). * Fix `test_attach_partition_using_copy`. [#64977](https://github.com/ClickHouse/ClickHouse/pull/64977) ([Alexander Tokmakov](https://github.com/tavplubix)). * Faster processing of scheduler queue activations. [#64985](https://github.com/ClickHouse/ClickHouse/pull/64985) ([Sergei Trifonov](https://github.com/serxa)). * CI: Fix nightly workflow. [#64987](https://github.com/ClickHouse/ClickHouse/pull/64987) ([Max K.](https://github.com/maxknv)). * Fix innocuous data race in detectLanguage. [#64988](https://github.com/ClickHouse/ClickHouse/pull/64988) ([Raúl Marín](https://github.com/Algunenano)). * CI: Builds in CI settings. [#64994](https://github.com/ClickHouse/ClickHouse/pull/64994) ([Max K.](https://github.com/maxknv)). -* Aggregate function groupConcat ([#63671](https://github.com/ClickHouse/ClickHouse/issues/63671)). [#65009](https://github.com/ClickHouse/ClickHouse/pull/65009) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). +* REVERTED. [#65009](https://github.com/ClickHouse/ClickHouse/pull/65009) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). * CI: Fix backports. [#65010](https://github.com/ClickHouse/ClickHouse/pull/65010) ([Max K.](https://github.com/maxknv)). * Try fix 03143_prewhere_profile_events. [#65014](https://github.com/ClickHouse/ClickHouse/pull/65014) ([Nikita Taranov](https://github.com/nickitat)). * Fix 03165_string_functions_with_token_text_indexes. [#65018](https://github.com/ClickHouse/ClickHouse/pull/65018) ([Julia Kartseva](https://github.com/jkartseva)). +* This change was reverted. [#65028](https://github.com/ClickHouse/ClickHouse/pull/65028) ([Sergei Trifonov](https://github.com/serxa)). * Bump googletest to latest HEAD. [#65038](https://github.com/ClickHouse/ClickHouse/pull/65038) ([Robert Schulze](https://github.com/rschu1ze)). * Improve comment about AsynchronousMetrics. [#65040](https://github.com/ClickHouse/ClickHouse/pull/65040) ([Antonio Andelic](https://github.com/antonio2368)). * CI: Remove fuzzer build from normal CI run (bugfix). [#65041](https://github.com/ClickHouse/ClickHouse/pull/65041) ([Max K.](https://github.com/maxknv)). -* CI: ci_config refactoring. [#65045](https://github.com/ClickHouse/ClickHouse/pull/65045) ([Max K.](https://github.com/maxknv)). +* CI config refactoring [#65045](https://github.com/ClickHouse/ClickHouse/pull/65045) ([Max K.](https://github.com/maxknv)). * Bump abseil to latest HEAD. [#65048](https://github.com/ClickHouse/ClickHouse/pull/65048) ([Robert Schulze](https://github.com/rschu1ze)). * Capture weak_ptr of ContextAccess for safety. [#65051](https://github.com/ClickHouse/ClickHouse/pull/65051) ([Alexander Gololobov](https://github.com/davenger)). * Stateless tests: add test for SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT. [#65056](https://github.com/ClickHouse/ClickHouse/pull/65056) ([Nikita Fomichev](https://github.com/fm4v)). @@ -358,23 +357,23 @@ * Fix `Initiator received more initial requests than there are replicas` with `loop` engine. [#65133](https://github.com/ClickHouse/ClickHouse/pull/65133) ([Nikita Taranov](https://github.com/nickitat)). * Fix 'Tasks in BackgroundSchedulePool cannot throw' caused by MergeTreeData::loadUnexpectedDataParts(). [#65135](https://github.com/ClickHouse/ClickHouse/pull/65135) ([Michael Kolupaev](https://github.com/al13n321)). * Fix bad error message. [#65137](https://github.com/ClickHouse/ClickHouse/pull/65137) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* rework tests with sleep: use sleep_until instead sleep_for. [#65152](https://github.com/ClickHouse/ClickHouse/pull/65152) ([Sema Checherinda](https://github.com/CheSema)). -* Fix AWS ECS. [#65164](https://github.com/ClickHouse/ClickHouse/pull/65164) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Just fixing flaky unit tests. [#65152](https://github.com/ClickHouse/ClickHouse/pull/65152) ([Sema Checherinda](https://github.com/CheSema)). +* This change was reverted. [#65164](https://github.com/ClickHouse/ClickHouse/pull/65164) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Ensure submodules are named consistently. [#65167](https://github.com/ClickHouse/ClickHouse/pull/65167) ([Robert Schulze](https://github.com/rschu1ze)). * Remove obsolete fix from aws submodule. [#65168](https://github.com/ClickHouse/ClickHouse/pull/65168) ([Robert Schulze](https://github.com/rschu1ze)). * CI: Fix not-merged cherry-picks for backports. [#65181](https://github.com/ClickHouse/ClickHouse/pull/65181) ([Max K.](https://github.com/maxknv)). * Add an assertion in ReplicatedMergeTreeQueue. [#65184](https://github.com/ClickHouse/ClickHouse/pull/65184) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix bug in short circuit optimization with cache dictionaries. [#65185](https://github.com/ClickHouse/ClickHouse/pull/65185) ([Raúl Marín](https://github.com/Algunenano)). +* Fix bug in unreleased code. [#65185](https://github.com/ClickHouse/ClickHouse/pull/65185) ([Raúl Marín](https://github.com/Algunenano)). * Fix docs for skipping-indexes.md. [#65194](https://github.com/ClickHouse/ClickHouse/pull/65194) ([morning-color](https://github.com/morning-color)). * Fix the descriptions of some server settings. [#65200](https://github.com/ClickHouse/ClickHouse/pull/65200) ([Raúl Marín](https://github.com/Algunenano)). -* Fix search issues for progress_func, add zstd for GH cache. [#65202](https://github.com/ClickHouse/ClickHouse/pull/65202) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix bug in short circuit optimization with direct dictionaries. [#65203](https://github.com/ClickHouse/ClickHouse/pull/65203) ([Raúl Marín](https://github.com/Algunenano)). +* Fix issue after [#64813](https://github.com/ClickHouse/ClickHouse/issues/64813) with broken search in the changelog, and missing zstd in a style-check image. [#65202](https://github.com/ClickHouse/ClickHouse/pull/65202) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Fix bug in unreleased code. [#65203](https://github.com/ClickHouse/ClickHouse/pull/65203) ([Raúl Marín](https://github.com/Algunenano)). * Add test prewhere merge. [#65207](https://github.com/ClickHouse/ClickHouse/pull/65207) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Sync ProfileEvents.h. [#65208](https://github.com/ClickHouse/ClickHouse/pull/65208) ([Kseniia Sumarokova](https://github.com/kssenii)). -* CI: FinishCheck to set failure if workflow failed. [#65228](https://github.com/ClickHouse/ClickHouse/pull/65228) ([Max K.](https://github.com/maxknv)). +* FinishCheck to set failure if workflow failed. [#65228](https://github.com/ClickHouse/ClickHouse/pull/65228) ([Max K.](https://github.com/maxknv)). * Update version_date.tsv and changelogs after v24.3.4.147-lts. [#65235](https://github.com/ClickHouse/ClickHouse/pull/65235) ([robot-clickhouse](https://github.com/robot-clickhouse)). * Update version_date.tsv and changelogs after v24.5.3.5-stable. [#65240](https://github.com/ClickHouse/ClickHouse/pull/65240) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Less flaky `01019_alter_materialized_view_consistent`. [#65245](https://github.com/ClickHouse/ClickHouse/pull/65245) ([Antonio Andelic](https://github.com/antonio2368)). +* Fails sometimes for debug build https://s3.amazonaws.com/clickhouse-test-reports/0/af6afd904316bfb771737faa147ce8aea72dd705/stateless_tests__debug__[4_5].html. [#65245](https://github.com/ClickHouse/ClickHouse/pull/65245) ([Antonio Andelic](https://github.com/antonio2368)). * Fix libunwind in CI. [#65247](https://github.com/ClickHouse/ClickHouse/pull/65247) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * CI: Do not skip FinishCheck in Merge Queue. [#65249](https://github.com/ClickHouse/ClickHouse/pull/65249) ([Max K.](https://github.com/maxknv)). * Add a test just in case. [#65271](https://github.com/ClickHouse/ClickHouse/pull/65271) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -394,17 +393,13 @@ * Fix compatibility release check. [#65394](https://github.com/ClickHouse/ClickHouse/pull/65394) ([Alexey Katsman](https://github.com/alexkats)). * Move a leaksan suppression from Poco into OpenSSL. [#65396](https://github.com/ClickHouse/ClickHouse/pull/65396) ([Robert Schulze](https://github.com/rschu1ze)). * Fix tidy build. [#65415](https://github.com/ClickHouse/ClickHouse/pull/65415) ([Sergei Trifonov](https://github.com/serxa)). -* CI: Master workflow dependencies fix. [#65416](https://github.com/ClickHouse/ClickHouse/pull/65416) ([Max K.](https://github.com/maxknv)). +* Remove Tests dependency on Builds_2. No tests depend on Builds_2. [#65416](https://github.com/ClickHouse/ClickHouse/pull/65416) ([Max K.](https://github.com/maxknv)). * CI: PR workflow dependencies fix. [#65442](https://github.com/ClickHouse/ClickHouse/pull/65442) ([Max K.](https://github.com/maxknv)). * Fix test_storage_s3_queue/test.py::test_max_set_age. [#65452](https://github.com/ClickHouse/ClickHouse/pull/65452) ([Kseniia Sumarokova](https://github.com/kssenii)). * CI: Rename A Sync status. [#65456](https://github.com/ClickHouse/ClickHouse/pull/65456) ([Max K.](https://github.com/maxknv)). * CI: Rename sync status. [#65464](https://github.com/ClickHouse/ClickHouse/pull/65464) ([Max K.](https://github.com/maxknv)). -* Fix `01926_order_by_desc_limit` test for 1MB HTTP buffers. [#65466](https://github.com/ClickHouse/ClickHouse/pull/65466) ([Sergei Trifonov](https://github.com/serxa)). -* Remove groupConcat. [#65480](https://github.com/ClickHouse/ClickHouse/pull/65480) ([Raúl Marín](https://github.com/Algunenano)). - -#### Packaging Improvement - -* Let `network` service be required when using the init script to start the ClickHouse server daemon. [#60650](https://github.com/ClickHouse/ClickHouse/pull/60650) ([Chun-Sheng, Li](https://github.com/peter279k)). +* This change was reverted. [#65466](https://github.com/ClickHouse/ClickHouse/pull/65466) ([Sergei Trifonov](https://github.com/serxa)). +* Remove a feature wasn't part of any release yet. [#65480](https://github.com/ClickHouse/ClickHouse/pull/65480) ([Raúl Marín](https://github.com/Algunenano)). ### ClickHouse release 24.5, 2024-05-30 From 9064e3611fed855a65eb34c36f40693651ed068f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 20 Jun 2024 18:02:59 +0000 Subject: [PATCH 287/318] Manually removed backported feature --- CHANGELOG.md | 1 - 1 file changed, 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ce6b46f70d6..3f00b596a08 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -20,7 +20,6 @@ * Introduce statistics of type "number of distinct values". [#59357](https://github.com/ClickHouse/ClickHouse/pull/59357) ([Han Fei](https://github.com/hanfei1991)). * Add Hilbert Curve encode and decode functions. [#60156](https://github.com/ClickHouse/ClickHouse/pull/60156) ([Artem Mustafin](https://github.com/Artemmm91)). * Added support for reading LINESTRING geometry in WKT format using function `readWKTLineString`. [#62519](https://github.com/ClickHouse/ClickHouse/pull/62519) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Implement Dynamic data type that allows to store values of any type inside it without knowing all of them in advance. Dynamic type is available under a setting `allow_experimental_dynamic_type`. [#63058](https://github.com/ClickHouse/ClickHouse/pull/63058) ([Kruglov Pavel](https://github.com/Avogar)). * Allow to attach parts from a different disk. [#63087](https://github.com/ClickHouse/ClickHouse/pull/63087) ([Unalian](https://github.com/Unalian)). * Allow proxy to be bypassed for hosts specified in `no_proxy` env variable and ClickHouse proxy configuration. [#63314](https://github.com/ClickHouse/ClickHouse/pull/63314) ([Arthur Passos](https://github.com/arthurpassos)). * Added a new table function `loop` to support returning query results in an infinite loop. [#63452](https://github.com/ClickHouse/ClickHouse/pull/63452) ([Sariel](https://github.com/sarielwxm)). From 2a4d055ea77947c0a506038991bab2a13a748cc7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Jun 2024 20:10:11 +0200 Subject: [PATCH 288/318] Fix test --- tests/integration/test_keeper_client/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index ca22c119281..f5f7d855808 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -127,7 +127,7 @@ def test_base_commands(client: KeeperClient): assert client.get("/test_create_zk_node1") == "testvalue1" client.create("/123", "1=2") - client.create("/123/321", "'foo;bar'") + client.create("/123/321", "foo;bar") assert client.get("/123") == "1=2" assert client.get("/123/321") == "foo;bar" From c011cc932499ba88035e3fc37108900328ddacd2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Jun 2024 20:10:11 +0200 Subject: [PATCH 289/318] Fix test --- tests/integration/test_keeper_client/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_client/test.py b/tests/integration/test_keeper_client/test.py index ca22c119281..f5f7d855808 100644 --- a/tests/integration/test_keeper_client/test.py +++ b/tests/integration/test_keeper_client/test.py @@ -127,7 +127,7 @@ def test_base_commands(client: KeeperClient): assert client.get("/test_create_zk_node1") == "testvalue1" client.create("/123", "1=2") - client.create("/123/321", "'foo;bar'") + client.create("/123/321", "foo;bar") assert client.get("/123") == "1=2" assert client.get("/123/321") == "foo;bar" From eed4b34e7d220262107ac6b244a090df73b6e6be Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 20 Jun 2024 20:29:51 +0200 Subject: [PATCH 290/318] Fix test --- tests/integration/helpers/keeper_utils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index ab66d1a8866..f798482212f 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -196,7 +196,7 @@ class KeeperClient(object): ) return self.execute_query( - f"reconfig {operation} {joining or leaving or new_members}", timeout + f"reconfig '{operation}' {joining or leaving or new_members}", timeout ) @classmethod From 75d175b08615d5bf93855e55f539323253bcc9aa Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 14 Jun 2024 16:13:36 +0000 Subject: [PATCH 291/318] Disable userspace page cache by default --- src/Core/ServerSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 1fde8d58c7b..6a2aa1da859 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -85,7 +85,7 @@ namespace DB M(Double, index_mark_cache_size_ratio, DEFAULT_INDEX_MARK_CACHE_SIZE_RATIO, "The size of the protected queue in the secondary index mark cache relative to the cache's total size.", 0) \ M(UInt64, page_cache_chunk_size, 2 << 20, "Bytes per chunk in userspace page cache. Rounded up to a multiple of page size (typically 4 KiB) or huge page size (typically 2 MiB, only if page_cache_use_thp is enabled).", 0) \ M(UInt64, page_cache_mmap_size, 1 << 30, "Bytes per memory mapping in userspace page cache. Not important.", 0) \ - M(UInt64, page_cache_size, 10ul << 30, "Amount of virtual memory to map for userspace page cache. If page_cache_use_madv_free is enabled, it's recommended to set this higher than the machine's RAM size. Use 0 to disable userspace page cache.", 0) \ + M(UInt64, page_cache_size, 0, "Amount of virtual memory to map for userspace page cache. If page_cache_use_madv_free is enabled, it's recommended to set this higher than the machine's RAM size. Use 0 to disable userspace page cache.", 0) \ M(Bool, page_cache_use_madv_free, DBMS_DEFAULT_PAGE_CACHE_USE_MADV_FREE, "If true, the userspace page cache will allow the OS to automatically reclaim memory from the cache on memory pressure (using MADV_FREE).", 0) \ M(Bool, page_cache_use_transparent_huge_pages, true, "Userspace will attempt to use transparent huge pages on Linux. This is best-effort.", 0) \ M(UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0) \ From 952aae8bc2161a011c30b3cca895bb1f9626c165 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Tue, 18 Jun 2024 19:44:08 +0000 Subject: [PATCH 292/318] Delete test --- src/Common/ICachePolicy.h | 2 +- .../configs/storage_conf_web.xml | 2 + .../0_stateless/02867_page_cache.reference | 21 ---- .../queries/0_stateless/02867_page_cache.sql | 106 ------------------ 4 files changed, 3 insertions(+), 128 deletions(-) delete mode 100644 tests/queries/0_stateless/02867_page_cache.reference delete mode 100644 tests/queries/0_stateless/02867_page_cache.sql diff --git a/src/Common/ICachePolicy.h b/src/Common/ICachePolicy.h index 8aa75d1d81f..301a5c6cbbd 100644 --- a/src/Common/ICachePolicy.h +++ b/src/Common/ICachePolicy.h @@ -48,7 +48,7 @@ public: /// HashFunction usually hashes the entire key and the found key will be equal the provided key. In such cases, use get(). It is also /// possible to store other, non-hashed data in the key. In that case, the found key is potentially different from the provided key. - /// Then use getWithKey() to also return the found key including it's non-hashed data. + /// Then use getWithKey() to also return the found key including its non-hashed data. virtual MappedPtr get(const Key & key) = 0; virtual std::optional getWithKey(const Key &) = 0; diff --git a/tests/integration/test_disk_over_web_server/configs/storage_conf_web.xml b/tests/integration/test_disk_over_web_server/configs/storage_conf_web.xml index 482aa78e611..2f9fc75bd8c 100644 --- a/tests/integration/test_disk_over_web_server/configs/storage_conf_web.xml +++ b/tests/integration/test_disk_over_web_server/configs/storage_conf_web.xml @@ -30,5 +30,7 @@ + 10000000000 + diff --git a/tests/queries/0_stateless/02867_page_cache.reference b/tests/queries/0_stateless/02867_page_cache.reference deleted file mode 100644 index c3d6484a175..00000000000 --- a/tests/queries/0_stateless/02867_page_cache.reference +++ /dev/null @@ -1,21 +0,0 @@ -cold read 54975576145920 -PageCacheBytesUnpinnedRoundedToHugePages 1 -PageCacheBytesUnpinnedRoundedToPages 1 -PageCacheChunkMisses 1 -ReadBufferFromS3Bytes 1 -repeat read 1 54975576145920 -PageCacheBytesUnpinnedRoundedToHugePages 1 -PageCacheBytesUnpinnedRoundedToPages 1 -PageCacheChunkDataHits 1 -dropped and bypassed cache 54975576145920 -PageCacheChunkMisses 1 -ReadBufferFromS3Bytes 1 -repeat read 2 54975576145920 -PageCacheBytesUnpinnedRoundedToHugePages 1 -PageCacheBytesUnpinnedRoundedToPages 1 -PageCacheChunkMisses 1 -ReadBufferFromS3Bytes 1 -repeat read 3 54975576145920 -PageCacheBytesUnpinnedRoundedToHugePages 1 -PageCacheBytesUnpinnedRoundedToPages 1 -PageCacheChunkDataHits 1 diff --git a/tests/queries/0_stateless/02867_page_cache.sql b/tests/queries/0_stateless/02867_page_cache.sql deleted file mode 100644 index f1882de4af6..00000000000 --- a/tests/queries/0_stateless/02867_page_cache.sql +++ /dev/null @@ -1,106 +0,0 @@ --- Tags: no-fasttest, no-parallel --- no-fasttest because we need an S3 storage policy --- no-parallel because we look at server-wide counters about page cache usage - -set use_page_cache_for_disks_without_file_cache = 1; -set page_cache_inject_eviction = 0; -set enable_filesystem_cache = 0; -set use_uncompressed_cache = 0; - -create table events_snapshot engine Memory as select * from system.events; -create view events_diff as - -- round all stats to 70 MiB to leave a lot of leeway for overhead - with if(event like '%Bytes%', 70*1024*1024, 35) as granularity, - -- cache hits counter can vary a lot depending on other settings: - -- e.g. if merge_tree_min_bytes_for_concurrent_read is small, multiple threads will read each chunk - -- so we just check that the value is not too low - if(event in ( - 'PageCacheBytesUnpinnedRoundedToPages', 'PageCacheBytesUnpinnedRoundedToHugePages', - 'PageCacheChunkDataHits'), 1, 1000) as clamp - select event, min2(intDiv(new.value - old.value, granularity), clamp) as diff - from system.events new - left outer join events_snapshot old - on old.event = new.event - where diff != 0 and - event in ( - 'ReadBufferFromS3Bytes', 'PageCacheChunkMisses', 'PageCacheChunkDataMisses', - 'PageCacheChunkDataHits', 'PageCacheChunkDataPartialHits', - 'PageCacheBytesUnpinnedRoundedToPages', 'PageCacheBytesUnpinnedRoundedToHugePages') - order by event; - -drop table if exists page_cache_03055; -create table page_cache_03055 (k Int64 CODEC(NONE)) engine MergeTree order by k settings storage_policy = 's3_cache'; - --- Write an 80 MiB file (40 x 2 MiB chunks), and a few small files. -system stop merges page_cache_03055; -insert into page_cache_03055 select * from numbers(10485760) settings max_block_size=100000000, preferred_block_size_bytes=1000000000; - -select * from events_diff; -truncate table events_snapshot; -insert into events_snapshot select * from system.events; - -system start merges page_cache_03055; -optimize table page_cache_03055 final; -truncate table events_snapshot; -insert into events_snapshot select * from system.events; - --- Cold read, should miss cache. (Populating cache on write is not implemented yet.) - -select 'cold read', sum(k) from page_cache_03055; - -select * from events_diff where event not in ('PageCacheChunkDataHits'); -truncate table events_snapshot; -insert into events_snapshot select * from system.events; - --- Repeat read, should hit cache. - -select 'repeat read 1', sum(k) from page_cache_03055; - -select * from events_diff; -truncate table events_snapshot; -insert into events_snapshot select * from system.events; - --- Drop cache and read again, should miss. Also don't write to cache. - -system drop page cache; - -select 'dropped and bypassed cache', sum(k) from page_cache_03055 settings read_from_page_cache_if_exists_otherwise_bypass_cache = 1; - --- Data could be read multiple times because we're not writing to cache. --- (Not checking PageCacheBytesUnpinned* because it's unreliable in this case because of an intentional race condition, see PageCache::evictChunk.) -select event, if(event in ('PageCacheChunkMisses', 'ReadBufferFromS3Bytes'), diff >= 1, diff) from events_diff where event not in ('PageCacheChunkDataHits', 'PageCacheBytesUnpinnedRoundedToPages', 'PageCacheBytesUnpinnedRoundedToHugePages'); -truncate table events_snapshot; -insert into events_snapshot select * from system.events; - --- Repeat read, should still miss, but populate cache. - -select 'repeat read 2', sum(k) from page_cache_03055; - -select * from events_diff where event not in ('PageCacheChunkDataHits'); -truncate table events_snapshot; -insert into events_snapshot select * from system.events; - --- Read again, hit the cache. - -select 'repeat read 3', sum(k) from page_cache_03055 settings read_from_page_cache_if_exists_otherwise_bypass_cache = 1; - -select * from events_diff; -truncate table events_snapshot; -insert into events_snapshot select * from system.events; - - --- Known limitation: cache is not invalidated if a table is dropped and created again at the same path. --- set allow_deprecated_database_ordinary=1; --- create database test_03055 engine = Ordinary; --- create table test_03055.t (k Int64) engine MergeTree order by k settings storage_policy = 's3_cache'; --- insert into test_03055.t values (1); --- select * from test_03055.t; --- drop table test_03055.t; --- create table test_03055.t (k Int64) engine MergeTree order by k settings storage_policy = 's3_cache'; --- insert into test_03055.t values (2); --- select * from test_03055.t; - - -drop table events_snapshot; -drop table page_cache_03055; -drop view events_diff; From 9639f62a51557a440ead90031271c692f0a81ef1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 21 Jun 2024 00:35:54 +0200 Subject: [PATCH 293/318] Fix test --- tests/integration/helpers/keeper_utils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index f798482212f..4721aa87725 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -196,7 +196,7 @@ class KeeperClient(object): ) return self.execute_query( - f"reconfig '{operation}' {joining or leaving or new_members}", timeout + f"reconfig '{operation}' '{joining or leaving or new_members}'", timeout ) @classmethod From 3b7d69a2fe49bd9eee8ff763d8ee90f0c26667b1 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Fri, 21 Jun 2024 00:35:52 +0200 Subject: [PATCH 294/318] Remove hostname from `clickhouse-local` prompt --- programs/local/LocalServer.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 4d5cfb09e6a..cb1c35743b2 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -732,11 +732,8 @@ void LocalServer::processConfig() attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); } - server_display_name = config().getString("display_name", getFQDNOrHostName()); - prompt_by_server_display_name = config().getRawString("prompt_by_server_display_name.default", "{display_name} :) "); - std::map prompt_substitutions{{"display_name", server_display_name}}; - for (const auto & [key, value] : prompt_substitutions) - boost::replace_all(prompt_by_server_display_name, "{" + key + "}", value); + server_display_name = config().getString("display_name", ""); + prompt_by_server_display_name = config().getRawString("prompt_by_server_display_name.default", ":) "); global_context->setQueryKindInitial(); global_context->setQueryKind(query_kind); From fa3bdf60503af9011d39edf1de2c28af102cd89d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 21 Jun 2024 09:15:27 +0000 Subject: [PATCH 295/318] Fix style --- tests/ci/changelog.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index d7a46d0a556..0cf1431a912 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -285,10 +285,13 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri entry = item.title # Normalize bug fixes - if re.match( - r"(?i)bug\Wfix", - category, - ) and "Critical Bug Fix" not in category: + if ( + re.match( + r"(?i)bug\Wfix", + category, + ) + and "Critical Bug Fix" not in category + ): category = "Bug Fix (user-visible misbehavior in an official stable release)" if backport_number != item.number: From 8249efb8c718bcc0f856ce7b80968f1805adb1f4 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 21 Jun 2024 09:37:06 +0000 Subject: [PATCH 296/318] Replace setting 'uniform_snowflake_conversion_functions' by 'allow_deprecated_snowflake_conversion_functions' --- docs/en/operations/settings/settings.md | 9 ++-- .../sql-reference/functions/uuid-functions.md | 8 ++-- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.h | 2 +- src/Functions/dateTimeToSnowflakeID.cpp | 27 +----------- src/Functions/snowflake.cpp | 34 +++++++-------- src/Functions/snowflakeIDToDateTime.cpp | 15 +------ .../0_stateless/00515_enhanced_time_zones.sql | 2 +- .../0_stateless/01942_dateTimeToSnowflake.sql | 6 +-- .../01942_dateTimeToSnowflakeID.sql | 4 -- .../01942_snowflakeIDToDateTime.sql | 4 -- .../0_stateless/01942_snowflakeToDateTime.sql | 42 +++++++++---------- 12 files changed, 58 insertions(+), 97 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 3de823321f2..3d6d776f4da 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5418,11 +5418,14 @@ When set to `false` than all attempts are made with identical timeouts. Default value: `true`. -## uniform_snowflake_conversion_functions {#uniform_snowflake_conversion_functions} +## allow_deprecated_snowflake_conversion_functions {#allow_deprecated_snowflake_conversion_functions} -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`). +Functions `snowflakeToDateTime`, `snowflakeToDateTime64`, `dateTimeToSnowflake`, and `dateTime64ToSnowflake` are deprecated and disabled by default. +Please use functions `snowflakeIDToDateTime`, `snowflakeIDToDateTime64`, `dateTimeToSnowflakeID`, and `dateTime64ToSnowflakeID` instead. -Default value: `true` +To re-enable the deprecated functions (e.g., during a transition period), please set this setting to `true`. + +Default value: `false` ## allow_experimental_variant_type {#allow_experimental_variant_type} diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index b7d095c796e..e990023efbc 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -611,7 +611,7 @@ 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. +This function is deprecated and can only be used if setting [allow_deprecated_snowflake_conversion_functions](../../operations/settings/settings.md#allow_deprecated_snowflake_conversion_functions) is enabled. The function will be removed at some point in future. ::: @@ -652,7 +652,7 @@ 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. +This function is deprecated and can only be used if setting [allow_deprecated_snowflake_conversion_functions](../../operations/settings/settings.md#allow_deprecated_snowflake_conversion_functions) is enabled. The function will be removed at some point in future. ::: @@ -693,7 +693,7 @@ 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. +This function is deprecated and can only be used if setting [allow_deprecated_snowflake_conversion_functions](../../operations/settings/settings.md#allow_deprecated_snowflake_conversion_functions) is enabled. The function will be removed at some point in future. ::: @@ -732,7 +732,7 @@ 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. +This function is deprecated and can only be used if setting [allow_deprecated_snowflake_conversion_functions](../../operations/settings/settings.md#allow_deprecated_snowflake_conversion_functions) is enabled. The function will be removed at some point in future. ::: diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 838cb1e0b1c..fbab72446a0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -934,7 +934,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, "Enables functions snowflakeIDToDateTime[64] and dateTime[64]ToSnowflakeID while disabling functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake.", 0) \ + M(Bool, allow_deprecated_snowflake_conversion_functions, false, "Enables deprecated 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. diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 895db9c7ca0..fbc414d4f2f 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -102,7 +102,7 @@ static const std::map(context); } - explicit FunctionDateTimeToSnowflakeID(ContextPtr context) - : uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions) - {} + static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -61,9 +50,6 @@ public: 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; @@ -86,16 +72,10 @@ public: 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) - {} + static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -118,9 +98,6 @@ public: 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(); diff --git a/src/Functions/snowflake.cpp b/src/Functions/snowflake.cpp index 801727e9eb9..5ff8a636058 100644 --- a/src/Functions/snowflake.cpp +++ b/src/Functions/snowflake.cpp @@ -13,7 +13,7 @@ /// ------------------------------------------------------------------------------------------------------------------------------ /// 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. +/// 'dateTime[64]ToSnowflakeID' by summer 2025. Please also mark setting `allow_deprecated_snowflake_conversion_functions` as obsolete then. /// ------------------------------------------------------------------------------------------------------------------------------ namespace DB @@ -40,7 +40,7 @@ constexpr int time_shift = 22; class FunctionDateTimeToSnowflake : public IFunction { private: - const bool uniform_snowflake_conversion_functions; + const bool allow_deprecated_snowflake_conversion_functions; public: static constexpr auto name = "dateTimeToSnowflake"; @@ -51,7 +51,7 @@ public: } explicit FunctionDateTimeToSnowflake(ContextPtr context) - : uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions) + : allow_deprecated_snowflake_conversion_functions(context->getSettingsRef().allow_deprecated_snowflake_conversion_functions) {} String getName() const override { return name; } @@ -71,8 +71,8 @@ public: 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()); + if (!allow_deprecated_snowflake_conversion_functions) + throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it set setting 'allow_deprecated_snowflake_conversion_functions' to 'true'", getName()); const auto & src = arguments[0]; const auto & src_column = *src.column; @@ -92,7 +92,7 @@ class FunctionSnowflakeToDateTime : public IFunction { private: const bool allow_nonconst_timezone_arguments; - const bool uniform_snowflake_conversion_functions; + const bool allow_deprecated_snowflake_conversion_functions; public: static constexpr auto name = "snowflakeToDateTime"; @@ -104,7 +104,7 @@ public: explicit FunctionSnowflakeToDateTime(ContextPtr context) : allow_nonconst_timezone_arguments(context->getSettingsRef().allow_nonconst_timezone_arguments) - , uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions) + , allow_deprecated_snowflake_conversion_functions(context->getSettingsRef().allow_deprecated_snowflake_conversion_functions) {} String getName() const override { return name; } @@ -132,8 +132,8 @@ public: 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()); + if (!allow_deprecated_snowflake_conversion_functions) + throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it set setting 'allow_deprecated_snowflake_conversion_functions' to 'true'", getName()); const auto & src = arguments[0]; const auto & src_column = *src.column; @@ -166,7 +166,7 @@ public: class FunctionDateTime64ToSnowflake : public IFunction { private: - const bool uniform_snowflake_conversion_functions; + const bool allow_deprecated_snowflake_conversion_functions; public: static constexpr auto name = "dateTime64ToSnowflake"; @@ -177,7 +177,7 @@ public: } explicit FunctionDateTime64ToSnowflake(ContextPtr context) - : uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions) + : allow_deprecated_snowflake_conversion_functions(context->getSettingsRef().allow_deprecated_snowflake_conversion_functions) {} String getName() const override { return name; } @@ -197,8 +197,8 @@ public: 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()); + if (!allow_deprecated_snowflake_conversion_functions) + throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it set setting 'allow_deprecated_snowflake_conversion_functions' to true", getName()); const auto & src = arguments[0]; @@ -226,7 +226,7 @@ class FunctionSnowflakeToDateTime64 : public IFunction { private: const bool allow_nonconst_timezone_arguments; - const bool uniform_snowflake_conversion_functions; + const bool allow_deprecated_snowflake_conversion_functions; public: static constexpr auto name = "snowflakeToDateTime64"; @@ -238,7 +238,7 @@ public: explicit FunctionSnowflakeToDateTime64(ContextPtr context) : allow_nonconst_timezone_arguments(context->getSettingsRef().allow_nonconst_timezone_arguments) - , uniform_snowflake_conversion_functions(context->getSettingsRef().uniform_snowflake_conversion_functions) + , allow_deprecated_snowflake_conversion_functions(context->getSettingsRef().allow_deprecated_snowflake_conversion_functions) {} String getName() const override { return name; } @@ -266,8 +266,8 @@ public: 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()); + if (!allow_deprecated_snowflake_conversion_functions) + throw Exception(ErrorCodes::DEPRECATED_FUNCTION, "Function {} is deprecated, to enable it set setting 'allow_deprecated_snowflake_conversion_functions' to true", getName()); const auto & src = arguments[0]; const auto & src_column = *src.column; diff --git a/src/Functions/snowflakeIDToDateTime.cpp b/src/Functions/snowflakeIDToDateTime.cpp index abaf09b165b..12023431a71 100644 --- a/src/Functions/snowflakeIDToDateTime.cpp +++ b/src/Functions/snowflakeIDToDateTime.cpp @@ -18,7 +18,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int UNKNOWN_FUNCTION; } namespace @@ -32,7 +31,6 @@ constexpr int time_shift = 22; class FunctionSnowflakeIDToDateTime : public IFunction { private: - const bool uniform_snowflake_conversion_functions; const bool allow_nonconst_timezone_arguments; public: @@ -40,8 +38,7 @@ public: 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) + : allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments) {} String getName() const override { return name; } @@ -70,9 +67,6 @@ public: 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; @@ -108,7 +102,6 @@ public: class FunctionSnowflakeIDToDateTime64 : public IFunction { private: - const bool uniform_snowflake_conversion_functions; const bool allow_nonconst_timezone_arguments; public: @@ -116,8 +109,7 @@ public: 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) + : allow_nonconst_timezone_arguments(context->getSettings().allow_nonconst_timezone_arguments) {} String getName() const override { return name; } @@ -146,9 +138,6 @@ public: 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; diff --git a/tests/queries/0_stateless/00515_enhanced_time_zones.sql b/tests/queries/0_stateless/00515_enhanced_time_zones.sql index e39b618b670..f7eb90fa5c8 100644 --- a/tests/queries/0_stateless/00515_enhanced_time_zones.sql +++ b/tests/queries/0_stateless/00515_enhanced_time_zones.sql @@ -1,4 +1,4 @@ -SET uniform_snowflake_conversion_functions = 0; +SET allow_deprecated_snowflake_conversion_functions = 1; 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); diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql index 0386717c933..6cce4863c15 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflake.sql @@ -1,4 +1,4 @@ -SET uniform_snowflake_conversion_functions = 0; -- Force-disable uniform snowflake conversion functions (in case this is randomized in CI) +SET allow_deprecated_snowflake_conversion_functions = 1; -- Force-enable deprecated snowflake conversion functions (in case this is randomized in CI) SET session_timezone = 'Africa/Juba'; -- Error cases @@ -11,8 +11,8 @@ 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 dateTimeToSnowflake(now()) SETTINGS allow_deprecated_snowflake_conversion_functions = 0; -- { serverError DEPRECATED_FUNCTION } +SELECT dateTime64ToSnowflake(now64()) SETTINGS allow_deprecated_snowflake_conversion_functions = 0; -- { serverError DEPRECATED_FUNCTION } SELECT '-- const / non-const inputs'; diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql index 33bac8aaa35..945b399157f 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql @@ -1,6 +1,5 @@ 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} @@ -12,9 +11,6 @@ SELECT dateTime64ToSnowflakeID(now64(), 'invalid_epoch'); -- {serverError ILLEG 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())); diff --git a/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql index b0e244ef814..48316691c71 100644 --- a/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql +++ b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql @@ -1,6 +1,5 @@ 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} @@ -16,9 +15,6 @@ SELECT snowflakeIDToDateTime64(123::UInt64, 42, 42); -- {serverError ILLEGAL_TY 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)); diff --git a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql index 1729a50ae44..34fe15ec187 100644 --- a/tests/queries/0_stateless/01942_snowflakeToDateTime.sql +++ b/tests/queries/0_stateless/01942_snowflakeToDateTime.sql @@ -1,6 +1,6 @@ -SET uniform_snowflake_conversion_functions = 0; -- Force-disable uniform snowflake conversion functions (in case this is randomized in CI) +SET allow_deprecated_snowflake_conversion_functions = 1; -- Force-enable deprecated snowflake conversion functions (in case this is randomized in CI) --- -- Error cases +-- Error cases SELECT snowflakeToDateTime(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} SELECT snowflakeToDateTime64(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} @@ -10,35 +10,35 @@ 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 snowflakeToDateTime(123::Int64) SETTINGS allow_deprecated_snowflake_conversion_functions = 0; -- { serverError DEPRECATED_FUNCTION } +SELECT snowflakeToDateTime64(123::Int64) SETTINGS allow_deprecated_snowflake_conversion_functions = 0; -- { serverError DEPRECATED_FUNCTION } SELECT 'const column'; WITH - CAST(1426860704886947840 AS Int64) AS i64, - 'UTC' AS tz + CAST(1426860704886947840 AS Int64) AS i64, + 'UTC' AS tz SELECT - tz, - i64, - snowflakeToDateTime(i64, tz) as dt, - toTypeName(dt), - snowflakeToDateTime64(i64, tz) as dt64, - toTypeName(dt64); + tz, + i64, + snowflakeToDateTime(i64, tz) as dt, + toTypeName(dt), + snowflakeToDateTime64(i64, tz) as dt64, + toTypeName(dt64); WITH - CAST(1426860704886947840 AS Int64) AS i64, - 'Asia/Shanghai' AS tz + CAST(1426860704886947840 AS Int64) AS i64, + 'Asia/Shanghai' AS tz SELECT - tz, - i64, - snowflakeToDateTime(i64, tz) as dt, - toTypeName(dt), - snowflakeToDateTime64(i64, tz) as dt64, - toTypeName(dt64); + tz, + i64, + snowflakeToDateTime(i64, tz) as dt, + toTypeName(dt), + snowflakeToDateTime64(i64, tz) as dt64, + toTypeName(dt64); DROP TABLE IF EXISTS tab; -CREATE TABLE tab(val Int64, tz String) engine=Log; +CREATE TABLE tab(val Int64, tz String) engine = Log; INSERT INTO tab VALUES (42, 'Asia/Singapore'); SELECT 1 FROM tab WHERE snowflakeToDateTime(42::Int64, tz) != now() SETTINGS allow_nonconst_timezone_arguments = 1; From 26aa65b9d17ddabbe1f0afaf009cf7360ff8dca0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 21 Jun 2024 09:49:26 +0000 Subject: [PATCH 297/318] Add 'Experimental Feature' category --- .github/PULL_REQUEST_TEMPLATE.md | 1 + tests/ci/changelog.py | 1 + 2 files changed, 2 insertions(+) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index d9f9e9d6c8b..e045170561d 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -6,6 +6,7 @@ tests/ci/cancel_and_rerun_workflow_lambda/app.py --> ### Changelog category (leave one): - New Feature +- Experimental Feature - Improvement - Performance Improvement - Backward Incompatible Change diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index 0cf1431a912..3ba618f3ae5 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -33,6 +33,7 @@ from version_helper import ( categories_preferred_order = ( "Backward Incompatible Change", "New Feature", + "Experimental Feature", "Performance Improvement", "Improvement", "Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC)", From 16457c67b9284da2c1b61e91cce4f402353fd1fa Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 21 Jun 2024 12:16:16 +0200 Subject: [PATCH 298/318] Fix --- .../StorageObjectStorageSource.cpp | 33 ++++++++++--------- 1 file changed, 18 insertions(+), 15 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 2fc6993369d..bc633d5db83 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -517,24 +517,22 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne else ++it; } + + if (filter_dag) + { + std::vector paths; + paths.reserve(new_batch.size()); + for (const auto & object_info : new_batch) + paths.push_back(getUniqueStoragePathIdentifier(*configuration, *object_info, false)); + + VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_dag, virtual_columns, getContext()); + + LOG_TEST(logger, "Filtered files: {} -> {}", paths.size(), new_batch.size()); + } } index = 0; - if (filter_dag) - { - std::vector paths; - paths.reserve(new_batch.size()); - for (const auto & object_info : new_batch) - { - chassert(object_info); - paths.push_back(getUniqueStoragePathIdentifier(*configuration, *object_info, false)); - } - - VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_dag, virtual_columns, getContext()); - LOG_TEST(logger, "Filtered files: {} -> {}", paths.size(), new_batch.size()); - } - if (read_keys) read_keys->insert(read_keys->end(), new_batch.begin(), new_batch.end()); @@ -551,7 +549,12 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne } if (index >= object_infos.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Index out of bound for blob metadata"); + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Index out of bound for blob metadata. Index: {}, size: {}", + index, object_infos.size()); + } return object_infos[index++]; } From 4064ca018f4d74813d2cd5d20d5966fb98563dd7 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 21 Jun 2024 12:19:26 +0200 Subject: [PATCH 299/318] CI: Try Rerun check removal --- tests/ci/ci.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 4e34e6b6135..77305542bbc 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1065,18 +1065,19 @@ def main() -> int: ) # rerun helper check - # FIXME: remove rerun_helper check and rely on ci cache only if check_name not in ( 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(): + print("WARNING: Rerunning job with GH status ") status = rerun_helper.get_finished_status() assert status - previous_status = status.state print("::group::Commit Status") print(status) print("::endgroup::") + # FIXME: try rerun, even if status is present. To enable manual restart via GH interface + # previous_status = status.state # ci cache check if not previous_status and not ci_settings.no_ci_cache: From df857ff17ad2645eafff0cfa0e9c7de77222956a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 21 Jun 2024 10:22:10 +0000 Subject: [PATCH 300/318] Bump re2 to latest HEAD --- contrib/re2 | 2 +- contrib/re2-cmake/CMakeLists.txt | 12 ++++++++---- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/contrib/re2 b/contrib/re2 index a807e8a3aac..85dd7ad833a 160000 --- a/contrib/re2 +++ b/contrib/re2 @@ -1 +1 @@ -Subproject commit a807e8a3aac2cc33c77b7071efea54fcabe38e0c +Subproject commit 85dd7ad833a73095ecf3e3baea608ba051bbe2c7 diff --git a/contrib/re2-cmake/CMakeLists.txt b/contrib/re2-cmake/CMakeLists.txt index f773bc65a69..99d61839b30 100644 --- a/contrib/re2-cmake/CMakeLists.txt +++ b/contrib/re2-cmake/CMakeLists.txt @@ -28,16 +28,20 @@ set(RE2_SOURCES add_library(_re2 ${RE2_SOURCES}) target_include_directories(_re2 PUBLIC "${SRC_DIR}") target_link_libraries(_re2 PRIVATE + absl::absl_check + absl::absl_log absl::base absl::core_headers absl::fixed_array + absl::flags absl::flat_hash_map absl::flat_hash_set + absl::hash absl::inlined_vector - absl::strings - absl::str_format - absl::synchronization absl::optional - absl::span) + absl::span + absl::str_format + absl::strings + absl::synchronization) add_library(ch_contrib::re2 ALIAS _re2) From 0a60dc1672a05c13110f6992af4e0f2fcac19dde Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 21 Jun 2024 12:57:44 +0000 Subject: [PATCH 301/318] OpenSSL: Replace temporary fix for unsynchronized access by official fix --- contrib/openssl | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/openssl b/contrib/openssl index 277de2ba202..5d81fa7068f 160000 --- a/contrib/openssl +++ b/contrib/openssl @@ -1 +1 @@ -Subproject commit 277de2ba202af4eb2291b363456d32ff0960e559 +Subproject commit 5d81fa7068fc8c07f4d0997d5b703f3c541a637c From f0ae3a7c9f76abeae6f100f0a1bf2bedc060f96c Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Fri, 21 Jun 2024 15:05:31 +0200 Subject: [PATCH 302/318] Update README.md Updating events. --- README.md | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/README.md b/README.md index 73d989210b5..119551d69b2 100644 --- a/README.md +++ b/README.md @@ -34,20 +34,18 @@ curl https://clickhouse.com/ | sh Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know. -* [v24.5 Community Call](https://clickhouse.com/company/events/v24-5-community-release-call) - May 30 +* [v24.6 Community Call](https://clickhouse.com/company/events/v24-5-community-release-call) - Jul 2 ## Upcoming Events Keep an eye out for upcoming meetups and events around the world. Somewhere else you want us to be? Please feel free to reach out to tyler `` clickhouse `` com. You can also peruse [ClickHouse Events](https://clickhouse.com/company/news-events) for a list of all upcoming trainings, meetups, speaking engagements, etc. -* [ClickHouse Happy Hour @ Tom's Watch Bar - Los Angeles](https://www.meetup.com/clickhouse-los-angeles-user-group/events/300740584/) - May 22 -* [ClickHouse & Confluent Meetup in Dubai](https://www.meetup.com/clickhouse-dubai-meetup-group/events/299629189/) - May 28 -* [ClickHouse Meetup in Stockholm](https://www.meetup.com/clickhouse-stockholm-user-group/events/299752651/) - Jun 3 -* [ClickHouse Meetup @ Cloudflare - San Francisco](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/300523061/) - Jun 4 -* [ClickHouse (クリックハウス) Meetup Tokyo](https://www.meetup.com/clickhouse-tokyo-user-group/events/300798053/) - Jun 5 +* [AWS Summit in DC](https://clickhouse.com/company/events/2024-06-aws-summit-dc) - Jun 26 * [ClickHouse Meetup in Amsterdam](https://www.meetup.com/clickhouse-netherlands-user-group/events/300781068/) - Jun 27 * [ClickHouse Meetup in Paris](https://www.meetup.com/clickhouse-france-user-group/events/300783448/) - Jul 9 +* [ClickHouse Cloud - Live Update Call](https://clickhouse.com/company/events/202407-cloud-update-live) - Jul 9 * [ClickHouse Meetup @ Ramp - New York City](https://www.meetup.com/clickhouse-new-york-user-group/events/300595845/) - Jul 9 +* [AWS Summit in New York](https://clickhouse.com/company/events/2024-07-awssummit-nyc) - Jul 10 * [ClickHouse Meetup @ Klaviyo - Boston](https://www.meetup.com/clickhouse-boston-user-group/events/300907870) - Jul 11 ## Recent Recordings From debeb369278fb1afc81f5bb11122a9891fd2fc9c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 21 Jun 2024 16:10:02 +0200 Subject: [PATCH 303/318] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 119551d69b2..dc253d4db2d 100644 --- a/README.md +++ b/README.md @@ -34,7 +34,7 @@ curl https://clickhouse.com/ | sh Every month we get together with the community (users, contributors, customers, those interested in learning more about ClickHouse) to discuss what is coming in the latest release. If you are interested in sharing what you've built on ClickHouse, let us know. -* [v24.6 Community Call](https://clickhouse.com/company/events/v24-5-community-release-call) - Jul 2 +* [v24.6 Community Call](https://clickhouse.com/company/events/v24-6-community-release-call) - Jul 2 ## Upcoming Events From 803b58cdfa4000622e7f712541c87be006e015d7 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 21 Jun 2024 16:44:20 +0200 Subject: [PATCH 304/318] remove tables in test, make that test non-parrallel --- .../queries/0_stateless/01526_client_start_and_exit.reference | 1 + tests/queries/0_stateless/01526_client_start_and_exit.sh | 4 +++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01526_client_start_and_exit.reference b/tests/queries/0_stateless/01526_client_start_and_exit.reference index e3e2e7b22af..35e453e9f82 100644 --- a/tests/queries/0_stateless/01526_client_start_and_exit.reference +++ b/tests/queries/0_stateless/01526_client_start_and_exit.reference @@ -1 +1,2 @@ Loaded 10000 queries. +Loaded 10000 queries. diff --git a/tests/queries/0_stateless/01526_client_start_and_exit.sh b/tests/queries/0_stateless/01526_client_start_and_exit.sh index 0c5c94e3eac..5f3a8c9bb2d 100755 --- a/tests/queries/0_stateless/01526_client_start_and_exit.sh +++ b/tests/queries/0_stateless/01526_client_start_and_exit.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest +# Tags: no-fasttest, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -26,3 +26,5 @@ for _ in {1..10}; do done wait + +${CLICKHOUSE_CLIENT} -q "select 'DROP TABLE ' || database || '.' || name || ';' from system.tables WHERE database = '${CLICKHOUSE_DATABASE}'" --format=TSVRaw | ${CLICKHOUSE_BENCHMARK} -c32 -i 10000 -d 0 2>&1 | grep -F 'Loaded 10000 queries' From e278584c017bc4bac1d887411d7e28fc2af3e37b Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 21 Jun 2024 16:46:03 +0200 Subject: [PATCH 305/318] Add a test --- tests/integration/test_storage_s3/test.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index b2ebd12ce00..fd5ce10e3f5 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2196,6 +2196,11 @@ def test_filtering_by_file_or_path(started_cluster): assert int(result) == 1 + assert 0 == int(instance.query( + f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_filter*.tsv') where _file = 'kek'" + )) + + def test_union_schema_inference_mode(started_cluster): bucket = started_cluster.minio_bucket From 581cc540ee2cb654a0c98d91f44aa6bf35ed46e0 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 21 Jun 2024 14:53:45 +0000 Subject: [PATCH 306/318] Automatic style fix --- tests/integration/test_storage_s3/test.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index fd5ce10e3f5..9a0cb352088 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -2196,10 +2196,11 @@ def test_filtering_by_file_or_path(started_cluster): assert int(result) == 1 - assert 0 == int(instance.query( - f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_filter*.tsv') where _file = 'kek'" - )) - + assert 0 == int( + instance.query( + f"select count() from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_filter*.tsv') where _file = 'kek'" + ) + ) def test_union_schema_inference_mode(started_cluster): From dd862d0abb333e9c095226b56e2d431e6615711d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 21 Jun 2024 15:16:14 +0000 Subject: [PATCH 307/318] add test --- .../AzureBlobStorage/AzureBlobStorageAuth.cpp | 2 +- .../test_storage_azure_blob_storage/test.py | 39 +++++++++++++++++-- 2 files changed, 36 insertions(+), 5 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index bae58f0b9c6..1a5388349f8 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -181,7 +181,7 @@ std::unique_ptr getAzureBlobStorageClientWithAuth( if (config.getBool(config_prefix + ".use_workload_identity", false)) { auto workload_identity_credential = std::make_shared(); - return std::make_unique(url, workload_identity_credential); + return std::make_unique(url, workload_identity_credential, client_options); } auto managed_identity_credential = std::make_shared(); diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index d986c1f9746..bcc06de9f5e 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -10,13 +10,10 @@ import threading import time from azure.storage.blob import BlobServiceClient -import helpers.client import pytest from helpers.cluster import ClickHouseCluster, ClickHouseInstance -from helpers.network import PartitionManager -from helpers.mock_servers import start_mock_servers -from helpers.test_tools import exec_query_with_retry from helpers.test_tools import assert_logs_contain_with_retry +from helpers.test_tools import TSV @pytest.fixture(scope="module") @@ -1431,3 +1428,37 @@ def test_respect_object_existence_on_partitioned_write(cluster): ) assert int(result) == 44 + + +def test_insert_create_new_file(cluster): + node = cluster.instances["node"] + storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"] + account_name = "devstoreaccount1" + account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_create_new_file.csv', '{account_name}', '{account_key}', 'a UInt64') VALUES (1)", + settings={ + "azure_truncate_on_insert": False, + "azure_create_new_file_on_insert": True, + }, + ) + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_create_new_file.csv', '{account_name}', '{account_key}', 'a UInt64') VALUES (2)", + settings={ + "azure_truncate_on_insert": False, + "azure_create_new_file_on_insert": True, + }, + ) + + res = azure_query( + node, + f"SELECT _file, * FROM azureBlobStorage('{storage_account_url}', 'cont', 'test_create_new_file*', '{account_name}', '{account_key}', 'a UInt64') ORDER BY a", + ) + + assert TSV(res) == TSV( + "test_create_new_file.csv\t1\ntest_create_new_file.1.csv\t2\n" + ) From 84ac7961522c540a46fccbac75661bf5b9e1d28c Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Fri, 21 Jun 2024 20:14:03 +0200 Subject: [PATCH 308/318] delete 01526_client_start_and_exit.sh --- ...ient_start_and_exit.expect-not-a-test-case | 10 ------- .../01526_client_start_and_exit.reference | 2 -- .../01526_client_start_and_exit.sh | 30 ------------------- 3 files changed, 42 deletions(-) delete mode 100755 tests/queries/0_stateless/01526_client_start_and_exit.expect-not-a-test-case delete mode 100644 tests/queries/0_stateless/01526_client_start_and_exit.reference delete mode 100755 tests/queries/0_stateless/01526_client_start_and_exit.sh diff --git a/tests/queries/0_stateless/01526_client_start_and_exit.expect-not-a-test-case b/tests/queries/0_stateless/01526_client_start_and_exit.expect-not-a-test-case deleted file mode 100755 index 00fb5c4e85b..00000000000 --- a/tests/queries/0_stateless/01526_client_start_and_exit.expect-not-a-test-case +++ /dev/null @@ -1,10 +0,0 @@ -#!/usr/bin/expect -f - -log_user 1 -set timeout 5 -match_max 100000 - -spawn bash -c "$env(CLICKHOUSE_CLIENT_BINARY) --no-warnings $env(CLICKHOUSE_CLIENT_OPT)" -expect ":) " -send -- "\4" -expect eof diff --git a/tests/queries/0_stateless/01526_client_start_and_exit.reference b/tests/queries/0_stateless/01526_client_start_and_exit.reference deleted file mode 100644 index 35e453e9f82..00000000000 --- a/tests/queries/0_stateless/01526_client_start_and_exit.reference +++ /dev/null @@ -1,2 +0,0 @@ -Loaded 10000 queries. -Loaded 10000 queries. diff --git a/tests/queries/0_stateless/01526_client_start_and_exit.sh b/tests/queries/0_stateless/01526_client_start_and_exit.sh deleted file mode 100755 index 5f3a8c9bb2d..00000000000 --- a/tests/queries/0_stateless/01526_client_start_and_exit.sh +++ /dev/null @@ -1,30 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, no-parallel - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -# Create a huge amount of tables, so Suggest will take a time to load -${CLICKHOUSE_CLIENT} -q "SELECT 'CREATE TABLE test_' || hex(randomPrintableASCII(40)) || '(x UInt8) Engine=Memory;' FROM numbers(10000)" --format=TSVRaw | ${CLICKHOUSE_BENCHMARK} -c32 -i 10000 -d 0 2>&1 | grep -F 'Loaded 10000 queries' - -function stress() -{ - # 2004l is ignored because parallel running expect emulated terminal doesn't - # work well with bracketed paste enabling sequence, which is \e033?2004l - # (https://cirw.in/blog/bracketed-paste) - while true; do - "${CURDIR}"/01526_client_start_and_exit.expect-not-a-test-case | grep -v -P 'ClickHouse client|Connecting|Connected|:\) Bye\.|new year|^\s*$|spawn bash|\?2004l|^0\s*$' - done -} - -export CURDIR -export -f stress - -for _ in {1..10}; do - timeout 3 bash -c stress & -done - -wait - -${CLICKHOUSE_CLIENT} -q "select 'DROP TABLE ' || database || '.' || name || ';' from system.tables WHERE database = '${CLICKHOUSE_DATABASE}'" --format=TSVRaw | ${CLICKHOUSE_BENCHMARK} -c32 -i 10000 -d 0 2>&1 | grep -F 'Loaded 10000 queries' From c318b773da99568dd27e2055c8aa2fb138e60060 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 22 Jun 2024 11:35:52 +0200 Subject: [PATCH 309/318] Remove note about stability for varpop --- docs/en/sql-reference/aggregate-functions/reference/varpop.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/varpop.md b/docs/en/sql-reference/aggregate-functions/reference/varpop.md index be57115f146..6ea6748f507 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varpop.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varpop.md @@ -8,10 +8,6 @@ sidebar_position: 32 Calculates the population variance. -:::note -This function uses a numerically unstable algorithm. If you need [numerical stability](https://en.wikipedia.org/wiki/Numerical_stability) in calculations, use the [`varPopStable`](../reference/varpopstable.md) function. It works slower but provides a lower computational error. -::: - **Syntax** ```sql From c9b4f4ecaa778a5b22bbde4d49fe053c7cdc0b33 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Sat, 22 Jun 2024 11:36:18 +0200 Subject: [PATCH 310/318] Remove note about stability for varsamp --- .../en/sql-reference/aggregate-functions/reference/varsamp.md | 4 ---- 1 file changed, 4 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/varsamp.md b/docs/en/sql-reference/aggregate-functions/reference/varsamp.md index 0feb1bac283..24cc5012edb 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/varsamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/varsamp.md @@ -8,10 +8,6 @@ sidebar_position: 33 Calculate the sample variance of a data set. -:::note -This function uses a numerically unstable algorithm. If you need [numerical stability](https://en.wikipedia.org/wiki/Numerical_stability) in calculations, use the [`varSampStable`](../reference/varsampstable.md) function. It works slower but provides a lower computational error. -::: - **Syntax** ```sql From 098df5cef0c8658565decd4d3832b538d32cea8f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jun 2024 02:30:59 +0200 Subject: [PATCH 311/318] Fix flaky test `01254_dict_load_after_detach_attach.sql` --- .../queries/0_stateless/01254_dict_load_after_detach_attach.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql index c11cb64735c..206ddeac612 100644 --- a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql +++ b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.sql @@ -17,7 +17,7 @@ LAYOUT(FLAT()); DETACH DATABASE dict_db_01254; ATTACH DATABASE dict_db_01254; -SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict'; +SELECT COALESCE((SELECT status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict')::Nullable(String), 'NOT_LOADED'); SYSTEM RELOAD DICTIONARY dict_db_01254.dict; SELECT query_count, status FROM system.dictionaries WHERE database = 'dict_db_01254' AND name = 'dict'; SELECT dictGetUInt64('dict_db_01254.dict', 'val', toUInt64(0)); From 824716efef080dec14d80e1b962086cb7f992012 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jun 2024 02:46:58 +0200 Subject: [PATCH 312/318] Fix tests --- tests/integration/helpers/keeper_utils.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index 4721aa87725..be710db37d1 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -137,7 +137,7 @@ class KeeperClient(object): def set(self, path: str, value: str, version: tp.Optional[int] = None) -> None: self.execute_query( - f"set '{path}' '{value}' '{version if version is not None else ''}'" + f"set '{path}' '{value}' {version if version is not None else ''}" ) def rm(self, path: str, version: tp.Optional[int] = None) -> None: @@ -196,7 +196,7 @@ class KeeperClient(object): ) return self.execute_query( - f"reconfig '{operation}' '{joining or leaving or new_members}'", timeout + f"reconfig {operation} '{joining or leaving or new_members}'", timeout ) @classmethod From 4e1a68f33dccd2591b04e3b280cf69a68d21a9c7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 23 Jun 2024 10:54:53 +0000 Subject: [PATCH 313/318] Show compiled expressions cache settings size in system.server_settings --- programs/server/Server.cpp | 4 ++-- src/Core/ServerSettings.h | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 7bc2be806f7..6f17faaf2e7 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1380,8 +1380,8 @@ try global_context->setQueryCache(query_cache_max_size_in_bytes, query_cache_max_entries, query_cache_query_cache_max_entry_size_in_bytes, query_cache_max_entry_size_in_rows); #if USE_EMBEDDED_COMPILER - size_t compiled_expression_cache_max_size_in_bytes = config().getUInt64("compiled_expression_cache_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE); - size_t compiled_expression_cache_max_elements = config().getUInt64("compiled_expression_cache_elements_size", DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES); + size_t compiled_expression_cache_max_size_in_bytes = server_settings.compiled_expression_cache_size; + size_t compiled_expression_cache_max_elements = server_settings.compiled_expression_cache_elements_size; CompiledExpressionCacheFactory::instance().init(compiled_expression_cache_max_size_in_bytes, compiled_expression_cache_max_elements); #endif diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 2dec122425e..9e3de87d3b9 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -90,6 +90,8 @@ namespace DB M(Bool, page_cache_use_madv_free, DBMS_DEFAULT_PAGE_CACHE_USE_MADV_FREE, "If true, the userspace page cache will allow the OS to automatically reclaim memory from the cache on memory pressure (using MADV_FREE).", 0) \ M(Bool, page_cache_use_transparent_huge_pages, true, "Userspace will attempt to use transparent huge pages on Linux. This is best-effort.", 0) \ M(UInt64, mmap_cache_size, DEFAULT_MMAP_CACHE_MAX_SIZE, "A cache for mmapped files.", 0) \ + M(UInt64, compiled_expression_cache_size, DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_SIZE, "Byte size of compiled expressions cache.", 0) \ + M(UInt64, compiled_expression_cache_elements_size, DEFAULT_COMPILED_EXPRESSION_CACHE_MAX_ENTRIES, "Maximum entries in compiled expressions cache.", 0) \ \ M(Bool, disable_internal_dns_cache, false, "Disable internal DNS caching at all.", 0) \ M(UInt64, dns_cache_max_entries, 10000, "Internal DNS cache max entries.", 0) \ From f3ce055e73dbdb564b015184811cd5a32315cceb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 23 Jun 2024 11:38:27 +0000 Subject: [PATCH 314/318] Fix ubsan issue --- src/Functions/dateTimeToSnowflakeID.cpp | 12 ++++++------ src/Functions/snowflakeIDToDateTime.cpp | 10 +++++----- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Functions/dateTimeToSnowflakeID.cpp b/src/Functions/dateTimeToSnowflakeID.cpp index 01467aebe83..968a7628ca5 100644 --- a/src/Functions/dateTimeToSnowflakeID.cpp +++ b/src/Functions/dateTimeToSnowflakeID.cpp @@ -18,7 +18,7 @@ namespace { /// See generateSnowflakeID.cpp -constexpr int time_shift = 22; +constexpr size_t time_shift = 22; } @@ -41,7 +41,7 @@ public: {"value", static_cast(&isDateTime), nullptr, "DateTime"} }; FunctionArgumentDescriptors optional_args{ - {"epoch", static_cast(&isNativeUInt), isColumnConst, "UInt*"} + {"epoch", static_cast(&isNativeUInt), isColumnConst, "const UInt*"} }; validateFunctionArgumentTypes(*this, arguments, args, optional_args); @@ -52,7 +52,7 @@ public: { const auto & col_src = *arguments[0].column; - size_t epoch = 0; + UInt64 epoch = 0; if (arguments.size() == 2 && input_rows_count != 0) { const auto & col_epoch = *arguments[1].column; @@ -89,7 +89,7 @@ public: {"value", static_cast(&isDateTime64), nullptr, "DateTime64"} }; FunctionArgumentDescriptors optional_args{ - {"epoch", static_cast(&isNativeUInt), isColumnConst, "UInt*"} + {"epoch", static_cast(&isNativeUInt), isColumnConst, "const UInt*"} }; validateFunctionArgumentTypes(*this, arguments, args, optional_args); @@ -101,7 +101,7 @@ public: const auto & col_src = *arguments[0].column; const auto & src_data = typeid_cast(col_src).getData(); - size_t epoch = 0; + UInt64 epoch = 0; if (arguments.size() == 2 && input_rows_count != 0) { const auto & col_epoch = *arguments[1].column; @@ -118,7 +118,7 @@ public: 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; + res_data[i] = std::llround(src_data[i] * factor - epoch) << time_shift; return col_res; } diff --git a/src/Functions/snowflakeIDToDateTime.cpp b/src/Functions/snowflakeIDToDateTime.cpp index 12023431a71..b799792a56f 100644 --- a/src/Functions/snowflakeIDToDateTime.cpp +++ b/src/Functions/snowflakeIDToDateTime.cpp @@ -24,7 +24,7 @@ namespace { /// See generateSnowflakeID.cpp -constexpr int time_shift = 22; +constexpr size_t time_shift = 22; } @@ -53,7 +53,7 @@ public: {"value", static_cast(&isUInt64), nullptr, "UInt64"} }; FunctionArgumentDescriptors optional_args{ - {"epoch", static_cast(&isNativeUInt), isColumnConst, "UInt*"}, + {"epoch", static_cast(&isNativeUInt), isColumnConst, "const UInt*"}, {"time_zone", static_cast(&isString), nullptr, "String"} }; validateFunctionArgumentTypes(*this, arguments, args, optional_args); @@ -69,7 +69,7 @@ public: { const auto & col_src = *arguments[0].column; - size_t epoch = 0; + UInt64 epoch = 0; if (arguments.size() >= 2 && input_rows_count != 0) { const auto & col_epoch = *arguments[1].column; @@ -124,7 +124,7 @@ public: {"value", static_cast(&isUInt64), nullptr, "UInt64"} }; FunctionArgumentDescriptors optional_args{ - {"epoch", static_cast(&isNativeUInt), isColumnConst, "UInt*"}, + {"epoch", static_cast(&isNativeUInt), isColumnConst, "const UInt*"}, {"time_zone", static_cast(&isString), nullptr, "String"} }; validateFunctionArgumentTypes(*this, arguments, args, optional_args); @@ -140,7 +140,7 @@ public: { const auto & col_src = *arguments[0].column; - size_t epoch = 0; + UInt64 epoch = 0; if (arguments.size() >= 2 && input_rows_count != 0) { const auto & col_epoch = *arguments[1].column; From b79d3d521ce2f0c38787021a18a5f6f82abaae12 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jun 2024 17:36:27 +0200 Subject: [PATCH 315/318] Improve flaky test to provide more diagnostics --- .../02210_processors_profile_log.reference | 30 ------------------- .../02210_processors_profile_log.sql | 7 ++--- 2 files changed, 3 insertions(+), 34 deletions(-) diff --git a/tests/queries/0_stateless/02210_processors_profile_log.reference b/tests/queries/0_stateless/02210_processors_profile_log.reference index 41543d0706a..035bd9897ad 100644 --- a/tests/queries/0_stateless/02210_processors_profile_log.reference +++ b/tests/queries/0_stateless/02210_processors_profile_log.reference @@ -1,38 +1,8 @@ --- { echo } -EXPLAIN PIPELINE SELECT sleep(1); (Expression) ExpressionTransform (ReadFromStorage) SourceFromSingleChunk 0 → 1 -SELECT sleep(1) SETTINGS log_processors_profiles=true, log_queries=1, log_queries_min_type='QUERY_FINISH'; 0 -SYSTEM FLUSH LOGS; -WITH - ( - SELECT query_id - FROM system.query_log - WHERE current_database = currentDatabase() AND Settings['log_processors_profiles']='1' - ) AS query_id_ -SELECT - name, - multiIf( - -- ExpressionTransform executes sleep(), - -- so IProcessor::work() will spend 1 sec. - name = 'ExpressionTransform', elapsed_us>=1e6, - -- SourceFromSingleChunk, that feed data to ExpressionTransform, - -- will feed first block and then wait in PortFull. - name = 'SourceFromSingleChunk', output_wait_elapsed_us>=1e6, - -- NullSource/LazyOutputFormatLazyOutputFormat are the outputs - -- so they cannot starts to execute before sleep(1) will be executed. - input_wait_elapsed_us>=1e6) - elapsed, - input_rows, - input_bytes, - output_rows, - output_bytes -FROM system.processors_profile_log -WHERE query_id = query_id_ -ORDER BY name; ExpressionTransform 1 1 1 1 1 LazyOutputFormat 1 1 1 0 0 LimitsCheckingTransform 1 1 1 1 1 diff --git a/tests/queries/0_stateless/02210_processors_profile_log.sql b/tests/queries/0_stateless/02210_processors_profile_log.sql index a15ed26fd67..59edbb71457 100644 --- a/tests/queries/0_stateless/02210_processors_profile_log.sql +++ b/tests/queries/0_stateless/02210_processors_profile_log.sql @@ -1,4 +1,3 @@ --- { echo } EXPLAIN PIPELINE SELECT sleep(1); SELECT sleep(1) SETTINGS log_processors_profiles=true, log_queries=1, log_queries_min_type='QUERY_FINISH'; @@ -15,13 +14,13 @@ SELECT multiIf( -- ExpressionTransform executes sleep(), -- so IProcessor::work() will spend 1 sec. - name = 'ExpressionTransform', elapsed_us>=1e6, + name = 'ExpressionTransform', elapsed_us>=1e6 ? 1 : elapsed_us, -- SourceFromSingleChunk, that feed data to ExpressionTransform, -- will feed first block and then wait in PortFull. - name = 'SourceFromSingleChunk', output_wait_elapsed_us>=1e6, + name = 'SourceFromSingleChunk', output_wait_elapsed_us>=1e6 ? 1 : output_wait_elapsed_us, -- NullSource/LazyOutputFormatLazyOutputFormat are the outputs -- so they cannot starts to execute before sleep(1) will be executed. - input_wait_elapsed_us>=1e6) + input_wait_elapsed_us>=1e6 ? 1 : input_wait_elapsed_us) elapsed, input_rows, input_bytes, From 7726f9e2af38120cdd69764d22db0a42fba1e6a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jun 2024 18:29:08 +0200 Subject: [PATCH 316/318] Fix reference --- .../0_stateless/01254_dict_load_after_detach_attach.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.reference b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.reference index 2f2d638a294..9c2c59f6379 100644 --- a/tests/queries/0_stateless/01254_dict_load_after_detach_attach.reference +++ b/tests/queries/0_stateless/01254_dict_load_after_detach_attach.reference @@ -1,4 +1,4 @@ -0 NOT_LOADED +NOT_LOADED 0 LOADED 10 1 LOADED From eceaa6d42aaa7ec5a6fac102264d646e983598f0 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 24 Jun 2024 12:33:00 +0200 Subject: [PATCH 317/318] Remove reverted & insignificat changes --- CHANGELOG.md | 242 +++------------------------------------------------ 1 file changed, 12 insertions(+), 230 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3f00b596a08..36a095d174e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -96,15 +96,22 @@ * Returned back the behaviour of how ClickHouse works and interprets Tuples in CSV format. This change effectively reverts https://github.com/ClickHouse/ClickHouse/pull/60994 and makes it available only under a few settings: `output_format_csv_serialize_tuple_into_separate_columns`, `input_format_csv_deserialize_separate_columns_into_tuple` and `input_format_csv_try_infer_strings_from_quoted_tuples`. [#65170](https://github.com/ClickHouse/ClickHouse/pull/65170) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Initialize global trace collector for Poco::ThreadPool (needed for keeper, etc). [#65239](https://github.com/ClickHouse/ClickHouse/pull/65239) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add validation when creating a user with bcrypt_hash. [#65242](https://github.com/ClickHouse/ClickHouse/pull/65242) ([Raúl Marín](https://github.com/Algunenano)). +* Unite s3/hdfs/azure storage implementations into a single class working with IObjectStorage. Same for *Cluster, data lakes and Queue storages. [#59767](https://github.com/ClickHouse/ClickHouse/pull/59767) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Refactor data part writer to remove dependencies on MergeTreeData and DataPart. [#63620](https://github.com/ClickHouse/ClickHouse/pull/63620) ([Alexander Gololobov](https://github.com/davenger)). +* Add profile events for number of rows read during/after prewhere. [#64198](https://github.com/ClickHouse/ClickHouse/pull/64198) ([Nikita Taranov](https://github.com/nickitat)). +* Print query in explain plan with parallel replicas. [#64298](https://github.com/ClickHouse/ClickHouse/pull/64298) ([vdimir](https://github.com/vdimir)). +* Rename `allow_deprecated_functions` to `allow_deprecated_error_prone_window_functions`. [#64358](https://github.com/ClickHouse/ClickHouse/pull/64358) ([Raúl Marín](https://github.com/Algunenano)). +* Respect `max_read_buffer_size` setting for file descriptors as well in file() table function. [#64532](https://github.com/ClickHouse/ClickHouse/pull/64532) ([Azat Khuzhin](https://github.com/azat)). +* Disable transactions for unsupported storages even for materialized views. [#64918](https://github.com/ClickHouse/ClickHouse/pull/64918) ([alesapin](https://github.com/alesapin)). +* Refactor `KeyCondition` and key analysis to improve PartitionPruner and trivial count optimization. This is separated from [#60463](https://github.com/ClickHouse/ClickHouse/issues/60463) . [#61459](https://github.com/ClickHouse/ClickHouse/pull/61459) ([Amos Bird](https://github.com/amosbird)). -#### Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +#### Bug Fix (user-visible misbehavior in an official stable release) * Fix a permission error where a user in a specific situation can escalate their privileges on the default database without necessary grants. [#64769](https://github.com/ClickHouse/ClickHouse/pull/64769) ([pufit](https://github.com/pufit)). * Fix crash with UniqInjectiveFunctionsEliminationPass and uniqCombined. [#65188](https://github.com/ClickHouse/ClickHouse/pull/65188) ([Raúl Marín](https://github.com/Algunenano)). * Fix a bug in ClickHouse Keeper that causes digest mismatch during closing session. [#65198](https://github.com/ClickHouse/ClickHouse/pull/65198) ([Aleksei Filatov](https://github.com/aalexfvk)). * Forbid `QUALIFY` clause in the old analyzer. The old analyzer ignored `QUALIFY`, so it could lead to unexpected data removal in mutations. [#65356](https://github.com/ClickHouse/ClickHouse/pull/65356) ([Dmitry Novik](https://github.com/novikd)). * Use correct memory alignment for Distinct combinator. Previously, crash could happen because of invalid memory allocation when the combinator was used. [#65379](https://github.com/ClickHouse/ClickHouse/pull/65379) ([Antonio Andelic](https://github.com/antonio2368)). - -#### Bug Fix (user-visible misbehavior in an official stable release) +* Fix crash with `DISTINCT` and window functions. [#64767](https://github.com/ClickHouse/ClickHouse/pull/64767) ([Igor Nikonov](https://github.com/devcrafter)). * Fixed 'set' skip index not working with IN and indexHint(). [#62083](https://github.com/ClickHouse/ClickHouse/pull/62083) ([Michael Kolupaev](https://github.com/al13n321)). * Support executing function during assignment of parameterized view value. [#63502](https://github.com/ClickHouse/ClickHouse/pull/63502) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). * Fixed parquet memory tracking. [#63584](https://github.com/ClickHouse/ClickHouse/pull/63584) ([Michael Kolupaev](https://github.com/al13n321)). @@ -156,7 +163,7 @@ * Respond with 5xx instead of 200 OK in case of receive timeout while reading (parts of) the request body from the client socket. [#65118](https://github.com/ClickHouse/ClickHouse/pull/65118) ([Julian Maicher](https://github.com/jmaicher)). * Fix possible crash for hedged requests. [#65206](https://github.com/ClickHouse/ClickHouse/pull/65206) ([Azat Khuzhin](https://github.com/azat)). * Fix the bug in Hashed and Hashed_Array dictionary short circuit evaluation, which may read uninitialized number, leading to various errors. [#65256](https://github.com/ClickHouse/ClickHouse/pull/65256) ([jsc0218](https://github.com/jsc0218)). -* This PR ensures that the type of the constant(IN operator's second parameter) is always visible during the IN operator's type conversion process. Otherwise, losing type information may cause some conversions to fail, such as the conversion from DateTime to Date. fix ([#64487](https://github.com/ClickHouse/ClickHouse/issues/64487)). [#65315](https://github.com/ClickHouse/ClickHouse/pull/65315) ([pn](https://github.com/chloro-pn)). +* This PR ensures that the type of the constant(IN operator's second parameter) is always visible during the IN operator's type conversion process. Otherwise, losing type information may cause some conversions to fail, such as the conversion from DateTime to Date. This fixes ([#64487](https://github.com/ClickHouse/ClickHouse/issues/64487)). [#65315](https://github.com/ClickHouse/ClickHouse/pull/65315) ([pn](https://github.com/chloro-pn)). #### Build/Testing/Packaging Improvement * Make `network` service be required when using the rc init script to start the ClickHouse server daemon. [#60650](https://github.com/ClickHouse/ClickHouse/pull/60650) ([Chun-Sheng, Li](https://github.com/peter279k)). @@ -172,233 +179,8 @@ * Support LLVM XRay on Linux amd64 only. [#64837](https://github.com/ClickHouse/ClickHouse/pull/64837) ([Tomer Shafir](https://github.com/tomershafir)). * Get rid of custom code in `tests/ci/download_release_packages.py` and `tests/ci/get_previous_release_tag.py` to avoid issues after the https://github.com/ClickHouse/ClickHouse/pull/64759 is merged. [#64848](https://github.com/ClickHouse/ClickHouse/pull/64848) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Decrease the `unit-test` image a few times. [#65102](https://github.com/ClickHouse/ClickHouse/pull/65102) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). - -#### NO CL ENTRY - -* NO CL ENTRY: 'Revert "Refactoring of Server.h: Isolate server management from other logic"'. [#64425](https://github.com/ClickHouse/ClickHouse/pull/64425) ([Alexander Tokmakov](https://github.com/tavplubix)). -* NO CL ENTRY: 'Revert "Remove some unnecessary `UNREACHABLE`s"'. [#64430](https://github.com/ClickHouse/ClickHouse/pull/64430) ([Alexander Tokmakov](https://github.com/tavplubix)). -* NO CL ENTRY: 'Revert "CI: fix build_report selection in case of job reuse"'. [#64516](https://github.com/ClickHouse/ClickHouse/pull/64516) ([Max K.](https://github.com/maxknv)). -* NO CL ENTRY: 'Revert "Revert "CI: fix build_report selection in case of job reuse""'. [#64531](https://github.com/ClickHouse/ClickHouse/pull/64531) ([Max K.](https://github.com/maxknv)). -* NO CL ENTRY: 'Revert "Add `fromReadableSize` function"'. [#64616](https://github.com/ClickHouse/ClickHouse/pull/64616) ([Robert Schulze](https://github.com/rschu1ze)). -* NO CL ENTRY: 'Update CHANGELOG.md'. [#64816](https://github.com/ClickHouse/ClickHouse/pull/64816) ([Paweł Kudzia](https://github.com/pakud)). -* NO CL ENTRY: 'Revert "Reduce lock contention for MergeTree tables (by renaming parts without holding lock)"'. [#64899](https://github.com/ClickHouse/ClickHouse/pull/64899) ([alesapin](https://github.com/alesapin)). -* NO CL ENTRY: 'Revert "Add dynamic untracked memory limits for more precise memory tracking"'. [#64969](https://github.com/ClickHouse/ClickHouse/pull/64969) ([Sergei Trifonov](https://github.com/serxa)). -* NO CL ENTRY: 'Revert "Fix duplicating Delete events in blob_storage_log"'. [#65049](https://github.com/ClickHouse/ClickHouse/pull/65049) ([Alexander Tokmakov](https://github.com/tavplubix)). -* NO CL ENTRY: 'Revert "Revert "Fix duplicating Delete events in blob_storage_log""'. [#65053](https://github.com/ClickHouse/ClickHouse/pull/65053) ([vdimir](https://github.com/vdimir)). -* NO CL ENTRY: 'Revert "S3: reduce retires time for queries, increase retries count for backups"'. [#65148](https://github.com/ClickHouse/ClickHouse/pull/65148) ([Raúl Marín](https://github.com/Algunenano)). -* NO CL ENTRY: 'Revert "Small fix for 02340_parts_refcnt_mergetree"'. [#65149](https://github.com/ClickHouse/ClickHouse/pull/65149) ([Raúl Marín](https://github.com/Algunenano)). -* NO CL ENTRY: 'Revert "Change default s3_throw_on_zero_files_match to true, document that presigned S3 URLs are not supported"'. [#65250](https://github.com/ClickHouse/ClickHouse/pull/65250) ([Max K.](https://github.com/maxknv)). -* NO CL ENTRY: 'Revert "Fix AWS ECS"'. [#65361](https://github.com/ClickHouse/ClickHouse/pull/65361) ([Alexander Tokmakov](https://github.com/tavplubix)). - -#### NOT FOR CHANGELOG / INSIGNIFICANT - -* Try abort on current thread join. [#42544](https://github.com/ClickHouse/ClickHouse/pull/42544) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* This change was reverted. [#51008](https://github.com/ClickHouse/ClickHouse/pull/51008) ([Michael Kolupaev](https://github.com/al13n321)). -* Analyzer fuzzer 2. [#57098](https://github.com/ClickHouse/ClickHouse/pull/57098) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Analyzer fuzzer 4. [#57101](https://github.com/ClickHouse/ClickHouse/pull/57101) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Check python code with flake8. [#58349](https://github.com/ClickHouse/ClickHouse/pull/58349) ([Azat Khuzhin](https://github.com/azat)). -* Unite s3/hdfs/azure storage implementations into a single class working with IObjectStorage. Same for *Cluster, data lakes and Queue storages. [#59767](https://github.com/ClickHouse/ClickHouse/pull/59767) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Refactor KeyCondition and key analysis to improve PartitionPruner and trivial count optimization. This is separated from [#60463](https://github.com/ClickHouse/ClickHouse/issues/60463) . [#61459](https://github.com/ClickHouse/ClickHouse/pull/61459) ([Amos Bird](https://github.com/amosbird)). -* This change was reverted. [#61973](https://github.com/ClickHouse/ClickHouse/pull/61973) ([Azat Khuzhin](https://github.com/azat)). -* Replay ZK logs using keeper-bench. [#62481](https://github.com/ClickHouse/ClickHouse/pull/62481) ([Antonio Andelic](https://github.com/antonio2368)). -* Reduce time-to-insert profiling data in case of logs cluster issues. [#63325](https://github.com/ClickHouse/ClickHouse/pull/63325) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Refactor data part writer to remove dependencies on MergeTreeData and DataPart. [#63620](https://github.com/ClickHouse/ClickHouse/pull/63620) ([Alexander Gololobov](https://github.com/davenger)). -* Try to fix flaky s3 tests test_seekable_formats and test_seekable_formats_url. [#63720](https://github.com/ClickHouse/ClickHouse/pull/63720) ([Kruglov Pavel](https://github.com/Avogar)). -* Rewrite plan for parallel replicas in Planner. [#63796](https://github.com/ClickHouse/ClickHouse/pull/63796) ([Igor Nikonov](https://github.com/devcrafter)). -* This PR was reverted. [#63857](https://github.com/ClickHouse/ClickHouse/pull/63857) ([Sema Checherinda](https://github.com/CheSema)). -* Make events like [timeouts](https://play.clickhouse.com/play?user=play#U0VMRUNUICogRlJPTSBjaGVja3MgV0hFUkUgdGVzdF9uYW1lID09ICdDaGVjayB0aW1lb3V0IGV4cGlyZWQnIEFORCBjaGVja19zdGFydF90aW1lIEJFVFdFRU4gdG9EYXRlKCcyMDI0LTA1LTEwJykgQU5EIHRvRGF0ZSgnMjAyNC0wNS0xNScp) visible in CI DB. [#63982](https://github.com/ClickHouse/ClickHouse/pull/63982) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Remove some unnecessary `UNREACHABLE`s. [#64035](https://github.com/ClickHouse/ClickHouse/pull/64035) ([Robert Schulze](https://github.com/rschu1ze)). -* Throw out some `inline`s. [#64110](https://github.com/ClickHouse/ClickHouse/pull/64110) ([Robert Schulze](https://github.com/rschu1ze)). -* Refactoring of Server.h: Isolate server management from other logic. [#64132](https://github.com/ClickHouse/ClickHouse/pull/64132) ([TTPO100AJIEX](https://github.com/TTPO100AJIEX)). -* Fix: 02124_insert_deduplication_token_multiple_blocks_replica. [#64181](https://github.com/ClickHouse/ClickHouse/pull/64181) ([Igor Nikonov](https://github.com/devcrafter)). -* Add profile events for number of rows read during/after prewhere. [#64198](https://github.com/ClickHouse/ClickHouse/pull/64198) ([Nikita Taranov](https://github.com/nickitat)). -* Update InterpreterCreateQuery.cpp. [#64207](https://github.com/ClickHouse/ClickHouse/pull/64207) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Remove unused storage_snapshot field from MergeTreeSelectProcessor. [#64217](https://github.com/ClickHouse/ClickHouse/pull/64217) ([Alexander Gololobov](https://github.com/davenger)). -* Add test for [#37090](https://github.com/ClickHouse/ClickHouse/issues/37090). [#64220](https://github.com/ClickHouse/ClickHouse/pull/64220) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). -* Make `settings_changes_history` const. [#64230](https://github.com/ClickHouse/ClickHouse/pull/64230) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). -* test for [#45804](https://github.com/ClickHouse/ClickHouse/issues/45804). [#64245](https://github.com/ClickHouse/ClickHouse/pull/64245) ([Denny Crane](https://github.com/den-crane)). -* Update version after release. [#64283](https://github.com/ClickHouse/ClickHouse/pull/64283) ([Raúl Marín](https://github.com/Algunenano)). -* Followup for [#63691](https://github.com/ClickHouse/ClickHouse/issues/63691). [#64285](https://github.com/ClickHouse/ClickHouse/pull/64285) ([vdimir](https://github.com/vdimir)). -* CI: dependency fix for changelog.py. [#64293](https://github.com/ClickHouse/ClickHouse/pull/64293) ([Max K.](https://github.com/maxknv)). -* Print query in explain plan with parallel replicas. [#64298](https://github.com/ClickHouse/ClickHouse/pull/64298) ([vdimir](https://github.com/vdimir)). -* CI: Cancel sync wf on new push. [#64299](https://github.com/ClickHouse/ClickHouse/pull/64299) ([Max K.](https://github.com/maxknv)). -* CI: master workflow with folded jobs. [#64340](https://github.com/ClickHouse/ClickHouse/pull/64340) ([Max K.](https://github.com/maxknv)). -* CI: Sync, Merge check, CI gh's statuses fixes. [#64348](https://github.com/ClickHouse/ClickHouse/pull/64348) ([Max K.](https://github.com/maxknv)). -* Enable 02494_query_cache_nested_query_bug for Analyzer. [#64357](https://github.com/ClickHouse/ClickHouse/pull/64357) ([Robert Schulze](https://github.com/rschu1ze)). -* Rename allow_deprecated_functions to allow_deprecated_error_prone_window_functions. [#64358](https://github.com/ClickHouse/ClickHouse/pull/64358) ([Raúl Marín](https://github.com/Algunenano)). -* Change input_format_parquet_use_native_reader to 24.6. [#64359](https://github.com/ClickHouse/ClickHouse/pull/64359) ([Raúl Marín](https://github.com/Algunenano)). -* Update description for settings `cross_join_min_rows_to_compress` and `cross_join_min_bytes_to_compress`. [#64360](https://github.com/ClickHouse/ClickHouse/pull/64360) ([Nikita Fomichev](https://github.com/fm4v)). -* Changed the unreleased setting `aggregate_function_group_array_has_limit_size` to `aggregate_function_group_array_action_when_limit_is_reached`. [#64362](https://github.com/ClickHouse/ClickHouse/pull/64362) ([Raúl Marín](https://github.com/Algunenano)). -* Split tests 03039_dynamic_all_merge_algorithms to avoid timeouts. [#64363](https://github.com/ClickHouse/ClickHouse/pull/64363) ([Kruglov Pavel](https://github.com/Avogar)). -* Try to fix GWPAsan. [#64365](https://github.com/ClickHouse/ClickHouse/pull/64365) ([Antonio Andelic](https://github.com/antonio2368)). -* CI: add secrets to reusable stage wf yml. [#64366](https://github.com/ClickHouse/ClickHouse/pull/64366) ([Max K.](https://github.com/maxknv)). -* Do not run tests tagged 'no-s3-storage-with-slow-build' with ASan. [#64367](https://github.com/ClickHouse/ClickHouse/pull/64367) ([vdimir](https://github.com/vdimir)). -* This change was reverted. [#64386](https://github.com/ClickHouse/ClickHouse/pull/64386) ([Francisco J. Jurado Moreno](https://github.com/Beetelbrox)). -* Update s3queue.md. [#64389](https://github.com/ClickHouse/ClickHouse/pull/64389) ([Kseniia Sumarokova](https://github.com/kssenii)). -* test for [#64211](https://github.com/ClickHouse/ClickHouse/issues/64211). [#64390](https://github.com/ClickHouse/ClickHouse/pull/64390) ([Denny Crane](https://github.com/den-crane)). -* Follow-up to [#59767](https://github.com/ClickHouse/ClickHouse/issues/59767). [#64398](https://github.com/ClickHouse/ClickHouse/pull/64398) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Remove wrong comment. [#64403](https://github.com/ClickHouse/ClickHouse/pull/64403) ([Sergei Trifonov](https://github.com/serxa)). -* Follow up to [#59767](https://github.com/ClickHouse/ClickHouse/issues/59767). [#64404](https://github.com/ClickHouse/ClickHouse/pull/64404) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Refactor s3 settings (move settings parsing into single place). [#64412](https://github.com/ClickHouse/ClickHouse/pull/64412) ([Kseniia Sumarokova](https://github.com/kssenii)). -* This PR was reverted. [#64423](https://github.com/ClickHouse/ClickHouse/pull/64423) ([Sergei Trifonov](https://github.com/serxa)). -* Fix test after [#64404](https://github.com/ClickHouse/ClickHouse/issues/64404). [#64432](https://github.com/ClickHouse/ClickHouse/pull/64432) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Faster TestKeeper shutdown. [#64433](https://github.com/ClickHouse/ClickHouse/pull/64433) ([Alexander Gololobov](https://github.com/davenger)). -* Remove some logging. [#64434](https://github.com/ClickHouse/ClickHouse/pull/64434) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Revert "Revert "Remove some unnecessary UNREACHABLEs"". [#64435](https://github.com/ClickHouse/ClickHouse/pull/64435) ([Robert Schulze](https://github.com/rschu1ze)). -* Clean settings in 02943_variant_read_subcolumns test. [#64437](https://github.com/ClickHouse/ClickHouse/pull/64437) ([Kruglov Pavel](https://github.com/Avogar)). -* Add a comment after [#64226](https://github.com/ClickHouse/ClickHouse/issues/64226). [#64449](https://github.com/ClickHouse/ClickHouse/pull/64449) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* CI: fix build_report selection in case of job reuse. [#64459](https://github.com/ClickHouse/ClickHouse/pull/64459) ([Max K.](https://github.com/maxknv)). -* Add Critical bugfix category in PR template. [#64480](https://github.com/ClickHouse/ClickHouse/pull/64480) ([Max K.](https://github.com/maxknv)). -* Remove `generateSnowflakeIDThreadMonotonic`. [#64499](https://github.com/ClickHouse/ClickHouse/pull/64499) ([Robert Schulze](https://github.com/rschu1ze)). -* Move analyzer attempt 2. [#64500](https://github.com/ClickHouse/ClickHouse/pull/64500) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Sync some code back from internal to public repository. [#64502](https://github.com/ClickHouse/ClickHouse/pull/64502) ([Robert Schulze](https://github.com/rschu1ze)). -* Remove `generateUUIDv7(NonMonotonic|ThreadMonotonic)` functions. [#64506](https://github.com/ClickHouse/ClickHouse/pull/64506) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix bash completion for settings. [#64521](https://github.com/ClickHouse/ClickHouse/pull/64521) ([Azat Khuzhin](https://github.com/azat)). -* Use max_read_buffer_size for file descriptors as well in file(). [#64532](https://github.com/ClickHouse/ClickHouse/pull/64532) ([Azat Khuzhin](https://github.com/azat)). -* Temporarily disable `enable_vertical_final` setting by default. This feature should not be used in older releases because it [might crash](https://github.com/ClickHouse/ClickHouse/issues/64543), but it's already fixed in 24.6 where this setting change has been reverted and `enable_vertical_final` is again enabled by default. [#64544](https://github.com/ClickHouse/ClickHouse/pull/64544) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Removed excessive calls to `flush logs` and disabled under sanitizers. [#64550](https://github.com/ClickHouse/ClickHouse/pull/64550) ([Nikita Taranov](https://github.com/nickitat)). -* Sync code moved in private repo back back to public repo. [#64551](https://github.com/ClickHouse/ClickHouse/pull/64551) ([Robert Schulze](https://github.com/rschu1ze)). -* Add support for custom type to ASTLiteral, or else the type may be lost when parse the ast. E.g. set a ASTLiteral to DataTime32 with value 19870, then it will be parsed to Int16. [#64562](https://github.com/ClickHouse/ClickHouse/pull/64562) ([shuai.xu](https://github.com/shuai-xu)). -* Add a temporary known host for git over ssh. [#64569](https://github.com/ClickHouse/ClickHouse/pull/64569) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Cache first analysis result in ReadFromMergeTree. [#64579](https://github.com/ClickHouse/ClickHouse/pull/64579) ([Igor Nikonov](https://github.com/devcrafter)). -* Derive script parameters (labels) from the --repo/--from-repo - fix to not create backports for all release branches if backport for specific branch only. [#64603](https://github.com/ClickHouse/ClickHouse/pull/64603) ([Max K.](https://github.com/maxknv)). -* Add Stateful asan test, Stateless asan, Stateless flaky chek into Required - Move binary_release to normal builds (not special). It builds fast and there are test jobs depending on it. - Add job description for A Sync. [#64605](https://github.com/ClickHouse/ClickHouse/pull/64605) ([Max K.](https://github.com/maxknv)). -* Double-checking [#59318](https://github.com/ClickHouse/ClickHouse/issues/59318) and docs for `Map`. [#64606](https://github.com/ClickHouse/ClickHouse/pull/64606) ([Robert Schulze](https://github.com/rschu1ze)). -* Update CHANGELOG.md. [#64609](https://github.com/ClickHouse/ClickHouse/pull/64609) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Tests: Convert numeric to symbolic error codes. [#64635](https://github.com/ClickHouse/ClickHouse/pull/64635) ([Robert Schulze](https://github.com/rschu1ze)). -* Move NamedCollectionsFactory into a separate file. [#64642](https://github.com/ClickHouse/ClickHouse/pull/64642) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Shuffle tests for parallel execution. [#64646](https://github.com/ClickHouse/ClickHouse/pull/64646) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* CI: Do not upload binaries for special builds in PRs. [#64653](https://github.com/ClickHouse/ClickHouse/pull/64653) ([Max K.](https://github.com/maxknv)). -* Update changelog. [#64654](https://github.com/ClickHouse/ClickHouse/pull/64654) ([Robert Schulze](https://github.com/rschu1ze)). -* Parallel replicas: simple cleanup. [#64655](https://github.com/ClickHouse/ClickHouse/pull/64655) ([Igor Nikonov](https://github.com/devcrafter)). -* Be more graceful with existing tables with `inverted` indexes. [#64656](https://github.com/ClickHouse/ClickHouse/pull/64656) ([Robert Schulze](https://github.com/rschu1ze)). -* CI: Build Report Check to verify only enabled builds. [#64669](https://github.com/ClickHouse/ClickHouse/pull/64669) ([Max K.](https://github.com/maxknv)). -* Tests: Convert error numbers to symbolic error codes, pt. II. [#64670](https://github.com/ClickHouse/ClickHouse/pull/64670) ([Robert Schulze](https://github.com/rschu1ze)). -* Split query analyzer. [#64672](https://github.com/ClickHouse/ClickHouse/pull/64672) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* By the end of CI, CI_Running status must be SUCCESS or FAILURE never PENDING. [#64693](https://github.com/ClickHouse/ClickHouse/pull/64693) ([Max K.](https://github.com/maxknv)). -* The following list of merged PRs is not present in the release branch and was added to the changelog by mistake:. [#64704](https://github.com/ClickHouse/ClickHouse/pull/64704) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* CI: MergeQueue: add binary_release and unit tests. [#64705](https://github.com/ClickHouse/ClickHouse/pull/64705) ([Max K.](https://github.com/maxknv)). -* Fix to get first good enough GH token instead of getting and comparing all of them. [#64709](https://github.com/ClickHouse/ClickHouse/pull/64709) ([Max K.](https://github.com/maxknv)). -* Check for missing Upload ID in CreateMultipartUpload reply. [#64714](https://github.com/ClickHouse/ClickHouse/pull/64714) ([Michael Kolupaev](https://github.com/al13n321)). -* Update version_date.tsv and changelogs after v24.5.1.1763-stable. [#64715](https://github.com/ClickHouse/ClickHouse/pull/64715) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Fix (unreleased) `loop()` table function crashing on empty table name. [#64716](https://github.com/ClickHouse/ClickHouse/pull/64716) ([Michael Kolupaev](https://github.com/al13n321)). -* Update CHANGELOG.md. [#64730](https://github.com/ClickHouse/ClickHouse/pull/64730) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* CI: ci.py refactoring. [#64734](https://github.com/ClickHouse/ClickHouse/pull/64734) ([Max K.](https://github.com/maxknv)). -* Return the explanation for session moved error. [#64747](https://github.com/ClickHouse/ClickHouse/pull/64747) ([Antonio Andelic](https://github.com/antonio2368)). -* It's another follow-up for https://github.com/ClickHouse/ClickHouse/pull/64039. We need to backport it to avoid issues with the new tag name. [#64759](https://github.com/ClickHouse/ClickHouse/pull/64759) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Do not try to write columns.txt if it does not exist for write-once storages. [#64762](https://github.com/ClickHouse/ClickHouse/pull/64762) ([Azat Khuzhin](https://github.com/azat)). -* Update 02482_load_parts_refcounts.sh. [#64765](https://github.com/ClickHouse/ClickHouse/pull/64765) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix crash with DISTINCT and window functions. [#64767](https://github.com/ClickHouse/ClickHouse/pull/64767) ([Igor Nikonov](https://github.com/devcrafter)). -* Fix assert in IObjectStorageIteratorAsync. [#64770](https://github.com/ClickHouse/ClickHouse/pull/64770) ([Michael Kolupaev](https://github.com/al13n321)). -* Make table functions always report engine 'StorageProxy' in system.tables. [#64771](https://github.com/ClickHouse/ClickHouse/pull/64771) ([Michael Kolupaev](https://github.com/al13n321)). -* Ask about company name on GitHub. [#64774](https://github.com/ClickHouse/ClickHouse/pull/64774) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix flaky tests about SQLite. [#64776](https://github.com/ClickHouse/ClickHouse/pull/64776) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Remove iostream debug helpers. [#64777](https://github.com/ClickHouse/ClickHouse/pull/64777) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Remove unnecessary comment. [#64785](https://github.com/ClickHouse/ClickHouse/pull/64785) ([Raúl Marín](https://github.com/Algunenano)). -* Follow-ups to some PRs. [#64787](https://github.com/ClickHouse/ClickHouse/pull/64787) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Attempt to fix 02228_merge_tree_insert_memory_usage.sql flakiness for s3. [#64800](https://github.com/ClickHouse/ClickHouse/pull/64800) ([Raúl Marín](https://github.com/Algunenano)). -* Add regression test for filter propagation through `Merge` engine. [#64806](https://github.com/ClickHouse/ClickHouse/pull/64806) ([Nikita Taranov](https://github.com/nickitat)). -* Migrate changelog.py to a descendant of fuzzywuzzy. [#64807](https://github.com/ClickHouse/ClickHouse/pull/64807) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* A follow-up for https://github.com/ClickHouse/ClickHouse/pull/64039 and [#64759](https://github.com/ClickHouse/ClickHouse/issues/64759). [#64813](https://github.com/ClickHouse/ClickHouse/pull/64813) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Make row order optimization non-experimental. [#64814](https://github.com/ClickHouse/ClickHouse/pull/64814) ([Robert Schulze](https://github.com/rschu1ze)). -* Didn't catch it at the time when all versions belonged to the current year. [#64817](https://github.com/ClickHouse/ClickHouse/pull/64817) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix clang-tidy build. [#64823](https://github.com/ClickHouse/ClickHouse/pull/64823) ([Robert Schulze](https://github.com/rschu1ze)). -* Sets all builds that we run tests on to normal build list. [#64824](https://github.com/ClickHouse/ClickHouse/pull/64824) ([Max K.](https://github.com/maxknv)). -* CI: fix CI await feature. [#64825](https://github.com/ClickHouse/ClickHouse/pull/64825) ([Max K.](https://github.com/maxknv)). -* Fix clang-tidy. [#64827](https://github.com/ClickHouse/ClickHouse/pull/64827) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Upload blob_storage_log from stateless tests. [#64843](https://github.com/ClickHouse/ClickHouse/pull/64843) ([alesapin](https://github.com/alesapin)). -* Follow-up to [#64349](https://github.com/ClickHouse/ClickHouse/issues/64349). [#64845](https://github.com/ClickHouse/ClickHouse/pull/64845) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Simplify handling of old 'inverted' indexes. [#64846](https://github.com/ClickHouse/ClickHouse/pull/64846) ([Robert Schulze](https://github.com/rschu1ze)). -* Templates defined in YAML provide more user-friendly experience. References: - [Documentation](https://docs.github.com/en/communities/using-templates-to-encourage-useful-issues-and-pull-requests/syntax-for-issue-forms) - [How it looks like in other projects. ](https://github.com/angular/angular/issues/new?assignees=&labels=&projects=&template=1-bug-report.yaml). [#64850](https://github.com/ClickHouse/ClickHouse/pull/64850) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Handle logs from rocksdb by ClickHouse internal logging. [#64856](https://github.com/ClickHouse/ClickHouse/pull/64856) ([Azat Khuzhin](https://github.com/azat)). -* Follow-up for https://github.com/ClickHouse/ClickHouse/pull/59357. [#64860](https://github.com/ClickHouse/ClickHouse/pull/64860) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* added mlock and mlockall to aspell-dict to be ignored. [#64863](https://github.com/ClickHouse/ClickHouse/pull/64863) ([Ali](https://github.com/xogoodnow)). -* A tiny fix for fancy quotes. [#64883](https://github.com/ClickHouse/ClickHouse/pull/64883) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix possible loss of "Query was cancelled" message in client. [#64888](https://github.com/ClickHouse/ClickHouse/pull/64888) ([Azat Khuzhin](https://github.com/azat)). -* We accidentally lost the way to set `PR Check` failure at some point. [#64890](https://github.com/ClickHouse/ClickHouse/pull/64890) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix global trace collector. [#64896](https://github.com/ClickHouse/ClickHouse/pull/64896) ([Antonio Andelic](https://github.com/antonio2368)). -* Fix test_mask_sensitive_info/test.py::test_create_table. [#64901](https://github.com/ClickHouse/ClickHouse/pull/64901) ([Azat Khuzhin](https://github.com/azat)). -* Update 03165_string_functions_with_token_text_indexes.sql. [#64903](https://github.com/ClickHouse/ClickHouse/pull/64903) ([Alexander Tokmakov](https://github.com/tavplubix)). -* When the branch is removed, it's impossible to get the diff by the labels. `print` in imported files spoils the `ipython` output. [#64904](https://github.com/ClickHouse/ClickHouse/pull/64904) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Disable transactions for unsupported storages even for materialized v…. [#64918](https://github.com/ClickHouse/ClickHouse/pull/64918) ([alesapin](https://github.com/alesapin)). -* additional log for cleanupDetachedTables. [#64919](https://github.com/ClickHouse/ClickHouse/pull/64919) ([Konstantin Morozov](https://github.com/k-morozov)). -* Fix tupleConcat of two empty tuples. This fixes [#64885](https://github.com/ClickHouse/ClickHouse/issues/64885). [#64923](https://github.com/ClickHouse/ClickHouse/pull/64923) ([Amos Bird](https://github.com/amosbird)). -* CI: Minor fixes in ci scripts. [#64950](https://github.com/ClickHouse/ClickHouse/pull/64950) ([Max K.](https://github.com/maxknv)). -* Fix error message (it was strange). [#64952](https://github.com/ClickHouse/ClickHouse/pull/64952) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Update fmtlib version to 9.1.0. [#64959](https://github.com/ClickHouse/ClickHouse/pull/64959) ([Duc Canh Le](https://github.com/canhld94)). -* Test 02908_many_requests_to_system_replicas makes a lot of heavy requests and it overloads server if it's an ASAN build. [#64966](https://github.com/ClickHouse/ClickHouse/pull/64966) ([Alexander Gololobov](https://github.com/davenger)). -* Fix (unreleased) bug in short circuit evaluation. [#64967](https://github.com/ClickHouse/ClickHouse/pull/64967) ([Raúl Marín](https://github.com/Algunenano)). -* Update version_date.tsv and changelogs after v24.4.2.141-stable. [#64968](https://github.com/ClickHouse/ClickHouse/pull/64968) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Fix `test_attach_partition_using_copy`. [#64977](https://github.com/ClickHouse/ClickHouse/pull/64977) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Faster processing of scheduler queue activations. [#64985](https://github.com/ClickHouse/ClickHouse/pull/64985) ([Sergei Trifonov](https://github.com/serxa)). -* CI: Fix nightly workflow. [#64987](https://github.com/ClickHouse/ClickHouse/pull/64987) ([Max K.](https://github.com/maxknv)). -* Fix innocuous data race in detectLanguage. [#64988](https://github.com/ClickHouse/ClickHouse/pull/64988) ([Raúl Marín](https://github.com/Algunenano)). -* CI: Builds in CI settings. [#64994](https://github.com/ClickHouse/ClickHouse/pull/64994) ([Max K.](https://github.com/maxknv)). -* REVERTED. [#65009](https://github.com/ClickHouse/ClickHouse/pull/65009) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). -* CI: Fix backports. [#65010](https://github.com/ClickHouse/ClickHouse/pull/65010) ([Max K.](https://github.com/maxknv)). -* Try fix 03143_prewhere_profile_events. [#65014](https://github.com/ClickHouse/ClickHouse/pull/65014) ([Nikita Taranov](https://github.com/nickitat)). -* Fix 03165_string_functions_with_token_text_indexes. [#65018](https://github.com/ClickHouse/ClickHouse/pull/65018) ([Julia Kartseva](https://github.com/jkartseva)). -* This change was reverted. [#65028](https://github.com/ClickHouse/ClickHouse/pull/65028) ([Sergei Trifonov](https://github.com/serxa)). -* Bump googletest to latest HEAD. [#65038](https://github.com/ClickHouse/ClickHouse/pull/65038) ([Robert Schulze](https://github.com/rschu1ze)). -* Improve comment about AsynchronousMetrics. [#65040](https://github.com/ClickHouse/ClickHouse/pull/65040) ([Antonio Andelic](https://github.com/antonio2368)). -* CI: Remove fuzzer build from normal CI run (bugfix). [#65041](https://github.com/ClickHouse/ClickHouse/pull/65041) ([Max K.](https://github.com/maxknv)). -* CI config refactoring [#65045](https://github.com/ClickHouse/ClickHouse/pull/65045) ([Max K.](https://github.com/maxknv)). -* Bump abseil to latest HEAD. [#65048](https://github.com/ClickHouse/ClickHouse/pull/65048) ([Robert Schulze](https://github.com/rschu1ze)). -* Capture weak_ptr of ContextAccess for safety. [#65051](https://github.com/ClickHouse/ClickHouse/pull/65051) ([Alexander Gololobov](https://github.com/davenger)). -* Stateless tests: add test for SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT. [#65056](https://github.com/ClickHouse/ClickHouse/pull/65056) ([Nikita Fomichev](https://github.com/fm4v)). -* Increase timeout in wait_for_all_mutations. [#65058](https://github.com/ClickHouse/ClickHouse/pull/65058) ([Alexander Gololobov](https://github.com/davenger)). -* Tests for _time virtual column in file alike storages. [#65064](https://github.com/ClickHouse/ClickHouse/pull/65064) ([Ilya Golshtein](https://github.com/ilejn)). -* Update odbc-bridge.md. [#65099](https://github.com/ClickHouse/ClickHouse/pull/65099) ([Alexander Gololobov](https://github.com/davenger)). -* Small fix for 02340_parts_refcnt_mergetree. [#65105](https://github.com/ClickHouse/ClickHouse/pull/65105) ([Nikita Taranov](https://github.com/nickitat)). +* Replay ZooKeeper logs using keeper-bench. [#62481](https://github.com/ClickHouse/ClickHouse/pull/62481) ([Antonio Andelic](https://github.com/antonio2368)). * Re-enable OpenSSL session caching. [#65111](https://github.com/ClickHouse/ClickHouse/pull/65111) ([Robert Schulze](https://github.com/rschu1ze)). -* Update test_replicated_database/test.py. [#65112](https://github.com/ClickHouse/ClickHouse/pull/65112) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix false positives leaky memory warnings in OpenSSL. [#65125](https://github.com/ClickHouse/ClickHouse/pull/65125) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix `Initiator received more initial requests than there are replicas` with `loop` engine. [#65133](https://github.com/ClickHouse/ClickHouse/pull/65133) ([Nikita Taranov](https://github.com/nickitat)). -* Fix 'Tasks in BackgroundSchedulePool cannot throw' caused by MergeTreeData::loadUnexpectedDataParts(). [#65135](https://github.com/ClickHouse/ClickHouse/pull/65135) ([Michael Kolupaev](https://github.com/al13n321)). -* Fix bad error message. [#65137](https://github.com/ClickHouse/ClickHouse/pull/65137) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Just fixing flaky unit tests. [#65152](https://github.com/ClickHouse/ClickHouse/pull/65152) ([Sema Checherinda](https://github.com/CheSema)). -* This change was reverted. [#65164](https://github.com/ClickHouse/ClickHouse/pull/65164) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Ensure submodules are named consistently. [#65167](https://github.com/ClickHouse/ClickHouse/pull/65167) ([Robert Schulze](https://github.com/rschu1ze)). -* Remove obsolete fix from aws submodule. [#65168](https://github.com/ClickHouse/ClickHouse/pull/65168) ([Robert Schulze](https://github.com/rschu1ze)). -* CI: Fix not-merged cherry-picks for backports. [#65181](https://github.com/ClickHouse/ClickHouse/pull/65181) ([Max K.](https://github.com/maxknv)). -* Add an assertion in ReplicatedMergeTreeQueue. [#65184](https://github.com/ClickHouse/ClickHouse/pull/65184) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix bug in unreleased code. [#65185](https://github.com/ClickHouse/ClickHouse/pull/65185) ([Raúl Marín](https://github.com/Algunenano)). -* Fix docs for skipping-indexes.md. [#65194](https://github.com/ClickHouse/ClickHouse/pull/65194) ([morning-color](https://github.com/morning-color)). -* Fix the descriptions of some server settings. [#65200](https://github.com/ClickHouse/ClickHouse/pull/65200) ([Raúl Marín](https://github.com/Algunenano)). -* Fix issue after [#64813](https://github.com/ClickHouse/ClickHouse/issues/64813) with broken search in the changelog, and missing zstd in a style-check image. [#65202](https://github.com/ClickHouse/ClickHouse/pull/65202) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix bug in unreleased code. [#65203](https://github.com/ClickHouse/ClickHouse/pull/65203) ([Raúl Marín](https://github.com/Algunenano)). -* Add test prewhere merge. [#65207](https://github.com/ClickHouse/ClickHouse/pull/65207) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Sync ProfileEvents.h. [#65208](https://github.com/ClickHouse/ClickHouse/pull/65208) ([Kseniia Sumarokova](https://github.com/kssenii)). -* FinishCheck to set failure if workflow failed. [#65228](https://github.com/ClickHouse/ClickHouse/pull/65228) ([Max K.](https://github.com/maxknv)). -* Update version_date.tsv and changelogs after v24.3.4.147-lts. [#65235](https://github.com/ClickHouse/ClickHouse/pull/65235) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Update version_date.tsv and changelogs after v24.5.3.5-stable. [#65240](https://github.com/ClickHouse/ClickHouse/pull/65240) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Fails sometimes for debug build https://s3.amazonaws.com/clickhouse-test-reports/0/af6afd904316bfb771737faa147ce8aea72dd705/stateless_tests__debug__[4_5].html. [#65245](https://github.com/ClickHouse/ClickHouse/pull/65245) ([Antonio Andelic](https://github.com/antonio2368)). -* Fix libunwind in CI. [#65247](https://github.com/ClickHouse/ClickHouse/pull/65247) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* CI: Do not skip FinishCheck in Merge Queue. [#65249](https://github.com/ClickHouse/ClickHouse/pull/65249) ([Max K.](https://github.com/maxknv)). -* Add a test just in case. [#65271](https://github.com/ClickHouse/ClickHouse/pull/65271) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Disable 02581_share_big_sets_between_multiple_mutations_tasks_long in coverage run. [#65295](https://github.com/ClickHouse/ClickHouse/pull/65295) ([Alexander Gololobov](https://github.com/davenger)). -* Update version_date.tsv and changelogs after v23.8.15.35-lts. [#65300](https://github.com/ClickHouse/ClickHouse/pull/65300) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* mute test test_query_is_canceled_with_inf_retries. [#65301](https://github.com/ClickHouse/ClickHouse/pull/65301) ([Sema Checherinda](https://github.com/CheSema)). -* Fix silly typo that caused wrong tags messages. [#65307](https://github.com/ClickHouse/ClickHouse/pull/65307) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Save server data for failed stateless tests. [#65309](https://github.com/ClickHouse/ClickHouse/pull/65309) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix 01246_buffer_flush flakiness (by tuning timeouts). [#65310](https://github.com/ClickHouse/ClickHouse/pull/65310) ([Azat Khuzhin](https://github.com/azat)). -* Remove outdated override in stress tests. [#65323](https://github.com/ClickHouse/ClickHouse/pull/65323) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix bad code in `system.session_log`. [#65332](https://github.com/ClickHouse/ClickHouse/pull/65332) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* add tests for 'boom filter index with map'. [#65333](https://github.com/ClickHouse/ClickHouse/pull/65333) ([iceFireser](https://github.com/iceFireser)). -* Fix crash in 03036_dynamic_read_subcolumns. [#65341](https://github.com/ClickHouse/ClickHouse/pull/65341) ([Kruglov Pavel](https://github.com/Avogar)). -* Move tests 02942_variant_cast and 02944_variant_as_common_type to analyzer_tech_debt.txt. [#65342](https://github.com/ClickHouse/ClickHouse/pull/65342) ([Kruglov Pavel](https://github.com/Avogar)). -* Add docs for groupConcat. [#65384](https://github.com/ClickHouse/ClickHouse/pull/65384) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). -* CI: Add Non-blocking (Woolen wolfdog) CI mode. [#65385](https://github.com/ClickHouse/ClickHouse/pull/65385) ([Max K.](https://github.com/maxknv)). -* Fix compatibility release check. [#65394](https://github.com/ClickHouse/ClickHouse/pull/65394) ([Alexey Katsman](https://github.com/alexkats)). -* Move a leaksan suppression from Poco into OpenSSL. [#65396](https://github.com/ClickHouse/ClickHouse/pull/65396) ([Robert Schulze](https://github.com/rschu1ze)). -* Fix tidy build. [#65415](https://github.com/ClickHouse/ClickHouse/pull/65415) ([Sergei Trifonov](https://github.com/serxa)). -* Remove Tests dependency on Builds_2. No tests depend on Builds_2. [#65416](https://github.com/ClickHouse/ClickHouse/pull/65416) ([Max K.](https://github.com/maxknv)). -* CI: PR workflow dependencies fix. [#65442](https://github.com/ClickHouse/ClickHouse/pull/65442) ([Max K.](https://github.com/maxknv)). -* Fix test_storage_s3_queue/test.py::test_max_set_age. [#65452](https://github.com/ClickHouse/ClickHouse/pull/65452) ([Kseniia Sumarokova](https://github.com/kssenii)). -* CI: Rename A Sync status. [#65456](https://github.com/ClickHouse/ClickHouse/pull/65456) ([Max K.](https://github.com/maxknv)). -* CI: Rename sync status. [#65464](https://github.com/ClickHouse/ClickHouse/pull/65464) ([Max K.](https://github.com/maxknv)). -* This change was reverted. [#65466](https://github.com/ClickHouse/ClickHouse/pull/65466) ([Sergei Trifonov](https://github.com/serxa)). -* Remove a feature wasn't part of any release yet. [#65480](https://github.com/ClickHouse/ClickHouse/pull/65480) ([Raúl Marín](https://github.com/Algunenano)). ### ClickHouse release 24.5, 2024-05-30 From 162fb0160cf9f27a89f7e2cff4b37f7b2a07d23c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 24 Jun 2024 12:47:58 +0200 Subject: [PATCH 318/318] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 36a095d174e..e7d50dc57dd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -14,7 +14,7 @@ #### Backward Incompatible Change * Some invalid queries will fail earlier during parsing. Note: disabled the support for inline KQL expressions (the experimental Kusto language) when they are put into a `kql` table function without a string literal, e.g. `kql(garbage | trash)` instead of `kql('garbage | trash')` or `kql($$garbage | trash$$)`. This feature was introduced unintentionally and should not exist. [#61500](https://github.com/ClickHouse/ClickHouse/pull/61500) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Rework parallel processing in `Ordered` mode of storage `S3Queue`. This PR is backward incompatible for Ordered mode if you used settings `s3queue_processing_threads_num` or `s3queue_total_shards_num`. Setting `s3queue_total_shards_num` is deleted, previously it was allowed to use only under `s3queue_allow_experimental_sharded_mode`, which is now deprecated. A new setting is added - `s3queue_buckets`. [#64349](https://github.com/ClickHouse/ClickHouse/pull/64349) ([Kseniia Sumarokova](https://github.com/kssenii)). -* New functions `snowflakeIDToDateTime`, `snowflakeIDToDateTime64`, `dateTimeToSnowflakeID`, and `dateTime64ToSnowflakeID` were added. Unlike the existing functions `snowflakeToDateTime`, `snowflakeToDateTime64`, `dateTimeToSnowflake`, and `dateTime64ToSnowflake`, the new functions are compatible with function `generateSnowflakeID`, i.e. they accept the snowflake IDs generated by `generateSnowflakeID` and produce snowflake IDs of the same type as `generateSnowflakeID` (i.e. `UInt64`). Furthermore, the new functions default to the UNIX epoch (aka. 1970-01-01), just like `generateSnowflakeID`. If necessary, a different epoch, e.g. Twitter's/X's epoch 2010-11-04 aka. 1288834974657 msec since UNIX epoch, can be passed. The old conversion functions are deprecated and will be removed after a transition period: to use them regardless, disable setting `uniform_snowflake_conversion_functions` (default: `true`). [#64948](https://github.com/ClickHouse/ClickHouse/pull/64948) ([Robert Schulze](https://github.com/rschu1ze)). +* New functions `snowflakeIDToDateTime`, `snowflakeIDToDateTime64`, `dateTimeToSnowflakeID`, and `dateTime64ToSnowflakeID` were added. Unlike the existing functions `snowflakeToDateTime`, `snowflakeToDateTime64`, `dateTimeToSnowflake`, and `dateTime64ToSnowflake`, the new functions are compatible with function `generateSnowflakeID`, i.e. they accept the snowflake IDs generated by `generateSnowflakeID` and produce snowflake IDs of the same type as `generateSnowflakeID` (i.e. `UInt64`). Furthermore, the new functions default to the UNIX epoch (aka. 1970-01-01), just like `generateSnowflakeID`. If necessary, a different epoch, e.g. Twitter's/X's epoch 2010-11-04 aka. 1288834974657 msec since UNIX epoch, can be passed. The old conversion functions are deprecated and will be removed after a transition period: to use them regardless, enable setting `allow_deprecated_snowflake_conversion_functions`. [#64948](https://github.com/ClickHouse/ClickHouse/pull/64948) ([Robert Schulze](https://github.com/rschu1ze)). #### New Feature * Introduce statistics of type "number of distinct values". [#59357](https://github.com/ClickHouse/ClickHouse/pull/59357) ([Han Fei](https://github.com/hanfei1991)).